[oozie] branch master updated: OOZIE-3526 Global job-xml not being overwritten by job-xml specified for an action (mgogineni via rohini)

2019-07-17 Thread rohini
This is an automated email from the ASF dual-hosted git repository.

rohini pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/oozie.git


The following commit(s) were added to refs/heads/master by this push:
 new 256c020  OOZIE-3526 Global job-xml not being overwritten by job-xml 
specified for an action (mgogineni via rohini)
256c020 is described below

commit 256c0208ef918cdebbfc537dd1640ac922c59efc
Author: Rohini Palaniswamy 
AuthorDate: Wed Jul 17 10:33:08 2019 -0700

OOZIE-3526 Global job-xml not being overwritten by job-xml specified for an 
action (mgogineni via rohini)
---
 .../oozie/workflow/lite/LiteWorkflowAppParser.java |  4 +-
 .../action/hadoop/ActionExecutorTestCase.java  |  9 +-
 release-log.txt|  1 +
 .../action/hadoop/TestMapReduceActionExecutor.java | 95 ++
 4 files changed, 106 insertions(+), 3 deletions(-)

diff --git 
a/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java 
b/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
index ae0cc76..1f121c5 100644
--- 
a/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
+++ 
b/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
@@ -532,6 +532,7 @@ public class LiteWorkflowAppParser {
 @SuppressWarnings("unchecked")
 List actionJobXmls = actionElement.getChildren(JOB_XML, 
actionNs);
 if (gData != null && gData.jobXmls != null) {
+int i = 0;
 for(String gJobXml : gData.jobXmls) {
 boolean alreadyExists = false;
 for (Element actionXml : actionJobXmls) {
@@ -543,7 +544,8 @@ public class LiteWorkflowAppParser {
 if (!alreadyExists) {
 Element ejobXml = new Element(JOB_XML, actionNs);
 ejobXml.setText(gJobXml);
-actionElement.addContent(ejobXml);
+actionElement.addContent(i, ejobXml);
+i++;
 }
 }
 }
diff --git 
a/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java 
b/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java
index 8b893b8..6f118bd 100644
--- 
a/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java
+++ 
b/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java
@@ -267,11 +267,16 @@ public abstract class ActionExecutorTestCase extends 
XHCatTestCase {
  * @throws Exception
  */
 protected WorkflowJobBean createBaseWorkflow(XConfiguration protoConf, 
String actionName) throws Exception {
-Path appUri = new Path(getAppPath(), "workflow.xml");
-
 String content = "";
 content += "";
 content += "";
+
+return createBaseWorkflow(protoConf, actionName, content);
+}
+
+protected WorkflowJobBean createBaseWorkflow(XConfiguration protoConf, 
String actionName, String content) throws Exception {
+Path appUri = new Path(getAppPath(), "workflow.xml");
+
 writeToFile(content, getAppPath(), "workflow.xml");
 
 WorkflowApp app = new LiteWorkflowApp("testApp", "",
diff --git a/release-log.txt b/release-log.txt
index 86693cc..bd927d4 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 5.2.0 release (trunk - unreleased)
 
+OOZIE-3526 Global job-xml not being overwritten by job-xml specified for an 
action (mgogineni via rohini)
 OOZIE-2755 Oozie HA: ZKJobsConcurrencyService throws runtime exception when 
numOozies is zero(asalamon74 via kmarton)
 OOZIE-3527 Oozie stuck in waiting state if CoordPushDependencyCheckXCommand is 
not requeued (mgogineni via rohini)
 OOZIE-3524 fs:fileSize() does not work correctly for files with extra slash in 
path (mgogineni via asalamon74)
diff --git 
a/sharelib/streaming/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
 
b/sharelib/streaming/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
index 476c9fd..75b94d1 100644
--- 
a/sharelib/streaming/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
+++ 
b/sharelib/streaming/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
@@ -22,6 +22,7 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
+import java.io.FileWriter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -227,6 +228,100 @@ public class TestMapReduceActionExecutor extends 
ActionExecutorTestCase {
  assertEquals("global-output-dir", actionConf.get("outputDir"));
 }
 
+   

[oozie] branch master updated: OOZIE-3527 Oozie stuck in waiting state if CoordPushDependencyCheckXCommand is not requeued (mgogineni via rohini)

2019-07-16 Thread rohini
This is an automated email from the ASF dual-hosted git repository.

rohini pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/oozie.git


The following commit(s) were added to refs/heads/master by this push:
 new 4e61deb  OOZIE-3527 Oozie stuck in waiting state if 
CoordPushDependencyCheckXCommand is not requeued (mgogineni via rohini)
4e61deb is described below

commit 4e61deb7598ec1ed32b5839e2a1ba978485b2cc8
Author: Rohini Palaniswamy 
AuthorDate: Tue Jul 16 15:31:37 2019 -0700

OOZIE-3527 Oozie stuck in waiting state if CoordPushDependencyCheckXCommand 
is not requeued (mgogineni via rohini)
---
 .../oozie/command/coord/CoordActionInputCheckXCommand.java   | 9 +++--
 .../oozie/command/coord/CoordPushDependencyCheckXCommand.java| 2 ++
 .../oozie/executor/jpa/CoordActionGetForCheckJPAExecutor.java| 4 
 release-log.txt  | 1 +
 4 files changed, 14 insertions(+), 2 deletions(-)

diff --git 
a/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
 
b/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
index 66875f1..b19dd50 100644
--- 
a/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
+++ 
b/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
@@ -501,8 +501,13 @@ public class CoordActionInputCheckXCommand extends 
CoordinatorXCommand {
 }
 try {
 coordAction = jpaService.execute(new 
CoordActionGetForInputCheckJPAExecutor(actionId));
-coordJob = 
CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_INPUT_CHECK,
-coordAction.getJobId());
+if (coordAction != null){
+coordJob = 
CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB_INPUT_CHECK,
+coordAction.getJobId());
+}
+else {
+throw new CommandException(ErrorCode.E0605, actionId);
+}
 }
 catch (JPAExecutorException je) {
 throw new CommandException(je);
diff --git 
a/core/src/main/java/org/apache/oozie/command/coord/CoordPushDependencyCheckXCommand.java
 
b/core/src/main/java/org/apache/oozie/command/coord/CoordPushDependencyCheckXCommand.java
index d7a143a..223c152 100644
--- 
a/core/src/main/java/org/apache/oozie/command/coord/CoordPushDependencyCheckXCommand.java
+++ 
b/core/src/main/java/org/apache/oozie/command/coord/CoordPushDependencyCheckXCommand.java
@@ -388,6 +388,8 @@ public class CoordPushDependencyCheckXCommand extends 
CoordinatorXCommand
 }
 }
 catch (JPAExecutorException je) {
+final CallableQueueService callableQueueService = 
Services.get().get(CallableQueueService.class);
+callableQueueService.queue(new 
CoordPushDependencyCheckXCommand(actionId), getCoordPushCheckRequeueInterval());
 throw new CommandException(je);
 }
 }
diff --git 
a/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionGetForCheckJPAExecutor.java
 
b/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionGetForCheckJPAExecutor.java
index 4ebe333..a84e6b6 100644
--- 
a/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionGetForCheckJPAExecutor.java
+++ 
b/core/src/main/java/org/apache/oozie/executor/jpa/CoordActionGetForCheckJPAExecutor.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Objects;
 
 import javax.persistence.EntityManager;
+import javax.persistence.NoResultException;
 import javax.persistence.Query;
 
 import org.apache.oozie.CoordinatorActionBean;
@@ -56,6 +57,9 @@ public class CoordActionGetForCheckJPAExecutor implements 
JPAExecutor

oozie git commit: OOZIE-3028 Oozie Pig Action fails with no python dependencies (dbist13 via rohini)

2017-08-10 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 03a4a5e0d -> 630171297


OOZIE-3028 Oozie Pig Action fails with no python dependencies (dbist13 via 
rohini)


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

Branch: refs/heads/master
Commit: 630171297cc75b56aafef27efdf2b9b746d86e33
Parents: 03a4a5e
Author: Rohini Palaniswamy 
Authored: Thu Aug 10 20:35:13 2017 -0700
Committer: Rohini Palaniswamy 
Committed: Thu Aug 10 20:35:13 2017 -0700

--
 pom.xml  | 4 ++--
 release-log.txt  | 1 +
 sharelib/pig/pom.xml | 2 +-
 3 files changed, 4 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/63017129/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 4886443..538702d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1331,8 +1331,8 @@
 
 
 org.python
-jython
-2.5.3
+jython-standalone
+2.7.1
 
 
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/63017129/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 2ac893c..0b509bf 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 5.0.0 release (trunk - unreleased)
 
+OOZIE-3028 Oozie Pig Action fails with no python dependencies (dbist13 via 
rohini)
 OOZIE-2670 Upgrade Hbase to 1.2 (gezapeti via asasvari)
 OOZIE-2608 Comma in oozie.service.JPAService.jdbc.password value results in 
authentication error (gezapeti via asasvari)
 OOZIE-3009 amend Number of Oozie tests executed dropped after OOZIE-2854 
(andras.piros via gezapeti)

http://git-wip-us.apache.org/repos/asf/oozie/blob/63017129/sharelib/pig/pom.xml
--
diff --git a/sharelib/pig/pom.xml b/sharelib/pig/pom.xml
index 94fba39..df6edd8 100644
--- a/sharelib/pig/pom.xml
+++ b/sharelib/pig/pom.xml
@@ -52,7 +52,7 @@
 
 
 org.python
-jython
+jython-standalone
 compile
 
 



oozie git commit: OOZIE-2742 Unable to kill applications based on tag (satishsaley via rohini)

2016-11-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 28ac95820 -> ed1e25208


OOZIE-2742 Unable to kill applications based on tag (satishsaley via rohini)


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

Branch: refs/heads/master
Commit: ed1e25208c57b8c15150feb2f49f57edfba7b340
Parents: 28ac958
Author: Rohini Palaniswamy 
Authored: Wed Nov 23 09:16:01 2016 -0800
Committer: Rohini Palaniswamy 
Committed: Wed Nov 23 09:16:01 2016 -0800

--
 .../oozie/action/hadoop/JavaActionExecutor.java |  15 ++-
 .../action/hadoop/TestJavaActionExecutor.java   | 117 +--
 .../action/hadoop/LauncherMainHadoopUtils.java  |  16 +--
 release-log.txt |   1 +
 4 files changed, 79 insertions(+), 70 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/ed1e2520/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index f315af7..4beeb96 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -25,6 +25,7 @@ import java.net.ConnectException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.UnknownHostException;
+import java.security.PrivilegedExceptionAction;
 import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -52,6 +53,7 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapred.RunningJob;
 import 
org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.DiskChecker;
@@ -69,6 +71,7 @@ import org.apache.oozie.service.HadoopAccessorService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.ShareLibService;
 import org.apache.oozie.service.URIHandlerService;
+import org.apache.oozie.service.UserGroupInformationService;
 import org.apache.oozie.service.WorkflowAppService;
 import org.apache.oozie.util.ELEvaluationException;
 import org.apache.oozie.util.ELEvaluator;
@@ -1594,7 +1597,7 @@ public class JavaActionExecutor extends ActionExecutor {
 boolean exception = false;
 try {
 Element actionXml = XmlUtils.parseXml(action.getConf());
-JobConf jobConf = createBaseHadoopConf(context, actionXml);
+final JobConf jobConf = createBaseHadoopConf(context, actionXml);
 WorkflowJob wfJob = context.getWorkflow();
 Configuration conf = null;
 if ( wfJob.getConf() != null ) {
@@ -1603,7 +1606,15 @@ public class JavaActionExecutor extends ActionExecutor {
 String launcherTag = LauncherMapperHelper.getActionYarnTag(conf, 
wfJob.getParentId(), action);
 jobConf.set(LauncherMainHadoopUtils.CHILD_MAPREDUCE_JOB_TAGS, 
LauncherMapperHelper.getTag(launcherTag));
 jobConf.set(LauncherMainHadoopUtils.OOZIE_JOB_LAUNCH_TIME, 
Long.toString(action.getStartTime().getTime()));
-LauncherMainHadoopUtils.killChildYarnJobs(jobConf);
+UserGroupInformation ugi = 
Services.get().get(UserGroupInformationService.class)
+.getProxyUser(context.getWorkflow().getUser());
+ugi.doAs(new PrivilegedExceptionAction() {
+@Override
+public Void run() throws Exception {
+LauncherMainHadoopUtils.killChildYarnJobs(jobConf);
+return null;
+}
+});
 jobClient = createJobClient(context, jobConf);
 RunningJob runningJob = getRunningJob(context, action, jobClient);
 if (runningJob != null) {

http://git-wip-us.apache.org/repos/asf/oozie/blob/ed1e2520/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java 
b/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
index 75301db..8965cdf 100644
--- 
a/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
+++ 
b/core/src/test/java/org/apache/oozie/action/had

oozie git commit: OOZIE-2742 Unable to kill applications based on tag (satishsaley via rohini)

2016-11-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.3 a73977f2b -> 70fad8055


OOZIE-2742 Unable to kill applications based on tag (satishsaley via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/70fad805
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/70fad805
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/70fad805

Branch: refs/heads/branch-4.3
Commit: 70fad8055e21b89f2db2be5a19901d2607c6be41
Parents: a73977f
Author: Rohini Palaniswamy 
Authored: Wed Nov 23 09:17:19 2016 -0800
Committer: Rohini Palaniswamy 
Committed: Wed Nov 23 09:17:19 2016 -0800

--
 .../oozie/action/hadoop/JavaActionExecutor.java |  15 ++-
 .../action/hadoop/TestJavaActionExecutor.java   | 117 +--
 .../action/hadoop/LauncherMainHadoopUtils.java  |  16 +--
 release-log.txt |   1 +
 4 files changed, 79 insertions(+), 70 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/70fad805/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index 65996d9..31d4817 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -25,6 +25,7 @@ import java.net.ConnectException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.UnknownHostException;
+import java.security.PrivilegedExceptionAction;
 import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -52,6 +53,7 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapred.RunningJob;
 import 
org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.DiskChecker;
@@ -69,6 +71,7 @@ import org.apache.oozie.service.HadoopAccessorService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.ShareLibService;
 import org.apache.oozie.service.URIHandlerService;
+import org.apache.oozie.service.UserGroupInformationService;
 import org.apache.oozie.service.WorkflowAppService;
 import org.apache.oozie.util.ELEvaluationException;
 import org.apache.oozie.util.ELEvaluator;
@@ -1590,7 +1593,7 @@ public class JavaActionExecutor extends ActionExecutor {
 boolean exception = false;
 try {
 Element actionXml = XmlUtils.parseXml(action.getConf());
-JobConf jobConf = createBaseHadoopConf(context, actionXml);
+final JobConf jobConf = createBaseHadoopConf(context, actionXml);
 WorkflowJob wfJob = context.getWorkflow();
 Configuration conf = null;
 if ( wfJob.getConf() != null ) {
@@ -1599,7 +1602,15 @@ public class JavaActionExecutor extends ActionExecutor {
 String launcherTag = LauncherMapperHelper.getActionYarnTag(conf, 
wfJob.getParentId(), action);
 jobConf.set(LauncherMainHadoopUtils.CHILD_MAPREDUCE_JOB_TAGS, 
LauncherMapperHelper.getTag(launcherTag));
 jobConf.set(LauncherMainHadoopUtils.OOZIE_JOB_LAUNCH_TIME, 
Long.toString(action.getStartTime().getTime()));
-LauncherMainHadoopUtils.killChildYarnJobs(jobConf);
+UserGroupInformation ugi = 
Services.get().get(UserGroupInformationService.class)
+.getProxyUser(context.getWorkflow().getUser());
+ugi.doAs(new PrivilegedExceptionAction() {
+@Override
+public Void run() throws Exception {
+LauncherMainHadoopUtils.killChildYarnJobs(jobConf);
+return null;
+}
+});
 jobClient = createJobClient(context, jobConf);
 RunningJob runningJob = getRunningJob(context, action, jobClient);
 if (runningJob != null) {

http://git-wip-us.apache.org/repos/asf/oozie/blob/70fad805/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java 
b/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
index 75301db..8965cdf 100644
--- 
a/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
+++ 
b/core/src/test/java/org/apache/oozie/action/had

oozie git commit: OOZIE-2720 Test failure - TestCoordMaterializeTriggerService#testMaxMatThrottleNotPicked (gezapeti via rohini)

2016-11-22 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 53554e89a -> d586b7a79


OOZIE-2720 Test failure - 
TestCoordMaterializeTriggerService#testMaxMatThrottleNotPicked (gezapeti via 
rohini)


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

Branch: refs/heads/master
Commit: d586b7a79e2e9bc52f32aa93e46fb847053a5b93
Parents: 53554e8
Author: Rohini Palaniswamy 
Authored: Tue Nov 22 10:40:22 2016 -0800
Committer: Rohini Palaniswamy 
Committed: Tue Nov 22 10:40:22 2016 -0800

--
 .../TestCoordMaterializeTriggerService.java | 55 ++--
 release-log.txt |  3 +-
 2 files changed, 41 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/d586b7a7/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
 
b/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
index 427c9b3..aadf3fb 100644
--- 
a/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
+++ 
b/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
@@ -30,6 +30,7 @@ import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.CoordinatorJob;
 import org.apache.oozie.client.CoordinatorJob.Execution;
 import org.apache.oozie.client.CoordinatorJob.Timeunit;
+import org.apache.oozie.client.Job;
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetActionsJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
@@ -40,6 +41,7 @@ import 
org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import 
org.apache.oozie.service.CoordMaterializeTriggerService.CoordMaterializeTriggerRunnable;
 import org.apache.oozie.service.UUIDService.ApplicationType;
 import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.test.XTestCase;
 import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.util.IOUtils;
 import org.apache.oozie.util.XLog;
@@ -47,12 +49,13 @@ import org.apache.oozie.util.XmlUtils;
 
 public class TestCoordMaterializeTriggerService extends XDataTestCase {
 private Services services;
-
+JPAService jpaService;
 @Override
 protected void setUp() throws Exception {
 super.setUp();
 services = new Services();
 services.init();
+jpaService = Services.get().get(JPAService.class);
 }
 
 @Override
@@ -70,17 +73,15 @@ public class TestCoordMaterializeTriggerService extends 
XDataTestCase {
  * @throws Exception
  */
 public void testCoordMaterializeTriggerService1() throws Exception {
-
 Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
 Date end = DateUtils.parseDateOozieTZ("2009-02-20T23:59Z");
 final CoordinatorJobBean job = 
addRecordToCoordJobTable(CoordinatorJob.Status.PREP, start, end, false, false, 
0);
 
-sleep(3000);
+waitForStatus(3, job, CoordinatorJob.Status.PREP);
 Runnable runnable = new CoordMaterializeTriggerRunnable(3600, 300);
 runnable.run();
-sleep(1000);
+waitForStatus(1, job, CoordinatorJob.Status.RUNNING);
 
-JPAService jpaService = Services.get().get(JPAService.class);
 CoordJobGetJPAExecutor coordGetCmd = new 
CoordJobGetJPAExecutor(job.getId());
 CoordinatorJobBean coordJob = jpaService.execute(coordGetCmd);
 assertEquals(CoordinatorJob.Status.RUNNING, coordJob.getStatus());
@@ -89,6 +90,17 @@ public class TestCoordMaterializeTriggerService extends 
XDataTestCase {
 assert (numWaitingActions <= coordJob.getMatThrottling());
 }
 
+private void waitForStatus(int timeout, final CoordinatorJobBean job, 
final CoordinatorJob.Status status) {
+waitFor(timeout, new Predicate() {
+@Override
+public boolean evaluate() throws Exception {
+CoordJobGetJPAExecutor coordGetCmd = new 
CoordJobGetJPAExecutor(job.getId());
+CoordinatorJobBean coordJob = jpaService.execute(coordGetCmd);
+return status == coordJob.getStatus();
+}
+});
+}
+
 /**
  * Test current mode. The job should be picked up for materialization.
  *
@@ -99,12 +111,11 @@ public class TestCoordMaterializeTriggerService extends 
XDataTestCase {
 Date end = new Date(start.getTime() + 3600 * 48 * 1000);
   

oozie git commit: OOZIE-2720 Test failure - TestCoordMaterializeTriggerService#testMaxMatThrottleNotPicked (gezapeti via rohini)

2016-11-22 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.3 6596475e0 -> a73977f2b


OOZIE-2720 Test failure - 
TestCoordMaterializeTriggerService#testMaxMatThrottleNotPicked (gezapeti via 
rohini)


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

Branch: refs/heads/branch-4.3
Commit: a73977f2b2639f8daaf0735d171329d8cc2dbb53
Parents: 6596475
Author: Rohini Palaniswamy 
Authored: Tue Nov 22 10:40:54 2016 -0800
Committer: Rohini Palaniswamy 
Committed: Tue Nov 22 10:40:54 2016 -0800

--
 .../TestCoordMaterializeTriggerService.java | 55 ++--
 release-log.txt |  1 +
 2 files changed, 40 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/a73977f2/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
 
b/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
index 427c9b3..aadf3fb 100644
--- 
a/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
+++ 
b/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
@@ -30,6 +30,7 @@ import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.CoordinatorJob;
 import org.apache.oozie.client.CoordinatorJob.Execution;
 import org.apache.oozie.client.CoordinatorJob.Timeunit;
+import org.apache.oozie.client.Job;
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetActionsJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
@@ -40,6 +41,7 @@ import 
org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import 
org.apache.oozie.service.CoordMaterializeTriggerService.CoordMaterializeTriggerRunnable;
 import org.apache.oozie.service.UUIDService.ApplicationType;
 import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.test.XTestCase;
 import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.util.IOUtils;
 import org.apache.oozie.util.XLog;
@@ -47,12 +49,13 @@ import org.apache.oozie.util.XmlUtils;
 
 public class TestCoordMaterializeTriggerService extends XDataTestCase {
 private Services services;
-
+JPAService jpaService;
 @Override
 protected void setUp() throws Exception {
 super.setUp();
 services = new Services();
 services.init();
+jpaService = Services.get().get(JPAService.class);
 }
 
 @Override
@@ -70,17 +73,15 @@ public class TestCoordMaterializeTriggerService extends 
XDataTestCase {
  * @throws Exception
  */
 public void testCoordMaterializeTriggerService1() throws Exception {
-
 Date start = DateUtils.parseDateOozieTZ("2009-02-01T01:00Z");
 Date end = DateUtils.parseDateOozieTZ("2009-02-20T23:59Z");
 final CoordinatorJobBean job = 
addRecordToCoordJobTable(CoordinatorJob.Status.PREP, start, end, false, false, 
0);
 
-sleep(3000);
+waitForStatus(3, job, CoordinatorJob.Status.PREP);
 Runnable runnable = new CoordMaterializeTriggerRunnable(3600, 300);
 runnable.run();
-sleep(1000);
+waitForStatus(1, job, CoordinatorJob.Status.RUNNING);
 
-JPAService jpaService = Services.get().get(JPAService.class);
 CoordJobGetJPAExecutor coordGetCmd = new 
CoordJobGetJPAExecutor(job.getId());
 CoordinatorJobBean coordJob = jpaService.execute(coordGetCmd);
 assertEquals(CoordinatorJob.Status.RUNNING, coordJob.getStatus());
@@ -89,6 +90,17 @@ public class TestCoordMaterializeTriggerService extends 
XDataTestCase {
 assert (numWaitingActions <= coordJob.getMatThrottling());
 }
 
+private void waitForStatus(int timeout, final CoordinatorJobBean job, 
final CoordinatorJob.Status status) {
+waitFor(timeout, new Predicate() {
+@Override
+public boolean evaluate() throws Exception {
+CoordJobGetJPAExecutor coordGetCmd = new 
CoordJobGetJPAExecutor(job.getId());
+CoordinatorJobBean coordJob = jpaService.execute(coordGetCmd);
+return status == coordJob.getStatus();
+}
+});
+}
+
 /**
  * Test current mode. The job should be picked up for materialization.
  *
@@ -99,12 +111,11 @@ public class TestCoordMaterializeTriggerService extends 
XDataTestCase {
 Date end = new Date(start.getTime() + 3600 * 

oozie git commit: OOZIE-2536 Hadoop's cleanup of local directory in uber mode causing failures (satishsaley via rohini)

2016-11-22 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.3 bc3b473dc -> 6596475e0


OOZIE-2536 Hadoop's cleanup of local directory in uber mode causing failures 
(satishsaley via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/6596475e
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/6596475e
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/6596475e

Branch: refs/heads/branch-4.3
Commit: 6596475e04ffd5eff7deec4216ed2f9723b83e20
Parents: bc3b473
Author: Rohini Palaniswamy 
Authored: Tue Nov 22 10:21:36 2016 -0800
Committer: Rohini Palaniswamy 
Committed: Tue Nov 22 10:21:36 2016 -0800

--
 release-log.txt   |  1 +
 .../action/hadoop/OozieLauncherOutputCommitter.java   | 14 ++
 2 files changed, 15 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/6596475e/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 3731114..b963f35 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release
 
+OOZIE-2536 Hadoop's cleanup of local directory in uber mode causing failures 
(satishsaley via rohini)
 OOZIE-2723 JSON.org license is now CatX (rkanter, abhishekbafna via shwethags)
 OOZIE-2725 Upgrade Tomcat to 6.0.47 for the latest security fixes (rkanter via 
shwethags)
 OOZIE-2724 coord:current resolves monthly/yearly dependencies incorrectly 
(satishsaley via shwethags)

http://git-wip-us.apache.org/repos/asf/oozie/blob/6596475e/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
index 153019b..84c09bb 100644
--- 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
+++ 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
@@ -19,6 +19,7 @@
 
 package org.apache.oozie.action.hadoop;
 
+import java.io.File;
 import java.io.IOException;
 
 import org.apache.hadoop.mapred.JobContext;
@@ -27,6 +28,19 @@ import org.apache.hadoop.mapred.TaskAttemptContext;
 
 public class OozieLauncherOutputCommitter extends OutputCommitter {
 
+public OozieLauncherOutputCommitter() {
+File propConf = new File(LauncherMapper.PROPAGATION_CONF_XML);
+if (!propConf.exists()) {
+try {
+propConf.createNewFile();
+}
+catch (IOException e) {
+System.out.println("Failed to create " + 
LauncherMapper.PROPAGATION_CONF_XML);
+e.printStackTrace(System.err);
+}
+}
+}
+
 @Override
 public void setupJob(JobContext jobContext) throws IOException {
 }



oozie git commit: OOZIE-2737 testConfigDefaultPropsToAction is flaky (satishsaley via rohini)

2016-11-18 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master b327fdb5b -> 1d54e476f


OOZIE-2737 testConfigDefaultPropsToAction is flaky (satishsaley via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/1d54e476
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/1d54e476
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/1d54e476

Branch: refs/heads/master
Commit: 1d54e476f7abe0317cffa526d9b42e21e2497fed
Parents: b327fdb
Author: Rohini Palaniswamy 
Authored: Fri Nov 18 12:44:12 2016 -0800
Committer: Rohini Palaniswamy 
Committed: Fri Nov 18 12:44:12 2016 -0800

--
 release-log.txt |  1 +
 .../hadoop/TestMapReduceActionExecutor.java | 23 ++--
 2 files changed, 17 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/1d54e476/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index caceaa9..acfac17 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.4.0 release (trunk - unreleased)
 
+OOZIE-2737 testConfigDefaultPropsToAction is flaky (satishsaley via rohini)
 OOZIE-2666 Support embedding Jetty into Oozie (asasvari via rkanter)
 OOZIE-1459 Remove the version in the child poms for maven-antrun-plugin (Jan 
Hentschel via rkanter)
 OOZIE-2225 Add wild card filter for gathering jobs (sai-krish,pbacsko via 
rkanter,rohini)

http://git-wip-us.apache.org/repos/asf/oozie/blob/1d54e476/sharelib/streaming/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
--
diff --git 
a/sharelib/streaming/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
 
b/sharelib/streaming/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
index 2713526..c7860be 100644
--- 
a/sharelib/streaming/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
+++ 
b/sharelib/streaming/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
@@ -33,6 +33,7 @@ import org.apache.oozie.WorkflowActionBean;
 import org.apache.oozie.WorkflowJobBean;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.WorkflowAction;
+import org.apache.oozie.client.WorkflowAction.Status;
 import org.apache.oozie.command.wf.StartXCommand;
 import org.apache.oozie.command.wf.SubmitXCommand;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor;
@@ -138,11 +139,10 @@ public class TestMapReduceActionExecutor extends 
ActionExecutorTestCase {
 
 String wfId = new SubmitXCommand(conf).call();
 new StartXCommand(wfId).call();
-sleep(3000);
+waitForWorkflowAction(wfId + "@mr-node");
 
 WorkflowActionBean mrAction = 
WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION,
 wfId + "@mr-node");
-
 // check NN and JT settings
 Element eAction = XmlUtils.parseXml(mrAction.getConf());
 Element eConf = eAction.getChild("name-node", eAction.getNamespace());
@@ -193,12 +193,12 @@ public class TestMapReduceActionExecutor extends 
ActionExecutorTestCase {
  writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
  IOUtils.copyCharStream(new StringReader(wfXml), writer);
 
- wfId = new SubmitXCommand(conf).call();
- new StartXCommand(wfId).call();
- sleep(3000);
+wfId = new SubmitXCommand(conf).call();
+new StartXCommand(wfId).call();
+waitForWorkflowAction(wfId + "@mr-node");
 
- mrAction = 
WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION,
- wfId + "@mr-node");
+mrAction = 
WorkflowActionQueryExecutor.getInstance().get(WorkflowActionQuery.GET_ACTION,
+wfId + "@mr-node");
 
  // check param
  eAction = XmlUtils.parseXml(mrAction.getConf());
@@ -1362,5 +1362,14 @@ public class TestMapReduceActionExecutor extends 
ActionExecutorTestCase {
 }
 }
 
+private void waitForWorkflowAction(final String actionId) {
+waitFor(3 * 60 * 1000, new Predicate() {
+public boolean evaluate() throws Exception {
+WorkflowActionBean mrAction = 
WorkflowActionQueryExecutor.getInstance()
+.get(WorkflowActionQuery.GET_ACTION, actionId);
+return mrAction.inTerminalState() || mrAction.getStatus() == 
Status.RUNNING;
+}
+});
+}
 
 }



oozie git commit: Fix test failures for OOZIE-2225 (pbacsko via rohini)

2016-11-18 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 3ee71d4ff -> b327fdb5b


Fix test failures for OOZIE-2225 (pbacsko via rohini)


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

Branch: refs/heads/master
Commit: b327fdb5b08cbcc797b2cf25b3e13c3310453a9b
Parents: 3ee71d4
Author: Rohini Palaniswamy 
Authored: Fri Nov 18 12:04:03 2016 -0800
Committer: Rohini Palaniswamy 
Committed: Fri Nov 18 12:04:03 2016 -0800

--
 .../executor/jpa/WorkflowsJobGetJPAExecutor.java |  1 +
 .../apache/oozie/store/StoreStatusFilter.java|  1 +
 .../jpa/TestBundleJobInfoGetJPAExecutor.java | 19 +++
 .../jpa/TestCoordJobInfoGetJPAExecutor.java  | 19 +++
 .../jpa/TestWorkflowsJobGetJPAExecutor.java  | 19 +++
 release-log.txt  |  2 +-
 6 files changed, 60 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/b327fdb5/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowsJobGetJPAExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowsJobGetJPAExecutor.java
 
b/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowsJobGetJPAExecutor.java
index 13af8f8..f50c1a4 100644
--- 
a/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowsJobGetJPAExecutor.java
+++ 
b/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowsJobGetJPAExecutor.java
@@ -278,6 +278,7 @@ public class WorkflowsJobGetJPAExecutor implements 
JPAExecutor {
 // w.id = text || w.appName.contains(text) || 
w.user.contains(text)
 else if 
(entry.getKey().equalsIgnoreCase(OozieClient.FILTER_TEXT)) {
 StoreStatusFilter.filterJobsUsingText(filter, sb, 
isEnabled, seletStr, valArray, orArray, colArray);
+isEnabled = true;
 }
 }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/b327fdb5/core/src/main/java/org/apache/oozie/store/StoreStatusFilter.java
--
diff --git a/core/src/main/java/org/apache/oozie/store/StoreStatusFilter.java 
b/core/src/main/java/org/apache/oozie/store/StoreStatusFilter.java
index b649ae5..0850b75 100644
--- a/core/src/main/java/org/apache/oozie/store/StoreStatusFilter.java
+++ b/core/src/main/java/org/apache/oozie/store/StoreStatusFilter.java
@@ -325,6 +325,7 @@ public class StoreStatusFilter {
 // job.id = text || job.appName.contains(text) || 
job.user.contains(text)
 else if 
(entry.getKey().equalsIgnoreCase(OozieClient.FILTER_TEXT)) {
 filterJobsUsingText(filter, sb, isEnabled, 
seletStr, valArray, orArray, colArray);
+isEnabled = true;
 }
 }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/b327fdb5/core/src/test/java/org/apache/oozie/executor/jpa/TestBundleJobInfoGetJPAExecutor.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/executor/jpa/TestBundleJobInfoGetJPAExecutor.java
 
b/core/src/test/java/org/apache/oozie/executor/jpa/TestBundleJobInfoGetJPAExecutor.java
index e0851f5..eb0d538 100644
--- 
a/core/src/test/java/org/apache/oozie/executor/jpa/TestBundleJobInfoGetJPAExecutor.java
+++ 
b/core/src/test/java/org/apache/oozie/executor/jpa/TestBundleJobInfoGetJPAExecutor.java
@@ -240,6 +240,25 @@ public class TestBundleJobInfoGetJPAExecutor extends 
XDataTestCase {
 assertEquals(ret.getBundleJobs().size(), 1);
 }
 
+public void testGetJobInfoForTextAndStatus() throws Exception {
+BundleJobBean bundleJob = 
addRecordToBundleJobTable(BundleJob.Status.RUNNING, false);
+bundleJob.setAppName("bundle-job-1");
+
BundleJobQueryExecutor.getInstance().executeUpdate(BundleJobQueryExecutor.BundleJobQuery.UPDATE_BUNDLE_JOB,
 bundleJob);
+
+Map> filter = new HashMap>();
+List textFilterList = new ArrayList();
+textFilterList.add("bundle-job-1");
+List textStatusList = new ArrayList();
+textStatusList.add(BundleJob.Status.RUNNING.toString());
+filter.put(OozieClient.FILTER_TEXT, textFilterList);
+filter.put(OozieClient.FILTER_STATUS, textStatusList);
+
+JPAService jpaService = Services.get().get(JPAService.class);
+BundleJobInfoGetJPAExecutor bundleInfoGetCmd = new 
BundleJobInfoGetJPAExecutor(filter, 1

oozie git commit: OOZIE-2536 Hadoop's cleanup of local directory in uber mode causing failures (satishsaley via rohini)

2016-10-31 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 7e7bdc5be -> 72ee25896


OOZIE-2536 Hadoop's cleanup of local directory in uber mode causing failures 
(satishsaley via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/72ee2589
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/72ee2589
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/72ee2589

Branch: refs/heads/master
Commit: 72ee25896a5dd8c12fdbbda0f7e3d6857685ed23
Parents: 7e7bdc5
Author: Rohini Palaniswamy 
Authored: Mon Oct 31 12:59:57 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Oct 31 12:59:57 2016 -0700

--
 release-log.txt   |  1 +
 .../action/hadoop/OozieLauncherOutputCommitter.java   | 14 ++
 2 files changed, 15 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/72ee2589/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 77cc5ad..8cb548e 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.4.0 release (trunk - unreleased)
 
+OOZIE-2536 Hadoop's cleanup of local directory in uber mode causing failures 
(satishsaley via rohini)
 OOZIE-1986 Add FindBugs report to pre-commit build (andras.piros via rkanter)
 OOZIE-2634 Queue dump command message is confusing when the queue is empty 
(andras.piros via rkanter)
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/72ee2589/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
index 153019b..84c09bb 100644
--- 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
+++ 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
@@ -19,6 +19,7 @@
 
 package org.apache.oozie.action.hadoop;
 
+import java.io.File;
 import java.io.IOException;
 
 import org.apache.hadoop.mapred.JobContext;
@@ -27,6 +28,19 @@ import org.apache.hadoop.mapred.TaskAttemptContext;
 
 public class OozieLauncherOutputCommitter extends OutputCommitter {
 
+public OozieLauncherOutputCommitter() {
+File propConf = new File(LauncherMapper.PROPAGATION_CONF_XML);
+if (!propConf.exists()) {
+try {
+propConf.createNewFile();
+}
+catch (IOException e) {
+System.out.println("Failed to create " + 
LauncherMapper.PROPAGATION_CONF_XML);
+e.printStackTrace(System.err);
+}
+}
+}
+
 @Override
 public void setupJob(JobContext jobContext) throws IOException {
 }



oozie git commit: OOZIE-2606 Set spark.yarn.jars to fix Spark 2.0 with Oozie (satishsaley via rohini) (cherry picked from commit a4dbeda9550e4bec5ef5adf0f86f4ea7be6cd155)

2016-09-30 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.3 c24c5e5df -> 4b6607c90


OOZIE-2606 Set spark.yarn.jars to fix Spark 2.0 with Oozie (satishsaley via 
rohini)
(cherry picked from commit a4dbeda9550e4bec5ef5adf0f86f4ea7be6cd155)


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

Branch: refs/heads/branch-4.3
Commit: 4b6607c90fd2b9777f95ddc89ea43b0f6e58f111
Parents: c24c5e5
Author: Rohini Palaniswamy 
Authored: Fri Sep 30 10:37:35 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Fri Sep 30 10:37:59 2016 -0700

--
 pom.xml |  24 +
 release-log.txt |   1 +
 sharelib/spark/pom.xml  |   4 +-
 .../apache/oozie/action/hadoop/SparkMain.java   | 105 ---
 .../oozie/action/hadoop/TestSparkMain.java  |  42 ++--
 5 files changed, 155 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/4b6607c9/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 08b70b6..1cbc3e0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -95,6 +95,8 @@
  
  1.4.3
  1.6.1
+ 1.6.1
+ 1.6.1
  14.0.1
  2.10
  hadoop100
@@ -1895,5 +1897,27 @@
 login
 
 
+
+spark-1
+
+true
+
+
+1.6.1
+
1.6.1
+1.6.1
+
+
+
+spark-2
+
+false
+
+
+2.0.0
+
1.6.2
+1.6.2
+
+
 
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/4b6607c9/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 34eb1ef..da1a0b3 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release
 
+OOZIE-2606 Set spark.yarn.jars to fix Spark 2.0 with Oozie (satishsaley via 
rohini)
 OOZIE-2673 Include XSD for shell-action:0.3 in documentation (abhishekbafna 
via rkanter)
 OOZIE-2194 oozie job -kill doesn't work with spark action (abhishekbafna via 
rohini)
 OOZIE-2501 ZK reentrant lock doesn't work for few cases (puru)

http://git-wip-us.apache.org/repos/asf/oozie/blob/4b6607c9/sharelib/spark/pom.xml
--
diff --git a/sharelib/spark/pom.xml b/sharelib/spark/pom.xml
index bfce949..d828113 100644
--- a/sharelib/spark/pom.xml
+++ b/sharelib/spark/pom.xml
@@ -202,13 +202,13 @@
 
 org.apache.spark
 
spark-streaming-kafka_${spark.scala.binary.version}
-${spark.version}
+${spark.streaming.kafka.version}
 compile
 
 
 org.apache.spark
 spark-bagel_${spark.scala.binary.version}
-${spark.version}
+${spark.bagel.version}
 compile
 
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/4b6607c9/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
--
diff --git 
a/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java 
b/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
index 407ba4b..539fb5c 100644
--- a/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
+++ b/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
@@ -18,14 +18,6 @@
 
 package org.apache.oozie.action.hadoop;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.filecache.DistributedCache;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.log4j.PropertyConfigurator;
-import org.apache.spark.deploy.SparkSubmit;
-
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -34,10 +26,23 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Properties;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
 import java.util.regex.Pattern;
 
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configura

oozie git commit: OOZIE-2606 Set spark.yarn.jars to fix Spark 2.0 with Oozie (satishsaley via rohini)

2016-09-30 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 3eca3c2bd -> a4dbeda95


OOZIE-2606 Set spark.yarn.jars to fix Spark 2.0 with Oozie (satishsaley via 
rohini)


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

Branch: refs/heads/master
Commit: a4dbeda9550e4bec5ef5adf0f86f4ea7be6cd155
Parents: 3eca3c2
Author: Rohini Palaniswamy 
Authored: Fri Sep 30 10:37:35 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Fri Sep 30 10:37:35 2016 -0700

--
 pom.xml |  24 +
 release-log.txt |   1 +
 sharelib/spark/pom.xml  |   4 +-
 .../apache/oozie/action/hadoop/SparkMain.java   | 105 ---
 .../oozie/action/hadoop/TestSparkMain.java  |  42 ++--
 5 files changed, 155 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/a4dbeda9/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 704a2ee..29184b2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -95,6 +95,8 @@
  
  1.4.3
  1.6.1
+ 1.6.1
+ 1.6.1
  14.0.1
  2.10
  hadoop100
@@ -1895,5 +1897,27 @@
 login
 
 
+
+spark-1
+
+true
+
+
+1.6.1
+
1.6.1
+1.6.1
+
+
+
+spark-2
+
+false
+
+
+2.0.0
+
1.6.2
+1.6.2
+
+
 
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/a4dbeda9/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 13c11df..1579289 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -3,6 +3,7 @@
 
 -- Oozie 4.3.0 release
 
+OOZIE-2606 Set spark.yarn.jars to fix Spark 2.0 with Oozie (satishsaley via 
rohini)
 OOZIE-2673 Include XSD for shell-action:0.3 in documentation (abhishekbafna 
via rkanter)
 OOZIE-2194 oozie job -kill doesn't work with spark action (abhishekbafna via 
rohini)
 OOZIE-2501 ZK reentrant lock doesn't work for few cases (puru)

http://git-wip-us.apache.org/repos/asf/oozie/blob/a4dbeda9/sharelib/spark/pom.xml
--
diff --git a/sharelib/spark/pom.xml b/sharelib/spark/pom.xml
index 905b9b7..fd783b3 100644
--- a/sharelib/spark/pom.xml
+++ b/sharelib/spark/pom.xml
@@ -202,13 +202,13 @@
 
 org.apache.spark
 
spark-streaming-kafka_${spark.scala.binary.version}
-${spark.version}
+${spark.streaming.kafka.version}
 compile
 
 
 org.apache.spark
 spark-bagel_${spark.scala.binary.version}
-${spark.version}
+${spark.bagel.version}
 compile
 
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/a4dbeda9/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
--
diff --git 
a/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java 
b/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
index 407ba4b..539fb5c 100644
--- a/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
+++ b/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
@@ -18,14 +18,6 @@
 
 package org.apache.oozie.action.hadoop;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.filecache.DistributedCache;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.log4j.PropertyConfigurator;
-import org.apache.spark.deploy.SparkSubmit;
-
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -34,10 +26,23 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Properties;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
 import java.util.regex.Pattern;
 
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.filecache.DistributedCache;
+import org.apache.hadoop.fs.FileSy

oozie git commit: OOZIE-2194 oozie job -kill doesn't work with spark action (abhishekbafna via rohini) (cherry picked from commit 376cdb3d8cd4f3a5c2c3f3e771df19c50e57b8f4)

2016-09-27 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.3 5731d9315 -> a46d09283


OOZIE-2194 oozie job -kill doesn't work with spark action (abhishekbafna via 
rohini)
(cherry picked from commit 376cdb3d8cd4f3a5c2c3f3e771df19c50e57b8f4)


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

Branch: refs/heads/branch-4.3
Commit: a46d09283af194e0e7b51284499215e7b8cf52e5
Parents: 5731d93
Author: Rohini Palaniswamy 
Authored: Tue Sep 27 16:22:39 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Sep 27 16:23:05 2016 -0700

--
 release-log.txt | 1 +
 .../src/main/java/org/apache/oozie/action/hadoop/JavaMain.java  | 1 +
 .../main/java/org/apache/oozie/action/hadoop/LauncherMain.java  | 2 +-
 .../src/main/java/org/apache/oozie/action/hadoop/ShellMain.java | 1 +
 .../src/main/java/org/apache/oozie/action/hadoop/SparkMain.java | 5 +
 5 files changed, 9 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/a46d0928/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 32c59a4..aeea660 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release
 
+OOZIE-2194 oozie job -kill doesn't work with spark action (abhishekbafna via 
rohini)
 OOZIE-2501 ZK reentrant lock doesn't work for few cases (puru)
 OOZIE-2582 Populating external child Ids for action failures (abhishekbafna 
via rohini)
 OOZIE-2678 Oozie job -kill doesn't work with tez jobs (abhishekbafna via 
rohini)

http://git-wip-us.apache.org/repos/asf/oozie/blob/a46d0928/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
index 16bd4e6..30d68e2 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
@@ -42,6 +42,7 @@ public class JavaMain extends LauncherMain {
 
 setYarnTag(actionConf);
 setApplicationTags(actionConf, TEZ_APPLICATION_TAGS);
+setApplicationTags(actionConf, SPARK_YARN_TAGS);
 
 LauncherMainHadoopUtils.killChildYarnJobs(actionConf);
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/a46d0928/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
index 815f60b..785ca5e 100644
--- 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
+++ 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
@@ -31,7 +31,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedHashSet;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -52,6 +51,7 @@ public abstract class LauncherMain {
 public static final String HADOOP_JOBS = "hadoopJobs";
 public static final String MAPREDUCE_JOB_TAGS = "mapreduce.job.tags";
 public static final String TEZ_APPLICATION_TAGS = "tez.application.tags";
+public static final String SPARK_YARN_TAGS = "spark.yarn.tags";
 protected static String[] HADOOP_SITE_FILES = new String[]
 {"core-site.xml", "hdfs-site.xml", "mapred-site.xml", 
"yarn-site.xml"};
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/a46d0928/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
index 1e9d8af..f109318 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
@@ -65,6 +65,7 @@ public class ShellMain extends LauncherMain {
 Configuration actionConf = loadActionConf();
 setYarnTag(actionConf);
 setApplicationTags(actionConf, TEZ_APPLICATION_TAGS);
+setApplication

oozie git commit: OOZIE-2194 oozie job -kill doesn't work with spark action (abhishekbafna via rohini)

2016-09-27 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master d330d4066 -> 376cdb3d8


OOZIE-2194 oozie job -kill doesn't work with spark action (abhishekbafna via 
rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/376cdb3d
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/376cdb3d
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/376cdb3d

Branch: refs/heads/master
Commit: 376cdb3d8cd4f3a5c2c3f3e771df19c50e57b8f4
Parents: d330d40
Author: Rohini Palaniswamy 
Authored: Tue Sep 27 16:22:39 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Sep 27 16:22:39 2016 -0700

--
 release-log.txt | 1 +
 .../src/main/java/org/apache/oozie/action/hadoop/JavaMain.java  | 1 +
 .../main/java/org/apache/oozie/action/hadoop/LauncherMain.java  | 2 +-
 .../src/main/java/org/apache/oozie/action/hadoop/ShellMain.java | 1 +
 .../src/main/java/org/apache/oozie/action/hadoop/SparkMain.java | 5 +
 5 files changed, 9 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/376cdb3d/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index b03a61a..1ca7e2e 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -3,6 +3,7 @@
 
 -- Oozie 4.3.0 release
 
+OOZIE-2194 oozie job -kill doesn't work with spark action (abhishekbafna via 
rohini)
 OOZIE-2501 ZK reentrant lock doesn't work for few cases (puru)
 OOZIE-2582 Populating external child Ids for action failures (abhishekbafna 
via rohini)
 OOZIE-2678 Oozie job -kill doesn't work with tez jobs (abhishekbafna via 
rohini)

http://git-wip-us.apache.org/repos/asf/oozie/blob/376cdb3d/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
index 16bd4e6..30d68e2 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
@@ -42,6 +42,7 @@ public class JavaMain extends LauncherMain {
 
 setYarnTag(actionConf);
 setApplicationTags(actionConf, TEZ_APPLICATION_TAGS);
+setApplicationTags(actionConf, SPARK_YARN_TAGS);
 
 LauncherMainHadoopUtils.killChildYarnJobs(actionConf);
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/376cdb3d/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
index 815f60b..785ca5e 100644
--- 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
+++ 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
@@ -31,7 +31,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedHashSet;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -52,6 +51,7 @@ public abstract class LauncherMain {
 public static final String HADOOP_JOBS = "hadoopJobs";
 public static final String MAPREDUCE_JOB_TAGS = "mapreduce.job.tags";
 public static final String TEZ_APPLICATION_TAGS = "tez.application.tags";
+public static final String SPARK_YARN_TAGS = "spark.yarn.tags";
 protected static String[] HADOOP_SITE_FILES = new String[]
 {"core-site.xml", "hdfs-site.xml", "mapred-site.xml", 
"yarn-site.xml"};
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/376cdb3d/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
index 1e9d8af..f109318 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
@@ -65,6 +65,7 @@ public class ShellMain extends LauncherMain {
 Configuration actionConf = loadActionConf();
 setYarnTag(actionConf);
 setApplicationTags(actionConf, TEZ_APPLICATION_TAGS);
+setApplicationTags(actionConf, SPARK_YARN_TAGS);
 
 int exitCode = execute

oozie git commit: OOZIE-2582 Populating external child Ids for action failures (abhishekbafna via rohini) (cherry picked from commit e8a9b24b9916df20cfbc1c48e5e1a755151d8245)

2016-09-27 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.3 3a7ca92d4 -> 3efc1fd8a


OOZIE-2582 Populating external child Ids for action failures (abhishekbafna via 
rohini)
(cherry picked from commit e8a9b24b9916df20cfbc1c48e5e1a755151d8245)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/3efc1fd8
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/3efc1fd8
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/3efc1fd8

Branch: refs/heads/branch-4.3
Commit: 3efc1fd8ae97e5b4768f7e5199aa18d92bb76ded
Parents: 3a7ca92
Author: Rohini Palaniswamy 
Authored: Tue Sep 27 12:05:19 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Sep 27 12:05:52 2016 -0700

--
 .../action/hadoop/DistcpActionExecutor.java | 27 +--
 .../action/hadoop/Hive2ActionExecutor.java  | 29 ++-
 .../oozie/action/hadoop/HiveActionExecutor.java | 29 ++-
 .../oozie/action/hadoop/JavaActionExecutor.java | 21 +++---
 .../action/hadoop/SparkActionExecutor.java  | 25 +--
 .../action/hadoop/SqoopActionExecutor.java  | 41 ++
 .../oozie/action/hadoop/TestDistcpMain.java |  2 +
 examples/src/main/apps/hive/script.q|  1 +
 release-log.txt |  1 +
 .../apache/oozie/action/hadoop/DistcpMain.java  |  9 ++-
 .../apache/oozie/action/hadoop/HiveMain.java|  5 +-
 .../action/hadoop/TestHiveActionExecutor.java   | 15 +---
 .../oozie/action/hadoop/TestHiveMain.java   |  8 +-
 .../apache/oozie/action/hadoop/Hive2Main.java   |  3 +-
 .../action/hadoop/TestHive2ActionExecutor.java  | 57 +++---
 .../oozie/action/hadoop/LauncherMain.java   | 79 ++--
 .../apache/oozie/action/hadoop/ShellMain.java   |  2 +-
 .../org/apache/oozie/action/hadoop/PigMain.java |  8 +-
 .../oozie/action/hadoop/PigMainWithOldAPI.java  |  2 +-
 .../apache/oozie/action/hadoop/SparkMain.java   |  1 +
 .../apache/oozie/action/hadoop/SqoopMain.java   | 16 ++--
 .../action/hadoop/TestSqoopActionExecutor.java  | 22 +-
 22 files changed, 158 insertions(+), 245 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/3efc1fd8/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
index 96726da..78cd257 100644
--- 
a/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
+++ 
b/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
@@ -18,24 +18,15 @@
 
 package org.apache.oozie.action.hadoop;
 
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.RunningJob;
 import org.apache.oozie.action.ActionExecutorException;
-import org.apache.oozie.action.ActionExecutor.Context;
-import org.apache.oozie.client.WorkflowAction;
 import org.apache.oozie.service.ConfigurationService;
-import org.apache.oozie.service.HadoopAccessorException;
-import org.apache.oozie.service.Services;
 import org.apache.oozie.util.XLog;
 import org.jdom.Element;
-import org.jdom.JDOMException;
+
+import java.util.ArrayList;
+import java.util.List;
 
 public class DistcpActionExecutor extends JavaActionExecutor{
 public static final String CONF_OOZIE_DISTCP_ACTION_MAIN_CLASS = 
"org.apache.oozie.action.hadoop.DistcpMain";
@@ -124,16 +115,4 @@ public class DistcpActionExecutor extends 
JavaActionExecutor{
 protected String getLauncherMain(Configuration launcherConf, Element 
actionXml) {
 return launcherConf.get(LauncherMapper.CONF_OOZIE_ACTION_MAIN_CLASS, 
CONF_OOZIE_DISTCP_ACTION_MAIN_CLASS);
 }
-
-@Override
-protected void getActionData(FileSystem actionFs, RunningJob runningJob, 
WorkflowAction action, Context context)
-throws HadoopAccessorException, JDOMException, IOException, 
URISyntaxException {
-super.getActionData(actionFs, runningJob, action, context);
-readExternalChildIDs(action, context);
-}
-
-@Override
-protected boolean getCaptureOutput(WorkflowAction action) throws 
JDOMException {
-return true;
-}
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/3efc1fd8/core/src/main/java/org/apache/oozie/action/hadoop/Hive2ActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/Hive2ActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/Hive2Action

oozie git commit: OOZIE-2582 Populating external child Ids for action failures (abhishekbafna via rohini)

2016-09-27 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 76b60cc82 -> e8a9b24b9


OOZIE-2582 Populating external child Ids for action failures (abhishekbafna via 
rohini)


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

Branch: refs/heads/master
Commit: e8a9b24b9916df20cfbc1c48e5e1a755151d8245
Parents: 76b60cc
Author: Rohini Palaniswamy 
Authored: Tue Sep 27 12:05:19 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Sep 27 12:05:19 2016 -0700

--
 .../action/hadoop/DistcpActionExecutor.java | 27 +--
 .../action/hadoop/Hive2ActionExecutor.java  | 29 ++-
 .../oozie/action/hadoop/HiveActionExecutor.java | 29 ++-
 .../oozie/action/hadoop/JavaActionExecutor.java | 21 +++---
 .../action/hadoop/SparkActionExecutor.java  | 25 +--
 .../action/hadoop/SqoopActionExecutor.java  | 41 ++
 .../oozie/action/hadoop/TestDistcpMain.java |  2 +
 examples/src/main/apps/hive/script.q|  1 +
 release-log.txt |  1 +
 .../apache/oozie/action/hadoop/DistcpMain.java  |  9 ++-
 .../apache/oozie/action/hadoop/HiveMain.java|  5 +-
 .../action/hadoop/TestHiveActionExecutor.java   | 15 +---
 .../oozie/action/hadoop/TestHiveMain.java   |  8 +-
 .../apache/oozie/action/hadoop/Hive2Main.java   |  3 +-
 .../action/hadoop/TestHive2ActionExecutor.java  | 57 +++---
 .../oozie/action/hadoop/LauncherMain.java   | 79 ++--
 .../apache/oozie/action/hadoop/ShellMain.java   |  2 +-
 .../org/apache/oozie/action/hadoop/PigMain.java |  8 +-
 .../oozie/action/hadoop/PigMainWithOldAPI.java  |  2 +-
 .../apache/oozie/action/hadoop/SparkMain.java   |  1 +
 .../apache/oozie/action/hadoop/SqoopMain.java   | 16 ++--
 .../action/hadoop/TestSqoopActionExecutor.java  | 22 +-
 22 files changed, 158 insertions(+), 245 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/e8a9b24b/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
index 96726da..78cd257 100644
--- 
a/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
+++ 
b/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
@@ -18,24 +18,15 @@
 
 package org.apache.oozie.action.hadoop;
 
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.RunningJob;
 import org.apache.oozie.action.ActionExecutorException;
-import org.apache.oozie.action.ActionExecutor.Context;
-import org.apache.oozie.client.WorkflowAction;
 import org.apache.oozie.service.ConfigurationService;
-import org.apache.oozie.service.HadoopAccessorException;
-import org.apache.oozie.service.Services;
 import org.apache.oozie.util.XLog;
 import org.jdom.Element;
-import org.jdom.JDOMException;
+
+import java.util.ArrayList;
+import java.util.List;
 
 public class DistcpActionExecutor extends JavaActionExecutor{
 public static final String CONF_OOZIE_DISTCP_ACTION_MAIN_CLASS = 
"org.apache.oozie.action.hadoop.DistcpMain";
@@ -124,16 +115,4 @@ public class DistcpActionExecutor extends 
JavaActionExecutor{
 protected String getLauncherMain(Configuration launcherConf, Element 
actionXml) {
 return launcherConf.get(LauncherMapper.CONF_OOZIE_ACTION_MAIN_CLASS, 
CONF_OOZIE_DISTCP_ACTION_MAIN_CLASS);
 }
-
-@Override
-protected void getActionData(FileSystem actionFs, RunningJob runningJob, 
WorkflowAction action, Context context)
-throws HadoopAccessorException, JDOMException, IOException, 
URISyntaxException {
-super.getActionData(actionFs, runningJob, action, context);
-readExternalChildIDs(action, context);
-}
-
-@Override
-protected boolean getCaptureOutput(WorkflowAction action) throws 
JDOMException {
-return true;
-}
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/e8a9b24b/core/src/main/java/org/apache/oozie/action/hadoop/Hive2ActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/Hive2ActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/Hive2ActionExecutor.java
index b5b1bf9..2aed936 100644
--- a/core/src/main/java/org/apache

oozie git commit: OOZIE-2678 Oozie job -kill doesn't work with tez jobs (abhishekbafna via rohini) (cherry picked from commit 76b60cc82840c0f66b0b3a75fd01aff2f57d27ec)

2016-09-27 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.3 e2fbc2416 -> 3a7ca92d4


OOZIE-2678 Oozie job -kill doesn't work with tez jobs (abhishekbafna via rohini)
(cherry picked from commit 76b60cc82840c0f66b0b3a75fd01aff2f57d27ec)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/3a7ca92d
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/3a7ca92d
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/3a7ca92d

Branch: refs/heads/branch-4.3
Commit: 3a7ca92d421aa97485ec84db75b69d68eccd961c
Parents: e2fbc24
Author: Rohini Palaniswamy 
Authored: Tue Sep 27 11:56:32 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Sep 27 11:59:18 2016 -0700

--
 release-log.txt  | 1 +
 .../main/java/org/apache/oozie/action/hadoop/HiveMain.java   | 1 +
 .../main/java/org/apache/oozie/action/hadoop/JavaMain.java   | 1 +
 .../java/org/apache/oozie/action/hadoop/LauncherMain.java| 8 
 .../main/java/org/apache/oozie/action/hadoop/ShellMain.java  | 2 ++
 .../main/java/org/apache/oozie/action/hadoop/PigMain.java| 1 +
 6 files changed, 14 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/3a7ca92d/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 84e454d..0a6e394 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release
 
+OOZIE-2678 Oozie job -kill doesn't work with tez jobs (abhishekbafna via 
rohini)
 OOZIE-2676 Make hadoop-2 as the default profile (gezapeti via rkanter)
 OOZIE-2487 Temporary workaround for Java 8 Javadoc errors (abhishekbafna via 
rkanter)
 OOZIE-1978 Forkjoin validation code is ridiculously slow in some cases 
(pbacsko via rkanter)

http://git-wip-us.apache.org/repos/asf/oozie/blob/3a7ca92d/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
--
diff --git 
a/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java 
b/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
index 8de3766..5866117 100644
--- a/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
+++ b/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
@@ -87,6 +87,7 @@ public class HiveMain extends LauncherMain {
 hiveConf.addResource(new Path("file:///", actionXml));
 
 setYarnTag(hiveConf);
+setApplicationTags(hiveConf, TEZ_APPLICATION_TAGS);
 
 // Propagate delegation related props from launcher job to Hive job
 String delegationToken = 
getFilePathFromEnv("HADOOP_TOKEN_FILE_LOCATION");

http://git-wip-us.apache.org/repos/asf/oozie/blob/3a7ca92d/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
index e4f4b43..16bd4e6 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
@@ -41,6 +41,7 @@ public class JavaMain extends LauncherMain {
 Configuration actionConf = loadActionConf();
 
 setYarnTag(actionConf);
+setApplicationTags(actionConf, TEZ_APPLICATION_TAGS);
 
 LauncherMainHadoopUtils.killChildYarnJobs(actionConf);
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/3a7ca92d/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
index 054f8ea..7c5713c 100644
--- 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
+++ 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
@@ -44,6 +44,7 @@ public abstract class LauncherMain {
 
 public static final String HADOOP_JOBS = "hadoopJobs";
 public static final String MAPREDUCE_JOB_TAGS = "mapreduce.job.tags";
+public static final String TEZ_APPLICATION_TAGS = "tez.application.tags";
 protected static String[] HADOOP_SITE_FILES = new String[]
 {"core-site.xml", "hdfs-site.xml", "mapred-site.xml", 
"yarn-site.xml"};
 
@@ -216,6 +217,13 @@ public abstract class LauncherMain {
 }
 }
 
+protected static void setApplicationTags(Con

oozie git commit: OOZIE-2678 Oozie job -kill doesn't work with tez jobs (abhishekbafna via rohini)

2016-09-27 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master a7d500e88 -> 76b60cc82


OOZIE-2678 Oozie job -kill doesn't work with tez jobs (abhishekbafna via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/76b60cc8
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/76b60cc8
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/76b60cc8

Branch: refs/heads/master
Commit: 76b60cc82840c0f66b0b3a75fd01aff2f57d27ec
Parents: a7d500e
Author: Rohini Palaniswamy 
Authored: Tue Sep 27 11:56:32 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Sep 27 11:56:32 2016 -0700

--
 release-log.txt  | 1 +
 .../main/java/org/apache/oozie/action/hadoop/HiveMain.java   | 1 +
 .../main/java/org/apache/oozie/action/hadoop/JavaMain.java   | 1 +
 .../java/org/apache/oozie/action/hadoop/LauncherMain.java| 8 
 .../main/java/org/apache/oozie/action/hadoop/ShellMain.java  | 2 ++
 .../main/java/org/apache/oozie/action/hadoop/PigMain.java| 1 +
 6 files changed, 14 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/76b60cc8/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 4805f84..ef11bb3 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -3,6 +3,7 @@
 
 -- Oozie 4.3.0 release
 
+OOZIE-2678 Oozie job -kill doesn't work with tez jobs (abhishekbafna via 
rohini)
 OOZIE-2676 Make hadoop-2 as the default profile (gezapeti via rkanter)
 OOZIE-2487 Temporary workaround for Java 8 Javadoc errors (abhishekbafna via 
rkanter)
 OOZIE-1978 Forkjoin validation code is ridiculously slow in some cases 
(pbacsko via rkanter)

http://git-wip-us.apache.org/repos/asf/oozie/blob/76b60cc8/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
--
diff --git 
a/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java 
b/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
index 8de3766..5866117 100644
--- a/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
+++ b/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
@@ -87,6 +87,7 @@ public class HiveMain extends LauncherMain {
 hiveConf.addResource(new Path("file:///", actionXml));
 
 setYarnTag(hiveConf);
+setApplicationTags(hiveConf, TEZ_APPLICATION_TAGS);
 
 // Propagate delegation related props from launcher job to Hive job
 String delegationToken = 
getFilePathFromEnv("HADOOP_TOKEN_FILE_LOCATION");

http://git-wip-us.apache.org/repos/asf/oozie/blob/76b60cc8/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
index e4f4b43..16bd4e6 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/JavaMain.java
@@ -41,6 +41,7 @@ public class JavaMain extends LauncherMain {
 Configuration actionConf = loadActionConf();
 
 setYarnTag(actionConf);
+setApplicationTags(actionConf, TEZ_APPLICATION_TAGS);
 
 LauncherMainHadoopUtils.killChildYarnJobs(actionConf);
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/76b60cc8/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
index 054f8ea..7c5713c 100644
--- 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
+++ 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
@@ -44,6 +44,7 @@ public abstract class LauncherMain {
 
 public static final String HADOOP_JOBS = "hadoopJobs";
 public static final String MAPREDUCE_JOB_TAGS = "mapreduce.job.tags";
+public static final String TEZ_APPLICATION_TAGS = "tez.application.tags";
 protected static String[] HADOOP_SITE_FILES = new String[]
 {"core-site.xml", "hdfs-site.xml", "mapred-site.xml", 
"yarn-site.xml"};
 
@@ -216,6 +217,13 @@ public abstract class LauncherMain {
 }
 }
 
+protected static void setApplicationTags(Configuration configName, String 
tagConfigName) {
+if (configName.

oozie git commit: OOZIE-2498 Oozie CallerId configuration for downstream components (abhishekbafna via rohini)

2016-09-19 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 3bd45e924 -> 3ad35372e


OOZIE-2498 Oozie CallerId configuration for downstream components 
(abhishekbafna via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/3ad35372
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/3ad35372
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/3ad35372

Branch: refs/heads/master
Commit: 3ad35372e20d9e995688d52548f7858537f22be3
Parents: 3bd45e9
Author: Rohini Palaniswamy 
Authored: Mon Sep 19 10:20:44 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Sep 19 10:20:44 2016 -0700

--
 release-log.txt | 1 +
 .../src/main/java/org/apache/oozie/action/hadoop/HiveMain.java  | 5 +
 .../src/main/java/org/apache/oozie/action/hadoop/PigMain.java   | 4 
 3 files changed, 10 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/3ad35372/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index a8afcf4..515521c 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2498 Oozie CallerId configuration for downstream components 
(abhishekbafna via rohini)
 OOZIE-2491 oozie acl cannot specify group,it does`t work (abhishekbafna via 
rohini)
 OOZIE-2569 Adding yarn-site, core-site, hdfs-site and mapred-site into spark 
launcher (abhishekbafna via rohini)
 OOZIE-2675 Drop support for hadoop 0.23 (abhishekbafna via rohini)

http://git-wip-us.apache.org/repos/asf/oozie/blob/3ad35372/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
--
diff --git 
a/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java 
b/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
index a2c75dc..8de3766 100644
--- a/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
+++ b/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
@@ -224,6 +224,11 @@ public class HiveMain extends LauncherMain {
 arguments.add("--hiveconf");
 arguments.add("hive.exec.log4j.file=" + new 
File(HIVE_EXEC_L4J_PROPS).getAbsolutePath());
 
+//setting oozie workflow id as caller context id for hive
+String callerId = "oozie:" + System.getProperty("oozie.job.id");
+arguments.add("--hiveconf");
+arguments.add("hive.log.trace.id=" + callerId);
+
 String scriptPath = hiveConf.get(HiveActionExecutor.HIVE_SCRIPT);
 String query = hiveConf.get(HiveActionExecutor.HIVE_QUERY);
 if (scriptPath != null) {

http://git-wip-us.apache.org/repos/asf/oozie/blob/3ad35372/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
--
diff --git 
a/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java 
b/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
index eaab1fe..44debbd 100644
--- a/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
+++ b/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
@@ -117,6 +117,10 @@ public class PigMain extends LauncherMain {
 System.out.println("Non-kerberoes execution");
 }
 
+//setting oozie workflow id as caller context id for pig
+String callerId = "oozie:" + System.getProperty("oozie.job.id");
+pigProperties.setProperty("pig.log.trace.id", callerId);
+
 OutputStream os = new FileOutputStream("pig.properties");
 pigProperties.store(os, "");
 os.close();



oozie git commit: OOZIE-2491 oozie acl cannot specify group, it does not work (abhishekbafna via rohini)

2016-09-19 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 0cdce5a16 -> 3bd45e924


OOZIE-2491 oozie acl cannot specify group,it does not work (abhishekbafna via 
rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/3bd45e92
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/3bd45e92
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/3bd45e92

Branch: refs/heads/master
Commit: 3bd45e92475ef7b983176b1c52c1bceb20de6e49
Parents: 0cdce5a
Author: Rohini Palaniswamy 
Authored: Mon Sep 19 10:13:25 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Sep 19 10:13:25 2016 -0700

--
 .../java/org/apache/oozie/service/AuthorizationService.java| 6 +++---
 .../org/apache/oozie/service/TestAuthorizationService.java | 5 -
 release-log.txt| 1 +
 3 files changed, 8 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/3bd45e92/core/src/main/java/org/apache/oozie/service/AuthorizationService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/AuthorizationService.java 
b/core/src/main/java/org/apache/oozie/service/AuthorizationService.java
index f281873..4e4ab17 100644
--- a/core/src/main/java/org/apache/oozie/service/AuthorizationService.java
+++ b/core/src/main/java/org/apache/oozie/service/AuthorizationService.java
@@ -295,7 +295,7 @@ public class AuthorizationService implements Service {
  *
  * @param user user name.
  * @param write indicates if the check is for read or write admin tasks 
(in this implementation this is ignored)
- * @throws AuthorizationException thrown if user does not have admin 
priviledges.
+ * @throws AuthorizationException thrown if user does not have admin 
privileges.
  */
 public void authorizeForAdmin(String user, boolean write) throws 
AuthorizationException {
 if (authorizationEnabled && write && !isAdmin(user)) {
@@ -417,7 +417,7 @@ public class AuthorizationService implements Service {
 String[] acl = aclStr.split(",");
 for (int i = 0; !userInAcl && i < acl.length; i++) {
 String aclItem = acl[i].trim();
-userInAcl = aclItem.equals(user) || 
groupsService.getGroups(user).equals(aclItem);
+userInAcl = aclItem.equals(user) || 
groupsService.getGroups(user).contains(aclItem);
 }
 }
 return userInAcl;
@@ -515,7 +515,7 @@ public class AuthorizationService implements Service {
  * @param user user name.
  * @param filter filter used to select jobs
  * @param start starting index of the jobs in DB
- * @param len maximum amount of jbos to select
+ * @param len maximum amount of jobs to select
  * @param write indicates if the check is for read or write job tasks.
  * @throws AuthorizationException thrown if the user is not authorized for 
the job.
  */

http://git-wip-us.apache.org/repos/asf/oozie/blob/3bd45e92/core/src/test/java/org/apache/oozie/service/TestAuthorizationService.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/service/TestAuthorizationService.java 
b/core/src/test/java/org/apache/oozie/service/TestAuthorizationService.java
index ce461c1..b5e527e 100644
--- a/core/src/test/java/org/apache/oozie/service/TestAuthorizationService.java
+++ b/core/src/test/java/org/apache/oozie/service/TestAuthorizationService.java
@@ -64,6 +64,9 @@ public class TestAuthorizationService extends XDataTestCase {
 if (getTestUser().equals(user)) {
 return Arrays.asList("users", getTestGroup());
 }
+if ("foo".equals(user)) {
+return Arrays.asList("users", "foogrp");
+}
 else {
 return Arrays.asList("users");
 }
@@ -129,7 +132,7 @@ public class TestAuthorizationService extends XDataTestCase 
{
 jobConf.set(OozieClient.GROUP_NAME, getTestGroup());
 }
 else {
-jobConf.set(OozieClient.GROUP_NAME, getTestGroup() + ",foo");
+jobConf.set(OozieClient.GROUP_NAME, getTestGroup() + ",foogrp");
 }
 
 jobConf.set(OozieClient.LOG_TOKEN, "t");

http://git-wip-us.apache.org/repos/asf/oozie/blob/3bd45e92/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 7851e81..a8afcf4 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2491 oozi

oozie git commit: OOZIE-2569 Adding yarn-site, core-site, hdfs-site and mapred-site into spark launcher (abhishekbafna via rohini)

2016-09-19 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master f86107155 -> 0cdce5a16


OOZIE-2569 Adding yarn-site, core-site, hdfs-site and mapred-site into spark 
launcher (abhishekbafna via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/0cdce5a1
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/0cdce5a1
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/0cdce5a1

Branch: refs/heads/master
Commit: 0cdce5a1645799d01520176c17da6ab7b9739c73
Parents: f861071
Author: Rohini Palaniswamy 
Authored: Mon Sep 19 10:07:57 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Sep 19 10:07:57 2016 -0700

--
 .../oozie/action/hadoop/SparkActionExecutor.java  |  6 ++
 core/src/main/resources/oozie-default.xml | 11 +++
 release-log.txt   |  1 +
 .../org/apache/oozie/action/hadoop/LauncherMain.java  | 14 ++
 .../org/apache/oozie/action/hadoop/ShellMain.java | 13 ++---
 .../org/apache/oozie/action/hadoop/SparkMain.java | 13 +
 6 files changed, 47 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/0cdce5a1/core/src/main/java/org/apache/oozie/action/hadoop/SparkActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/SparkActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/SparkActionExecutor.java
index 97355fd..15a641b 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/SparkActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/SparkActionExecutor.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.oozie.action.ActionExecutorException;
 import org.apache.oozie.client.WorkflowAction;
+import org.apache.oozie.service.ConfigurationService;
 import org.apache.oozie.service.HadoopAccessorException;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.SparkConfigurationService;
@@ -49,6 +50,7 @@ public class SparkActionExecutor extends JavaActionExecutor {
 public static final String SPARK_CLASS = "oozie.spark.class";
 public static final String SPARK_JAR = "oozie.spark.jar";
 public static final String MAPRED_CHILD_ENV = "mapred.child.env";
+private static final String CONF_OOZIE_SPARK_SETUP_HADOOP_CONF_DIR = 
"oozie.action.spark.setup.hadoop.conf.dir";
 
 public SparkActionExecutor() {
 super("spark");
@@ -95,6 +97,10 @@ public class SparkActionExecutor extends JavaActionExecutor {
 actionConf.set(SPARK_OPTS, sparkOptsSb.toString().trim());
 }
 
+// Setting if SparkMain should setup hadoop config *-site.xml
+boolean setupHadoopConf = 
actionConf.getBoolean(CONF_OOZIE_SPARK_SETUP_HADOOP_CONF_DIR,
+
ConfigurationService.getBoolean(CONF_OOZIE_SPARK_SETUP_HADOOP_CONF_DIR));
+actionConf.setBoolean(CONF_OOZIE_SPARK_SETUP_HADOOP_CONF_DIR, 
setupHadoopConf);
 return actionConf;
 }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/0cdce5a1/core/src/main/resources/oozie-default.xml
--
diff --git a/core/src/main/resources/oozie-default.xml 
b/core/src/main/resources/oozie-default.xml
index 40a5fe5..e71ebe3 100644
--- a/core/src/main/resources/oozie-default.xml
+++ b/core/src/main/resources/oozie-default.xml
@@ -1815,6 +1815,17 @@ will be the requeue interval for the actions which are 
waiting for a long time w
 
 
 
+oozie.action.spark.setup.hadoop.conf.dir
+false
+
+Oozie action.xml (oozie.action.conf.xml) contains all the hadoop 
configuration and user provided configurations.
+This property will allow users to copy Oozie action.xml as hadoop 
*-site configurations files. The advantage is,
+user need not to manage these files into spark sharelib. If user 
wants to manage the hadoop configurations
+themselves, it should should disable it.
+
+
+
+
 oozie.action.shell.setup.hadoop.conf.dir
 false
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/0cdce5a1/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 681e5ee..7851e81 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2569 Adding yarn-site, core-site, hdfs-site and mapred-site into spark 
launcher (abhishekbafna via rohini)
 OOZIE-2675 Drop support for hadoop 0.23 (abhishekbafna vi

[1/2] oozie git commit: OOZIE-2675 Drop support for hadoop 0.23 (abhishekbafna via rohini)

2016-09-19 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 9dc474e83 -> f86107155


OOZIE-2675 Drop support for hadoop 0.23 (abhishekbafna via rohini)


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

Branch: refs/heads/master
Commit: 2fd64fa646708ae684f2274729a0df6623598709
Parents: 9dc474e
Author: Rohini Palaniswamy 
Authored: Mon Sep 19 09:53:25 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Sep 19 09:53:25 2016 -0700

--
 .../oozie/action/hadoop/HadoopELFunctions.java  |  5 --
 .../oozie/command/wf/SubmitMRXCommand.java  |  3 --
 .../java/org/apache/oozie/util/JobUtils.java|  2 +-
 .../java/org/apache/oozie/test/XTestCase.java   |  4 +-
 docs/src/site/twiki/DG_QuickStart.twiki |  5 +-
 .../site/twiki/DG_SqoopActionExtension.twiki|  2 +-
 docs/src/site/twiki/ENG_Building.twiki  |  5 +-
 hadooplibs/hadoop-auth-0.23/pom.xml | 45 -
 hadooplibs/hadoop-distcp-0.23/pom.xml   | 43 -
 hadooplibs/hadoop-utils-0.23/pom.xml| 42 
 .../action/hadoop/LauncherMainHadoopUtils.java  | 35 --
 .../apache/oozie/hadoop/utils/HadoopShims.java  | 51 
 hadooplibs/pom.xml  | 11 -
 pom.xml | 12 -
 release-log.txt |  1 +
 15 files changed, 8 insertions(+), 258 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/2fd64fa6/core/src/main/java/org/apache/oozie/action/hadoop/HadoopELFunctions.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/HadoopELFunctions.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/HadoopELFunctions.java
index c322887..ad2a71d 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/HadoopELFunctions.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/HadoopELFunctions.java
@@ -52,11 +52,6 @@ public class HadoopELFunctions {
 Map> counters = (Map>) obj;
 if (counters == null) {
 counters = getCounters(nodeName);
-// In Hadoop 0.23 they deprecated 
'org.apache.hadoop.mapred.Task$Counter' and they REMOVED IT
-// Here we are getting the new Name and inserting it using the old 
name if the old name is not found
-if (counters.get(RECORDS) == null) {
-counters.put(RECORDS, counters.get(RECORDS_023));
-}
 instance.setTransientVar(nodeName + 
WorkflowInstance.NODE_VAR_SEPARATOR + HADOOP_COUNTERS, counters);
 }
 return counters;

http://git-wip-us.apache.org/repos/asf/oozie/blob/2fd64fa6/core/src/main/java/org/apache/oozie/command/wf/SubmitMRXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/wf/SubmitMRXCommand.java 
b/core/src/main/java/org/apache/oozie/command/wf/SubmitMRXCommand.java
index 9124a45..cc61d3d 100644
--- a/core/src/main/java/org/apache/oozie/command/wf/SubmitMRXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/wf/SubmitMRXCommand.java
@@ -43,9 +43,6 @@ public class SubmitMRXCommand extends SubmitHttpXCommand {
 SKIPPED_CONFS.add(WorkflowAppService.HADOOP_USER);
 SKIPPED_CONFS.add(XOozieClient.JT);
 SKIPPED_CONFS.add(XOozieClient.NN);
-// a brillant mind made a change in Configuration that 
'fs.default.name' key gets converted to 'fs.defaultFS'
-// in Hadoop 0.23, we need skip that one too, keeping the old one 
because of Hadoop 1
-SKIPPED_CONFS.add(XOozieClient.NN_2);
 
 DEPRECATE_MAP.put(XOozieClient.NN, XOozieClient.NN_2);
 DEPRECATE_MAP.put(XOozieClient.JT, XOozieClient.JT_2);

http://git-wip-us.apache.org/repos/asf/oozie/blob/2fd64fa6/core/src/main/java/org/apache/oozie/util/JobUtils.java
--
diff --git a/core/src/main/java/org/apache/oozie/util/JobUtils.java 
b/core/src/main/java/org/apache/oozie/util/JobUtils.java
index a4d7272..a7a53b3 100644
--- a/core/src/main/java/org/apache/oozie/util/JobUtils.java
+++ b/core/src/main/java/org/apache/oozie/util/JobUtils.java
@@ -162,7 +162,7 @@ public class JobUtils {
 URI uri = fs.makeQualified(file).toUri();
 DistributedCache.addCacheFile(uri, conf);
 }
-else { // Hadoop 0.23/2.x
+else { // Hadoop 2.x
 DistributedCache.addFileToClassPath(file, conf, fs);
 }
 }

http://git-wip-us.a

[2/2] oozie git commit: New testcase of OOZIE-2588 which was missed in the first commit

2016-09-19 Thread rohini
New testcase of OOZIE-2588 which was missed in the first commit


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

Branch: refs/heads/master
Commit: f86107155453d08d7e21cdfe41bc5bb3916d991b
Parents: 2fd64fa
Author: Rohini Palaniswamy 
Authored: Mon Sep 19 09:55:13 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Sep 19 09:55:13 2016 -0700

--
 .../action/hadoop/TestHCatCredentials.java  | 258 +++
 1 file changed, 258 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/f8610715/core/src/test/java/org/apache/oozie/action/hadoop/TestHCatCredentials.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/action/hadoop/TestHCatCredentials.java 
b/core/src/test/java/org/apache/oozie/action/hadoop/TestHCatCredentials.java
new file mode 100644
index 000..e6d43ca
--- /dev/null
+++ b/core/src/test/java/org/apache/oozie/action/hadoop/TestHCatCredentials.java
@@ -0,0 +1,258 @@
+/**
+ * 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.action.hadoop;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.HashMap;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.oozie.service.HCatAccessorService;
+import org.apache.oozie.service.ServiceException;
+import org.apache.oozie.service.Services;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ HCatCredentialHelper.class, HCatCredentials.class })
+public class TestHCatCredentials {
+private Services services;
+private static File OOZIE_HOME_DIR = null;
+private static final String TEST_HIVE_METASTORE_PRINCIPAL = 
"hcat/test-hcat1@oozie.example.com";
+private static final String TEST_HIVE_METASTORE_URI = 
"thrift://test-hcat1.com:9898";
+private static final String TEST_HIVE_METASTORE_PRINCIPAL2 = 
"hcat/test-hcat2@oozie.example.com";
+private static final String TEST_HIVE_METASTORE_URI2 = 
"thrift://test-hcat2.com:9898";
+final String HIVE_METASTORE_PRINCIPAL = "hive.principal";
+final String HIVE_METASTORE_URI = "hive.uri";
+final String HCAT_METASTORE_PRINCIPAL = "hcat.principal";
+final String HCAT_METASTORE_URI = "hcat.uri";
+private static File hiveSiteXml = null;
+private static ClassLoader prevClassloader = null;
+
+@BeforeClass
+public static void initialize() throws Exception {
+OOZIE_HOME_DIR = new File(new File("").getAbsolutePath(), 
"test-oozie-home");
+if (!OOZIE_HOME_DIR.exists()) {
+OOZIE_HOME_DIR.mkdirs();
+}
+System.setProperty(Services.OOZIE_HOME_DIR, 
OOZIE_HOME_DIR.getAbsolutePath());
+Services.setOozieHome();
+File oozieConfDir = new File(OOZIE_HOME_DIR.getAbsolutePath(), "conf");
+oozieConfDir.mkdir();
+File hadoopConfDir = new File(oozieConfDir, "hadoop-conf");
+hadoopConfDir.m

oozie git commit: OOZIE-2588 Support getting credentials for same cluster hcat when credentials config is empty (satishsaley via rohini)

2016-09-19 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master dd053ebf0 -> 9dc474e83


OOZIE-2588 Support getting credentials for same cluster hcat when credentials 
config is empty (satishsaley via rohini)


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

Branch: refs/heads/master
Commit: 9dc474e8313638680bd674ff9d0e81f760ab82a4
Parents: dd053eb
Author: Rohini Palaniswamy 
Authored: Mon Sep 19 09:49:46 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Sep 19 09:49:46 2016 -0700

--
 core/pom.xml| 18 +++
 .../oozie/action/hadoop/HCatCredentials.java| 49 +---
 pom.xml | 23 -
 release-log.txt |  1 +
 4 files changed, 84 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/9dc474e8/core/pom.xml
--
diff --git a/core/pom.xml b/core/pom.xml
index 62ace06..4c63cca 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -321,6 +321,24 @@
 
 
 
+org.powermock
+powermock-core
+test
+
+
+
+org.powermock
+powermock-api-mockito
+test
+
+
+
+org.powermock
+powermock-module-junit4
+test
+
+
+
 org.apache.activemq
 activemq-broker
 test

http://git-wip-us.apache.org/repos/asf/oozie/blob/9dc474e8/core/src/main/java/org/apache/oozie/action/hadoop/HCatCredentials.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/HCatCredentials.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/HCatCredentials.java
index 330e6e3..d7689a9 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/HCatCredentials.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/HCatCredentials.java
@@ -18,11 +18,19 @@
 
 package org.apache.oozie.action.hadoop;
 
+import java.util.HashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.action.ActionExecutor.Context;
+import org.apache.oozie.service.HCatAccessorService;
+import org.apache.oozie.service.Services;
 import org.apache.oozie.util.XLog;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Credentials implementation to store in jobConf, HCat-specific properties 
such as Principal and Uri
  * User specifies these credential properties along with the action 
configuration
@@ -37,6 +45,10 @@ public class HCatCredentials extends Credentials {
 private static final String HCAT_METASTORE_URI = "hcat.metastore.uri";
 private static final String HIVE_METASTORE_PRINCIPAL = 
"hive.metastore.kerberos.principal";
 private static final String HIVE_METASTORE_URI = "hive.metastore.uris";
+private final static Configuration hiveConf = new Configuration(false);
+static {
+hiveConf.addResource("hive-site.xml");
+}
 
 /* (non-Javadoc)
  * @see 
org.apache.oozie.action.hadoop.Credentials#addtoJobConf(org.apache.hadoop.mapred.JobConf,
 org.apache.oozie.action.hadoop.CredentialsProperties, 
org.apache.oozie.action.ActionExecutor.Context)
@@ -44,15 +56,14 @@ public class HCatCredentials extends Credentials {
 @Override
 public void addtoJobConf(JobConf jobconf, CredentialsProperties props, 
Context context) throws Exception {
 try {
-String principal = 
props.getProperties().get(HCAT_METASTORE_PRINCIPAL) == null
-? props.getProperties().get(HIVE_METASTORE_PRINCIPAL)
-: props.getProperties().get(HCAT_METASTORE_PRINCIPAL);
+
+String principal = getProperty(props.getProperties(), 
HCAT_METASTORE_PRINCIPAL, HIVE_METASTORE_PRINCIPAL);
 if (principal == null || principal.isEmpty()) {
 throw new CredentialException(ErrorCode.E0510,
 HCAT_METASTORE_PRINCIPAL + " is required to get hcat 
credential");
 }
-String server = props.getProperties().get(HCAT_METASTORE_URI) == 
null
-? props.getProperties().get(HIVE_METASTORE_URI) : 
props.getProperties().get(HCAT_METASTORE_URI);
+
+String server = getProperty(props.getProperties(), 
HCAT_METASTORE_URI, HIVE_METASTORE_URI);
 if (server == n

oozie git commit: Additional patch fixing issues for OOZIE-1402

2016-08-05 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master eba0f0365 -> cdde957ac


Additional patch fixing issues for OOZIE-1402


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

Branch: refs/heads/master
Commit: cdde957ac97c6e17607c63184bf52d309289b347
Parents: eba0f03
Author: Rohini Palaniswamy 
Authored: Fri Aug 5 15:37:24 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Fri Aug 5 15:37:24 2016 -0700

--
 .../command/coord/CoordActionInputCheckXCommand.java   | 13 +++--
 core/src/main/resources/oozie-default.xml  |  2 +-
 2 files changed, 8 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/cdde957a/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
 
b/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
index c3f01d8..650e7f8 100644
--- 
a/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
+++ 
b/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
@@ -71,8 +71,8 @@ public class CoordActionInputCheckXCommand extends 
CoordinatorXCommand {
  */
 public static final String CONF_COORD_INPUT_CHECK_REQUEUE_INTERVAL = 
Service.CONF_PREFIX
 + "coord.input.check.requeue.interval";
-public static final String 
CONF_COORD_INPUT_CHECK_REQUEUE_INTERVAL_INCREASE = Service.CONF_PREFIX
-+ "coord.input.check.requeue.interval.increase";
+public static final String 
CONF_COORD_INPUT_CHECK_REQUEUE_INTERVAL_ADDITIONAL_DELAY = Service.CONF_PREFIX
++ "coord.input.check.requeue.interval.additional.delay";
 private CoordinatorActionBean coordAction = null;
 private CoordinatorJobBean coordJob = null;
 private JPAService jpaService = null;
@@ -191,11 +191,12 @@ public class CoordActionInputCheckXCommand extends 
CoordinatorXCommand {
 new CoordActionReadyXCommand(coordAction.getJobId()).call();
 }
 else if (!isTimeout(currentTime)) {
-if (status == false) {
-long increase = 
ConfigurationService.getInt(CONF_COORD_INPUT_CHECK_REQUEUE_INTERVAL_INCREASE)
-* 1000L;
+if (!status) {
+long addtionalDelay = isChangeInDependency ? 0
+: 
ConfigurationService.getInt(CONF_COORD_INPUT_CHECK_REQUEUE_INTERVAL_ADDITIONAL_DELAY)
+* 1000L;
 queue(new 
CoordActionInputCheckXCommand(coordAction.getId(), coordAction.getJobId()),
-increase + getCoordInputCheckRequeueInterval());
+addtionalDelay + 
getCoordInputCheckRequeueInterval());
 }
 updateCoordAction(coordAction, isChangeInDependency);
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/cdde957a/core/src/main/resources/oozie-default.xml
--
diff --git a/core/src/main/resources/oozie-default.xml 
b/core/src/main/resources/oozie-default.xml
index 6a9d5c4..40a5fe5 100644
--- a/core/src/main/resources/oozie-default.xml
+++ b/core/src/main/resources/oozie-default.xml
@@ -546,7 +546,7 @@

 
 
-oozie.service.coord.input.check.requeue.interval.increase
+
oozie.service.coord.input.check.requeue.interval.additional.delay
 0
 This value (in seconds) will be added into 
oozie.service.coord.input.check.requeue.interval and resulting value
 will be the requeue interval for the actions which are waiting for a long time 
without any input.



oozie git commit: OOZIE-2623 Oozie should use a dummy OutputFormat (satishsaley via rohini)

2016-08-05 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master aec39dd26 -> eba0f0365


OOZIE-2623 Oozie should use a dummy OutputFormat (satishsaley via rohini)


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

Branch: refs/heads/master
Commit: eba0f0365f28604a73229a941a4b2e3e4c566c46
Parents: aec39dd
Author: Rohini Palaniswamy 
Authored: Fri Aug 5 13:15:38 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Fri Aug 5 13:15:38 2016 -0700

--
 .../oozie/action/hadoop/JavaActionExecutor.java |  2 +
 .../action/hadoop/LauncherMapperHelper.java |  3 +-
 release-log.txt |  1 +
 .../hadoop/OozieLauncherOutputCommitter.java| 65 
 .../hadoop/OozieLauncherOutputFormat.java   | 48 +++
 5 files changed, 118 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/eba0f036/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index e2e0510..9e1682c 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -146,6 +146,8 @@ public class JavaActionExecutor extends ActionExecutor {
 List classes = new ArrayList();
 classes.add(LauncherMapper.class);
 classes.add(OozieLauncherInputFormat.class);
+classes.add(OozieLauncherOutputFormat.class);
+classes.add(OozieLauncherOutputCommitter.class);
 classes.add(LauncherMainHadoopUtils.class);
 classes.add(HadoopShims.class);
 
classes.addAll(Services.get().get(URIHandlerService.class).getClassesForLauncher());

http://git-wip-us.apache.org/repos/asf/oozie/blob/eba0f036/core/src/main/java/org/apache/oozie/action/hadoop/LauncherMapperHelper.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/LauncherMapperHelper.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/LauncherMapperHelper.java
index ef6b99d..ed06707 100644
--- 
a/core/src/main/java/org/apache/oozie/action/hadoop/LauncherMapperHelper.java
+++ 
b/core/src/main/java/org/apache/oozie/action/hadoop/LauncherMapperHelper.java
@@ -159,7 +159,8 @@ public class LauncherMapperHelper {
 }
 
 launcherConf.setInputFormat(OozieLauncherInputFormat.class);
-launcherConf.set("mapred.output.dir", new Path(actionDir, 
"output").toString());
+launcherConf.setOutputFormat(OozieLauncherOutputFormat.class);
+launcherConf.setOutputCommitter(OozieLauncherOutputCommitter.class);
 }
 
 public static void setupYarnRestartHandling(JobConf launcherJobConf, 
Configuration actionConf, String launcherTag,

http://git-wip-us.apache.org/repos/asf/oozie/blob/eba0f036/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 7df9643..f83aa33 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2623 Oozie should use a dummy OutputFormat (satishsaley via rohini)
 OOZIE-2625 Drop workflowgenerator (rkanter)
 OOZIE-2602 Upgrade oozie to pig 0.16.0 (nperiwal via jaydeepvishwakarma)
 OOZIE-2493 TestDistcpMain deletes action.xml from wrong filesystem 
(abhishekbafna via rkanter)

http://git-wip-us.apache.org/repos/asf/oozie/blob/eba0f036/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
new file mode 100644
index 000..153019b
--- /dev/null
+++ 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherOutputCommitter.java
@@ -0,0 +1,65 @@
+/**
+ * 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

oozie git commit: OOZIE-2581 Oozie should reset SecurityManager in finally block (satishsaley via rohini)

2016-07-13 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 1cb23a554 -> aa6e2eb44


OOZIE-2581 Oozie should reset SecurityManager in finally block (satishsaley via 
rohini)


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

Branch: refs/heads/master
Commit: aa6e2eb44c20b153de27afa60f91e8f765bc09a2
Parents: 1cb23a5
Author: Rohini Palaniswamy 
Authored: Wed Jul 13 14:11:32 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jul 13 14:11:32 2016 -0700

--
 release-log.txt |  1 +
 .../oozie/action/hadoop/LauncherMapper.java | 20 
 2 files changed, 17 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/aa6e2eb4/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 2036dbc..686378c 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2581 Oozie should reset SecurityManager in finally block (satishsaley 
via rohini)
 OOZIE-2579 Bulk kill tests in TestBulkWorkflowXCommand might fail because of a 
race condition (pbacsko via rkanter)
 OOZIE-2587 Disable SchedulerService on certain tests (pbacsko via rkanter)
 OOZIE-2603 Give thread pools a meaningful name in CallableQueueService and 
SchedulerService (pbacsko via rkanter)

http://git-wip-us.apache.org/repos/asf/oozie/blob/aa6e2eb4/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMapper.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMapper.java
 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMapper.java
index 545f8e2..17ba97d 100644
--- 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMapper.java
+++ 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMapper.java
@@ -28,8 +28,6 @@ import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
 import java.io.StringWriter;
-import java.io.OutputStream;
-import java.io.FileOutputStream;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.security.Permission;
@@ -150,6 +148,7 @@ public class LauncherMapper implements 
Mapper, R
 
 @Override
 public void map(K1 key, V1 value, OutputCollector collector, 
Reporter reporter) throws IOException {
+SecurityManager initialSecurityManager = System.getSecurityManager();
 try {
 if (configFailure) {
 throw configureFailureEx;
@@ -163,7 +162,6 @@ public class LauncherMapper implements 
Mapper, R
 int errorCode = 0;
 Throwable errorCause = null;
 String errorMessage = null;
-
 try {
 new LauncherSecurityManager();
 }
@@ -321,6 +319,7 @@ public class LauncherMapper implements 
Mapper, R
 }
 finally {
 uploadActionDataToHDFS();
+resetSecurityManager(initialSecurityManager);
 }
 }
 
@@ -601,6 +600,19 @@ public class LauncherMapper implements 
Mapper, R
 }
 }
 
+private void resetSecurityManager(SecurityManager initialSecurityManager) {
+try {
+SecurityManager prev = System.getSecurityManager();
+System.setSecurityManager(initialSecurityManager);
+System.out
+.println("Successfully reset security manager from " + 
prev + " to " + System.getSecurityManager());
+}
+catch (Throwable t) {
+System.err.println("Failed to reset security manager: " + 
t.getMessage());
+t.printStackTrace(System.err);
+}
+}
+
 }
 
 class LauncherSecurityManager extends SecurityManager {
@@ -659,4 +671,4 @@ class JavaMainException extends Exception {
 public JavaMainException(Throwable t) {
 super(t);
 }
-}
+}
\ No newline at end of file



oozie git commit: OOZIE-2547 Add mapreduce.job.cache.files to spark action (satishsaley via rohini)

2016-06-10 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master eee0a4ee4 -> c22364554


OOZIE-2547 Add mapreduce.job.cache.files to spark action (satishsaley via 
rohini)


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

Branch: refs/heads/master
Commit: c22364554ca51a422319776b89a4c9b727714499
Parents: eee0a4e
Author: Rohini Palaniswamy 
Authored: Fri Jun 10 10:13:25 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Fri Jun 10 10:13:25 2016 -0700

--
 .../site/twiki/DG_SparkActionExtension.twiki|  60 +-
 release-log.txt |   1 +
 .../apache/oozie/action/hadoop/SparkMain.java   | 189 ---
 3 files changed, 142 insertions(+), 108 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/c2236455/docs/src/site/twiki/DG_SparkActionExtension.twiki
--
diff --git a/docs/src/site/twiki/DG_SparkActionExtension.twiki 
b/docs/src/site/twiki/DG_SparkActionExtension.twiki
index d7d75a1..74875bb 100644
--- a/docs/src/site/twiki/DG_SparkActionExtension.twiki
+++ b/docs/src/site/twiki/DG_SparkActionExtension.twiki
@@ -153,7 +153,7 @@ to navigate to the Oozie Launcher map-reduce job task logs 
via the Hadoop job-tr
 To ensure that your Spark job shows up in the Spark History Server, make sure 
to specify these three Spark configuration properties
 either in =spark-opts= with =--conf= or from 
=oozie.service.SparkConfigurationService.spark.configurations= in 
oozie-site.xml.
 
-1. spark.yarn.historyServer.address=http://SPH-HOST:18088
+1. spark.yarn.historyServer.address=SPH-HOST:18088
 
 2. spark.eventLog.dir=hdfs://NN:8020/user/spark/applicationHistory
 
@@ -261,6 +261,64 @@ it's localized to the working directory with just its name.
 
 
 
+--- Spark Action Schema Version 0.2
+
+http://www.w3.org/2001/XMLSchema";
+   xmlns:spark="uri:oozie:spark-action:0.2" 
elementFormDefault="qualified"
+   targetNamespace="uri:oozie:spark-action:0.2">
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
 [[index][::Go back to Oozie Documentation Index::]]
 
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/c2236455/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 4a556e6..88f66e9 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2547 Add mapreduce.job.cache.files to spark action (satishsaley via 
rohini)
 OOZIE-2550 Flaky tests in TestZKUUIDService.java (pbacsko via rkanter)
 OOZIE-2445 Doc for - Specifying coordinator input datasets in more logical 
ways (puru)
 OOZIE-2541 Possible resource leak in Hive2Credentials (pbacsko via rkanter)

http://git-wip-us.apache.org/repos/asf/oozie/blob/c2236455/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
--
diff --git 
a/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java 
b/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
index 13c1075..3acaef9 100644
--- a/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
+++ b/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
@@ -21,6 +21,8 @@ package org.apache.oozie.action.hadoop;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.filecache.DistributedCache;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.PropertyConfigurator;
 import org.apache.spark.deploy.SparkSubmit;
 
@@ -28,9 +30,10 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Properties;
 import java.util.regex.Pattern;
@@ -43,18 +46,12 

oozie git commit: OOZIE-2563 Pass spark-defaults.conf to spark action (satishsaley via rohini)

2016-06-08 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 7779f3184 -> fd7708949


OOZIE-2563 Pass spark-defaults.conf to spark action (satishsaley via rohini)


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

Branch: refs/heads/master
Commit: fd7708949e0ae5eb0b65a9f6ff5e4af09a6dde57
Parents: 7779f31
Author: Rohini Palaniswamy 
Authored: Wed Jun 8 15:29:37 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jun 8 15:29:37 2016 -0700

--
 release-log.txt |  1 +
 .../apache/oozie/action/hadoop/SparkMain.java   | 29 ++--
 2 files changed, 27 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/fd770894/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 2f69344..5fd8ef8 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2563 Pass spark-defaults.conf to spark action (satishsaley via rohini)
 OOZIE-2556 TestAbandonedCoordChecker.testCatchupJob is flaky (puru)
 OOZIE-2522 There can be multiple coord submit from bundle in case of ZK glitch 
(puru)
 OOZIE-2553 Cred tag is required for all actions in the workflow even if an 
action does not require it (me.venkatr via rohini)

http://git-wip-us.apache.org/repos/asf/oozie/blob/fd770894/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
--
diff --git 
a/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java 
b/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
index 0e6e271..13c1075 100644
--- a/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
+++ b/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
@@ -50,6 +50,8 @@ public class SparkMain extends LauncherMain {
 
 private static final Pattern[] PYSPARK_DEP_FILE_PATTERN = { 
Pattern.compile("py4\\S*src.zip"),
 Pattern.compile("pyspark.zip") };
+private static final Pattern SPARK_DEFAULTS_FILE_PATTERN = 
Pattern.compile("spark-defaults.conf");
+
 private String sparkJars = null;
 private String sparkClasspath = null;
 
@@ -198,6 +200,11 @@ public class SparkMain extends LauncherMain {
 sparkArgs.add("--conf");
 sparkArgs.add(HBASE_SECURITY_TOKEN + "=false");
 }
+File defaultConfFile = getMatchingFile(SPARK_DEFAULTS_FILE_PATTERN);
+if (defaultConfFile != null) {
+sparkArgs.add("--properties-file");
+sparkArgs.add(SPARK_DEFAULTS_FILE_PATTERN.toString());
+}
 if (!sparkArgs.contains(VERBOSE_OPTION)) {
 sparkArgs.add(VERBOSE_OPTION);
 }
@@ -242,7 +249,7 @@ public class SparkMain extends LauncherMain {
 }
 
 for(Pattern fileNamePattern : PYSPARK_DEP_FILE_PATTERN) {
-File file = getMatchingFile(fileNamePattern);
+File file = getMatchingPyFile(fileNamePattern);
 File destination = new File(pythonLibDir, file.getName());
 FileUtils.copyFile(file, destination);
 System.out.println("Copied " + file + " to " + 
destination.getAbsolutePath());
@@ -256,6 +263,23 @@ public class SparkMain extends LauncherMain {
  * @return the file if there is one
  * @throws OozieActionConfiguratorException if there is are no files 
matching the pattern
  */
+private File getMatchingPyFile(Pattern fileNamePattern) throws 
OozieActionConfiguratorException {
+File f = getMatchingFile(fileNamePattern);
+if (f != null) {
+return f;
+}
+throw new OozieActionConfiguratorException("Missing py4j and/or 
pyspark zip files. Please add them to "
++ "the lib folder or to the Spark sharelib.");
+}
+
+/**
+ * Searches for a file in the current directory that matches the given
+ * pattern. If there are multiple files matching the pattern returns one of
+ * them.
+ *
+ * @param fileNamePattern the pattern to look for
+ * @return the file if there is one else it returns null
+ */
 private File getMatchingFile(Pattern fileNamePattern) throws 
OozieActionConfiguratorException {
 File localDir = new File(".");
 for(String fileName : localDir.list()){
@@ -263,8 +287,7 @@ public class SparkMain extends LauncherMain {
 return new File(fileName)

oozie git commit: OOZIE-2553 Cred tag is required for all actions in the workflow even if an action does not require it (me.venkatr via rohini)

2016-06-07 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master fc9aeb13a -> f83f484a1


OOZIE-2553 Cred tag is required for all actions in the workflow even if an 
action does not require it (me.venkatr via rohini)


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

Branch: refs/heads/master
Commit: f83f484a1a34c9446cf43ba7d627aefabaf58921
Parents: fc9aeb1
Author: Rohini Palaniswamy 
Authored: Tue Jun 7 08:32:13 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Jun 7 08:32:13 2016 -0700

--
 .../oozie/action/hadoop/JavaActionExecutor.java | 12 ---
 .../org/apache/oozie/workflow/lite/NodeDef.java |  5 ++-
 .../action/hadoop/ActionExecutorTestCase.java   |  2 --
 .../action/hadoop/TestJavaActionExecutor.java   | 38 +++-
 .../command/wf/TestActionCheckXCommand.java |  1 -
 .../command/wf/TestActionStartXCommand.java |  4 ---
 .../apache/oozie/event/TestEventGeneration.java |  1 -
 .../oozie/service/TestRecoveryService.java  |  1 -
 .../apache/oozie/store/TestDBWorkflowStore.java |  1 -
 .../org/apache/oozie/test/XDataTestCase.java|  1 -
 release-log.txt |  1 +
 11 files changed, 49 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/f83f484a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index 6893bb8..639003e 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -1307,11 +1307,13 @@ public class JavaActionExecutor extends ActionExecutor {
 HashMap props = new HashMap();
 if (context != null && action != null) {
 String credsInAction = action.getCred();
-LOG.debug("Get credential '" + credsInAction + "' properties for 
action : " + action.getId());
-String[] credNames = credsInAction.split(",");
-for (String credName : credNames) {
-CredentialsProperties credProps = getCredProperties(context, 
credName);
-props.put(credName, credProps);
+if (credsInAction != null) {
+LOG.debug("Get credential '" + credsInAction + "' properties 
for action : " + action.getId());
+String[] credNames = credsInAction.split(",");
+for (String credName : credNames) {
+CredentialsProperties credProps = 
getCredProperties(context, credName);
+props.put(credName, credProps);
+}
 }
 }
 else {

http://git-wip-us.apache.org/repos/asf/oozie/blob/f83f484a/core/src/main/java/org/apache/oozie/workflow/lite/NodeDef.java
--
diff --git a/core/src/main/java/org/apache/oozie/workflow/lite/NodeDef.java 
b/core/src/main/java/org/apache/oozie/workflow/lite/NodeDef.java
index a395b77..9e66d28 100644
--- a/core/src/main/java/org/apache/oozie/workflow/lite/NodeDef.java
+++ b/core/src/main/java/org/apache/oozie/workflow/lite/NodeDef.java
@@ -40,7 +40,7 @@ public class NodeDef implements Writable {
 private Class handlerClass;
 private String conf = null;
 private List transitions = new ArrayList();
-private String cred = "null";
+private String cred = null;
 private String userRetryMax = "null";
 private String userRetryInterval = "null";
 
@@ -154,6 +154,9 @@ public class NodeDef implements Writable {
 nodeDefVersion = LiteWorkflowStoreService.NODE_DEF_VERSION_1;
 name = dataInput.readUTF();
 cred = dataInput.readUTF();
+if (cred.equals("null")) {
+cred = null;
+}
 String handlerClassName = dataInput.readUTF();
 if ((handlerClassName != null) && (handlerClassName.length() > 0)) {
 try {

http://git-wip-us.apache.org/repos/asf/oozie/blob/f83f484a/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java 
b/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java
index e1c450c..d74160a 100644

oozie git commit: OOZIE-2475 Oozie does not cleanup action dir of killed actions (satishsaley via rohini)

2016-05-27 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 3c50d5c32 -> c5f5f9d6b


OOZIE-2475 Oozie does not cleanup action dir of killed actions (satishsaley via 
rohini)


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

Branch: refs/heads/master
Commit: c5f5f9d6bae8ab44b54f728ebee15851478fa272
Parents: 3c50d5c
Author: Rohini Palaniswamy 
Authored: Fri May 27 13:07:27 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Fri May 27 13:07:27 2016 -0700

--
 .../oozie/command/wf/ActionKillXCommand.java| 46 
 release-log.txt |  1 +
 2 files changed, 39 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/c5f5f9d6/core/src/main/java/org/apache/oozie/command/wf/ActionKillXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/wf/ActionKillXCommand.java 
b/core/src/main/java/org/apache/oozie/command/wf/ActionKillXCommand.java
index 33498bf..ac096cc 100644
--- a/core/src/main/java/org/apache/oozie/command/wf/ActionKillXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/wf/ActionKillXCommand.java
@@ -22,33 +22,37 @@ import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
 
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.SLAEventBean;
 import org.apache.oozie.WorkflowActionBean;
 import org.apache.oozie.WorkflowJobBean;
 import org.apache.oozie.XException;
+import org.apache.oozie.action.ActionExecutor;
+import org.apache.oozie.action.ActionExecutor.Context;
+import org.apache.oozie.action.ActionExecutorException;
+import org.apache.oozie.action.control.ControlNodeActionExecutor;
 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.command.CommandException;
 import org.apache.oozie.command.PreconditionException;
-import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
+import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor;
-import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import 
org.apache.oozie.executor.jpa.WorkflowActionQueryExecutor.WorkflowActionQuery;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
-import org.apache.oozie.action.ActionExecutor;
-import org.apache.oozie.action.ActionExecutorException;
-import org.apache.oozie.action.control.ControlNodeActionExecutor;
 import org.apache.oozie.service.ActionService;
 import org.apache.oozie.service.EventHandlerService;
 import org.apache.oozie.service.JPAService;
-import org.apache.oozie.service.UUIDService;
 import org.apache.oozie.service.Services;
-import org.apache.oozie.util.LogUtils;
+import org.apache.oozie.service.UUIDService;
 import org.apache.oozie.util.Instrumentation;
+import org.apache.oozie.util.LogUtils;
 import org.apache.oozie.util.db.SLADbXOperations;
 
 /**
@@ -129,10 +133,11 @@ public class ActionKillXCommand extends 
ActionXCommand {
 if (wfAction.isPending()) {
 ActionExecutor executor = 
Services.get().get(ActionService.class).getExecutor(wfAction.getType());
 if (executor != null) {
+ActionExecutorContext context = null;
 try {
 boolean isRetry = false;
 boolean isUserRetry = false;
-ActionExecutorContext context = new 
ActionXCommand.ActionExecutorContext(wfJob, wfAction,
+context = new ActionXCommand.ActionExecutorContext(wfJob, 
wfAction,
 isRetry, isUserRetry);
 incrActionCounter(wfAction.getType(), 1);
 
@@ -179,6 +184,7 @@ public class ActionKillXCommand extends 
ActionXCommand {
 }
 finally {
 try {
+cleanupActionDir(context);
 
BatchQueryExecutor.getInstance().executeBatchInsertUpdateDelete(insertList, 
updateList, null);
 if (!(executor instanceof ControlNodeActionExecutor) 
&& EventHandlerService.isEnabled()) {
 generateEvent(wfAction, wfJob.getUser());
@@ -194,4 +200,2

oozie git commit: OOZIE-2529 Support adding secret keys to Credentials of Launcher (satishsaley via rohini)

2016-05-13 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master c715efdc0 -> 5fbd3eb3f


OOZIE-2529 Support adding secret keys to Credentials of Launcher (satishsaley 
via rohini)


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

Branch: refs/heads/master
Commit: 5fbd3eb3fc12e78c3087e3456a83470564264c1f
Parents: c715efd
Author: Rohini Palaniswamy 
Authored: Fri May 13 15:49:18 2016 -0700
Committer: Rohini Palaniswamy 
Committed: Fri May 13 15:49:18 2016 -0700

--
 .../apache/oozie/action/hadoop/JavaActionExecutor.java | 13 +
 .../apache/oozie/action/hadoop/InsertTestToken.java|  4 
 .../oozie/action/hadoop/TestJavaActionExecutor.java|  4 
 release-log.txt|  1 +
 4 files changed, 22 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/5fbd3eb3/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index ff084d1..a081e66 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -1164,6 +1164,19 @@ public class JavaActionExecutor extends ActionExecutor {
 LOG.debug("ADDING TOKEN: " + fauxAlias);
 launcherJobConf.getCredentials().addToken(fauxAlias, 
tk);
 }
+if (credentialsConf.getCredentials().numberOfSecretKeys() 
> 0) {
+for (Entry entry : 
credentialsProperties.entrySet()) {
+CredentialsProperties credProps = entry.getValue();
+if (credProps != null) {
+Text credName = new Text(credProps.getName());
+byte[] secKey = 
credentialsConf.getCredentials().getSecretKey(credName);
+if (secKey != null) {
+LOG.debug("ADDING CREDENTIAL: " + 
credProps.getName());
+
launcherJobConf.getCredentials().addSecretKey(credName, secKey);
+}
+}
+}
+}
 }
 else {
 LOG.info("No need to inject credentials.");

http://git-wip-us.apache.org/repos/asf/oozie/blob/5fbd3eb3/core/src/test/java/org/apache/oozie/action/hadoop/InsertTestToken.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/action/hadoop/InsertTestToken.java 
b/core/src/test/java/org/apache/oozie/action/hadoop/InsertTestToken.java
index e1c439f..9da8fbe 100644
--- a/core/src/test/java/org/apache/oozie/action/hadoop/InsertTestToken.java
+++ b/core/src/test/java/org/apache/oozie/action/hadoop/InsertTestToken.java
@@ -27,6 +27,7 @@ import org.apache.oozie.util.XLog;
 
 
 public class InsertTestToken extends Credentials{
+public static String DUMMY_SECRET_KEY = "DummySecretKey";
 public InsertTestToken() {
 }
 
@@ -39,6 +40,9 @@ public class InsertTestToken extends Credentials{
 Token abctoken = new 
Token();
 jobconf.getCredentials().addToken(new Text("ABC Token"), abctoken);
 XLog.getLog(getClass()).debug("Added the ABC token in job conf");
+
+jobconf.getCredentials().addSecretKey(new Text(DUMMY_SECRET_KEY), 
DUMMY_SECRET_KEY.getBytes("UTF-8"));
+XLog.getLog(getClass()).debug("Added the " + DUMMY_SECRET_KEY + " 
in job conf");
 }
 catch (Exception e) {
 XLog.getLog(getClass()).warn("Exception in addtoJobConf", e);

http://git-wip-us.apache.org/repos/asf/oozie/blob/5fbd3eb3/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java 
b/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
index d2f90c6..3dad557 100644
--- 
a/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
+++ 
b/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
@@ -980,6 +980,10 @@ public class TestJavaActionE

oozie git commit: OOZIE-2431: Support hive.metastore.uris as alias for hcat.metastore.uri - Fix incorrect logic in initial patch

2016-01-28 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 8aa465979 -> 05fce1f78


OOZIE-2431: Support hive.metastore.uris as alias for hcat.metastore.uri - Fix 
incorrect logic in initial patch


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/05fce1f7
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/05fce1f7
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/05fce1f7

Branch: refs/heads/master
Commit: 05fce1f78adf1f7d2899f901fb8f4a9c2b6314b0
Parents: 8aa4659
Author: Rohini Palaniswamy 
Authored: Thu Jan 28 10:13:28 2016 -0800
Committer: Rohini Palaniswamy 
Committed: Thu Jan 28 10:13:28 2016 -0800

--
 .../java/org/apache/oozie/action/hadoop/HCatCredentials.java| 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/05fce1f7/core/src/main/java/org/apache/oozie/action/hadoop/HCatCredentials.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/HCatCredentials.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/HCatCredentials.java
index b182c1d..330e6e3 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/HCatCredentials.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/HCatCredentials.java
@@ -45,13 +45,14 @@ public class HCatCredentials extends Credentials {
 public void addtoJobConf(JobConf jobconf, CredentialsProperties props, 
Context context) throws Exception {
 try {
 String principal = 
props.getProperties().get(HCAT_METASTORE_PRINCIPAL) == null
-? props.getProperties().get(HIVE_METASTORE_PRINCIPAL) : 
null;
+? props.getProperties().get(HIVE_METASTORE_PRINCIPAL)
+: props.getProperties().get(HCAT_METASTORE_PRINCIPAL);
 if (principal == null || principal.isEmpty()) {
 throw new CredentialException(ErrorCode.E0510,
 HCAT_METASTORE_PRINCIPAL + " is required to get hcat 
credential");
 }
 String server = props.getProperties().get(HCAT_METASTORE_URI) == 
null
-? props.getProperties().get(HIVE_METASTORE_URI) : null;
+? props.getProperties().get(HIVE_METASTORE_URI) : 
props.getProperties().get(HCAT_METASTORE_URI);
 if (server == null || server.isEmpty()) {
 throw new CredentialException(ErrorCode.E0510,
 HCAT_METASTORE_URI + " is required to get hcat 
credential");



oozie git commit: Fix documentation for OOZIE-2444

2016-01-26 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master ff3c2956d -> 8aa465979


Fix documentation for OOZIE-2444


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

Branch: refs/heads/master
Commit: 8aa4659791e232c67b1f1fb264605edd3afcd473
Parents: ff3c295
Author: Rohini Palaniswamy 
Authored: Tue Jan 26 13:20:43 2016 -0800
Committer: Rohini Palaniswamy 
Committed: Tue Jan 26 13:20:43 2016 -0800

--
 docs/src/site/twiki/BundleFunctionalSpec.twiki | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/8aa46597/docs/src/site/twiki/BundleFunctionalSpec.twiki
--
diff --git a/docs/src/site/twiki/BundleFunctionalSpec.twiki 
b/docs/src/site/twiki/BundleFunctionalSpec.twiki
index c4e1e36..9749df5 100644
--- a/docs/src/site/twiki/BundleFunctionalSpec.twiki
+++ b/docs/src/site/twiki/BundleFunctionalSpec.twiki
@@ -94,9 +94,8 @@ A bundle definition is defined in XML by a name, controls and 
one or more coordi
 * *%BLUE% controls: %ENDCOLOR%* The control specification for the bundle.
   * *%BLUE% kick-off-time: %ENDCOLOR%* It defines when the bundle job 
should start and submit the coordinator applications. This field is optional 
and the default is *NOW* that means the job should start right-a-way.
* *%BLUE% coordinator: %ENDCOLOR%* Coordinator application specification. 
There should be at least one coordinator application in any bundle.
-  * *%BLUE% name: %ENDCOLOR%* Name of the coordinator application. It can 
be used for referring this application through
-  bundle to control such as kill, suspend, rerun. Enabled can be used to 
enable or disable a coordinator. It is optional.
-  The default value for enabled is true.
+  * *%BLUE% name: %ENDCOLOR%* Name of the coordinator application. It can 
be used for referring this application through bundle to control such as kill, 
suspend, rerun.
+  * *%BLUE% enabled: %ENDCOLOR%* Enabled can be used to enable or disable 
a coordinator. It is optional. The default value for enabled is true.
   * *%BLUE% app-path: %ENDCOLOR%* Path of the coordinator application 
definition in hdfs. This is a mandatory element.
   * *%BLUE% configuration: %ENDCOLOR%* A hadoop like configuration to 
parameterize corresponding coordinator application. This is optional.
 * *%BLUE% Parameterization: %ENDCOLOR%*  Configuration properties that are 
a valid Java identifier, [A-Za-z_][0-9A-Za-z_]*, are available as =${NAME}= 
variables within the bundle application definition. Configuration properties 
that are not a valid Java identifier, for example =job.tracker=, are available 
via the =${bundle:conf(String name)}= function. Valid Java identifier 
properties are available via this function as well.



oozie git commit: OOZIE-2030 Configuration properties from global section is not getting set in Hadoop job conf when using sub-workflow action in Oozie workflow.xml (jaydeepvishwakarma via rohini)

2016-01-09 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master c2fe52b83 -> 0f086d41b


OOZIE-2030 Configuration properties from global section is not getting set in 
Hadoop job conf when using sub-workflow action in Oozie workflow.xml 
(jaydeepvishwakarma via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/0f086d41
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/0f086d41
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/0f086d41

Branch: refs/heads/master
Commit: 0f086d41b8f274abe0959705f76fa3225ab5ff2e
Parents: c2fe52b
Author: Rohini Palaniswamy 
Authored: Sat Jan 9 16:02:12 2016 -0800
Committer: Rohini Palaniswamy 
Committed: Sat Jan 9 16:02:12 2016 -0800

--
 .../action/oozie/SubWorkflowActionExecutor.java |   4 +
 .../workflow/lite/LiteWorkflowAppParser.java| 135 +--
 .../oozie/TestSubWorkflowActionExecutor.java| 113 +++-
 release-log.txt |   1 +
 4 files changed, 238 insertions(+), 15 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/0f086d41/core/src/main/java/org/apache/oozie/action/oozie/SubWorkflowActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/oozie/SubWorkflowActionExecutor.java
 
b/core/src/main/java/org/apache/oozie/action/oozie/SubWorkflowActionExecutor.java
index 33efc60..6bf3598 100644
--- 
a/core/src/main/java/org/apache/oozie/action/oozie/SubWorkflowActionExecutor.java
+++ 
b/core/src/main/java/org/apache/oozie/action/oozie/SubWorkflowActionExecutor.java
@@ -306,4 +306,8 @@ public class SubWorkflowActionExecutor extends 
ActionExecutor {
 public boolean isCompleted(String externalStatus) {
 return FINAL_STATUS.contains(externalStatus);
 }
+
+public boolean supportsConfigurationJobXML() {
+return true;
+}
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/0f086d41/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java 
b/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
index d3a6523..03c84f1 100644
--- 
a/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
+++ 
b/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
@@ -18,8 +18,10 @@
 
 package org.apache.oozie.workflow.lite;
 
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.io.Writable;
+import org.apache.oozie.action.oozie.SubWorkflowActionExecutor;
 import org.apache.oozie.service.ConfigurationService;
-import org.apache.oozie.workflow.WorkflowException;
 import org.apache.oozie.util.ELUtils;
 import org.apache.oozie.util.IOUtils;
 import org.apache.oozie.util.XConfiguration;
@@ -27,7 +29,9 @@ import org.apache.oozie.util.XmlUtils;
 import org.apache.oozie.util.ParamChecker;
 import org.apache.oozie.util.ParameterVerifier;
 import org.apache.oozie.util.ParameterVerifierException;
+import org.apache.oozie.util.WritableUtils;
 import org.apache.oozie.ErrorCode;
+import org.apache.oozie.workflow.WorkflowException;
 import org.apache.oozie.action.ActionExecutor;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.ActionService;
@@ -46,6 +50,12 @@ import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
 import java.io.StringWriter;
+import java.io.ByteArrayOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Deque;
@@ -54,6 +64,7 @@ import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.zip.*;
 
 /**
  * Class to parse and validate workflow xml
@@ -95,6 +106,7 @@ public class LiteWorkflowAppParser {
 
 public static final String DEFAULT_NAME_NODE = 
"oozie.actions.default.name-node";
 public static final String DEFAULT_JOB_TRACKER = 
"oozie.actions.default.job-tracker";
+public static final String OOZIE_GLOBAL = "oozie.wf.globalconf";
 
 private static final String JOB_TRACKER = "job-tracker";
 private static final String NAME_NODE = "name-node";
@@ -417,7 +429,9 @@ public class LiteWorkflowAppParser {
 throws WorkflowException {
 Namespace ns = root.getNamespace();
 LiteWorkflowApp def = null;
-GlobalSectionData gData = null;
+GlobalSectionData gData = jobConf.get(OOZIE_GLOBAL) == null ?
+ 

oozie git commit: OOZIE-2303 Typo in documentation (lars_francke via rohini)

2015-10-30 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 546d185d2 -> 4c898349c


OOZIE-2303 Typo in documentation (lars_francke via rohini)


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

Branch: refs/heads/master
Commit: 4c898349cb0bbf73d9d2b8c91ca72c6520c49a6f
Parents: 546d185
Author: Rohini Palaniswamy 
Authored: Fri Oct 30 13:33:13 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Fri Oct 30 13:33:13 2015 -0700

--
 docs/src/site/twiki/WorkflowFunctionalSpec.twiki | 2 +-
 release-log.txt  | 1 +
 2 files changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/4c898349/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
--
diff --git a/docs/src/site/twiki/WorkflowFunctionalSpec.twiki 
b/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
index a084c1a..2ce56bf 100644
--- a/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
+++ b/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
@@ -1451,7 +1451,7 @@ A =java= action can create a Hadoop configuration for 
interacting with a cluster
 Oozie prepares a Hadoop configuration file which includes the environments 
site configuration files (e.g. hdfs-site.xml, 
 mapred-site.xml, etc) plus the properties added to the == 
section of the =java= action. The Hadoop configuration 
 file is made available as a local file to the Java application in its running 
directory. It can be added to the =java= actions 
-Hadoop configuration by referencing the system property: 
=oozie-action.conf.xml=. For example:
+Hadoop configuration by referencing the system property: 
=oozie.action.conf.xml=. For example:
 
 
 // loading action conf prepared by Oozie

http://git-wip-us.apache.org/repos/asf/oozie/blob/4c898349/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 9dbcd1d..d1b14a9 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2303 Typo in documentation (lars_francke via rohini)
 OOZIE-2328 Coordinator endtime change should check if the last action is in 
database (kailongs via puru) 
 OOZIE-2367 fs delete should support skipTrash option (jaydeepvishwakarma via 
rohini)
 OOZIE-2368 coord:dateOffset and coord:dateTzOffset can't be used for coord 
initial-instance (puru)



oozie git commit: OOZIE-2367 fs delete should support skipTrash option (jaydeepvishwakarma via rohini)

2015-10-29 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master f713d6bd7 -> 6be29c61f


OOZIE-2367 fs delete should support skipTrash option (jaydeepvishwakarma via 
rohini)


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

Branch: refs/heads/master
Commit: 6be29c61f11a9f6226bd2b771a2902ea4eed4668
Parents: f713d6b
Author: Rohini Palaniswamy 
Authored: Thu Oct 29 14:30:23 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Thu Oct 29 14:30:23 2015 -0700

--
 .../src/main/resources/oozie-workflow-0.5.xsd   |  3 +-
 .../oozie/action/hadoop/FsActionExecutor.java   | 40 
 .../action/hadoop/TestFsActionExecutor.java | 22 ++-
 .../src/site/twiki/WorkflowFunctionalSpec.twiki |  7 +++-
 release-log.txt |  1 +
 5 files changed, 62 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/6be29c61/client/src/main/resources/oozie-workflow-0.5.xsd
--
diff --git a/client/src/main/resources/oozie-workflow-0.5.xsd 
b/client/src/main/resources/oozie-workflow-0.5.xsd
index b01580c..fda49ed 100644
--- a/client/src/main/resources/oozie-workflow-0.5.xsd
+++ b/client/src/main/resources/oozie-workflow-0.5.xsd
@@ -282,7 +282,8 @@
 
 
 
-
+
+
 
 
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/6be29c61/core/src/main/java/org/apache/oozie/action/hadoop/FsActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/FsActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/FsActionExecutor.java
index 8d96a47..b8649ac 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/FsActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/FsActionExecutor.java
@@ -21,6 +21,7 @@ package org.apache.oozie.action.hadoop;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -31,11 +32,14 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.oozie.action.ActionExecutor;
 import org.apache.oozie.action.ActionExecutorException;
+import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.WorkflowAction;
 import org.apache.oozie.dependency.FSURIHandler;
 import org.apache.oozie.dependency.URIHandler;
@@ -43,6 +47,7 @@ import org.apache.oozie.service.ConfigurationService;
 import org.apache.oozie.service.HadoopAccessorException;
 import org.apache.oozie.service.HadoopAccessorService;
 import org.apache.oozie.service.Services;
+import org.apache.oozie.service.UserGroupInformationService;
 import org.apache.oozie.service.URIHandlerService;
 import org.apache.oozie.util.XConfiguration;
 import org.apache.oozie.util.XmlUtils;
@@ -178,7 +183,12 @@ public class FsActionExecutor extends ActionExecutor {
 else {
 if (command.equals("delete")) {
 Path path = getPath(commandElement, "path");
-delete(context, fsConf, nameNodePath, path);
+boolean skipTrash = true;
+if (commandElement.getAttributeValue("skip-trash") != 
null &&
+
commandElement.getAttributeValue("skip-trash").equals("false")) {
+skipTrash = false;
+}
+delete(context, fsConf, nameNodePath, path, skipTrash);
 }
 else {
 if (command.equals("move")) {
@@ -315,7 +325,6 @@ public class FsActionExecutor extends ActionExecutor {
 /**
  * @param path
  * @param user
- * @param group
  * @return FileSystem
  * @throws HadoopAccessorException
  */
@@ -354,7 +363,7 @@ public class FsActionExecutor extends ActionExecutor {
  * @throws ActionExecutorException
  */
 public void delete(Context context, Path path) throws 
ActionExecutorException {
-delete(cont

oozie git commit: OOZIE-2365 oozie fail to start when smtp password not set (rohini)

2015-09-24 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master ed8ec4f97 -> 38e33811b


OOZIE-2365 oozie fail to start when smtp password not set (rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/38e33811
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/38e33811
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/38e33811

Branch: refs/heads/master
Commit: 38e33811bc3f9d3b1a1a024dcab6af760734b696
Parents: ed8ec4f
Author: Rohini Palaniswamy 
Authored: Thu Sep 24 14:25:40 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Thu Sep 24 14:25:40 2015 -0700

--
 .../oozie/action/email/EmailActionExecutor.java |  6 -
 .../oozie/service/ConfigurationService.java | 26 +---
 .../sla/listener/SLAEmailEventListener.java |  7 --
 .../apache/oozie/hadoop/utils/HadoopShims.java  |  2 +-
 release-log.txt |  1 +
 5 files changed, 35 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/38e33811/core/src/main/java/org/apache/oozie/action/email/EmailActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/email/EmailActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/email/EmailActionExecutor.java
index dc58236..7ddbc19 100644
--- a/core/src/main/java/org/apache/oozie/action/email/EmailActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/email/EmailActionExecutor.java
@@ -168,7 +168,7 @@ public class EmailActionExecutor extends ActionExecutor {
 String smtpPort = getOozieConf().get(EMAIL_SMTP_PORT, "25");
 Boolean smtpAuth = getOozieConf().getBoolean(EMAIL_SMTP_AUTH, false);
 String smtpUser = getOozieConf().get(EMAIL_SMTP_USER, "");
-String smtpPassword = 
ConfigurationService.getPassword(EMAIL_SMTP_PASS);
+String smtpPassword = 
ConfigurationService.getPassword(EMAIL_SMTP_PASS, "");
 String fromAddr = getOozieConf().get(EMAIL_SMTP_FROM, 
"oozie@localhost");
 
 Properties properties = new Properties();
@@ -322,18 +322,22 @@ public class EmailActionExecutor extends ActionExecutor {
 fs = has.createFileSystem(user, uri, fsConf);
 }
 
+@Override
 public InputStream getInputStream() throws IOException {
 return fs.open(new Path(uri));
 }
 
+@Override
 public OutputStream getOutputStream() throws IOException {
 return fs.create(new Path(uri));
 }
 
+@Override
 public String getContentType() {
 return "application/octet-stream";
 }
 
+@Override
 public String getName() {
 return uri.getPath();
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/38e33811/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/ConfigurationService.java 
b/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
index 4fba996..4246764 100644
--- a/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
+++ b/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
@@ -148,6 +148,7 @@ public class ConfigurationService implements Service, 
Instrumentable {
  * @param services services instance.
  * @throws ServiceException thrown if the log service could not be 
initialized.
  */
+@Override
 public void init(Services services) throws ServiceException {
 configDir = getConfigurationDirectory();
 configFile = System.getProperty(OOZIE_CONFIG_FILE, SITE_CONFIG_FILE);
@@ -179,6 +180,7 @@ public class ConfigurationService implements Service, 
Instrumentable {
 /**
  * Destroy the configuration service.
  */
+@Override
 public void destroy() {
 configuration = null;
 }
@@ -188,6 +190,7 @@ public class ConfigurationService implements Service, 
Instrumentable {
  *
  * @return {@link ConfigurationService}.
  */
+@Override
 public Class getInterface() {
 return ConfigurationService.class;
 }
@@ -316,11 +319,13 @@ public class ConfigurationService implements Service, 
Instrumentable {
 }
 }
 
+@Override
 public String[] getStrings(String name) {
 String s = get(name);
 return (s != null && s.trim().length() > 0) ? 
super.getStrings(name) : new String[0];
 }
 
+@Override
 public String[] getStrings(String name, String[] defaultValue) {
 String s = get(name);
 if (s 

oozie git commit: OOZIE-2358 Coord rerun cleanup should reuse hcat connections (rohini)

2015-09-14 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 1502633e7 -> 39e2ed5d9


OOZIE-2358 Coord rerun cleanup should reuse hcat connections (rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/39e2ed5d
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/39e2ed5d
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/39e2ed5d

Branch: refs/heads/master
Commit: 39e2ed5d93e03da03de3c9f4f4e23f4003c741b6
Parents: 1502633
Author: Rohini Palaniswamy 
Authored: Mon Sep 14 14:43:33 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Sep 14 14:43:33 2015 -0700

--
 .../oozie/command/coord/CoordRerunXCommand.java | 78 +++-
 .../apache/oozie/dependency/HCatURIHandler.java | 14 +++-
 release-log.txt |  1 +
 3 files changed, 54 insertions(+), 39 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/39e2ed5d/core/src/main/java/org/apache/oozie/command/coord/CoordRerunXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/coord/CoordRerunXCommand.java 
b/core/src/main/java/org/apache/oozie/command/coord/CoordRerunXCommand.java
index 19b9219..72e0f75 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordRerunXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordRerunXCommand.java
@@ -26,6 +26,7 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.hadoop.conf.Configuration;
@@ -141,10 +142,11 @@ public class CoordRerunXCommand extends 
RerunTransitionXCommand uriHandlerContextMap)
 throws CommandException {
 Element outputList = eAction.getChild("output-events", 
eAction.getNamespace());
 if (outputList != null) {
+
 for (Element data : (List) 
outputList.getChildren("data-out", eAction.getNamespace())) {
 String nocleanup = data.getAttributeValue("nocleanup");
 if (data.getChild("uris", data.getNamespace()) != null
@@ -152,25 +154,16 @@ public class CoordRerunXCommand extends 
RerunTransitionXCommand contextMap = new 
HashMap();
 try {
 for (String uriStr : uriArr) {
 URI uri = new URI(uriStr);
 URIHandler handler = 
Services.get().get(URIHandlerService.class).getURIHandler(uri);
 String schemeWithAuthority = uri.getScheme() + 
"://" + uri.getAuthority();
-if 
(!contextMap.containsKey(schemeWithAuthority)) {
-Context context = handler.getContext(uri, 
actionConf, coordJob.getUser(), false);
-contextMap.put(schemeWithAuthority, 
context);
+if 
(!uriHandlerContextMap.containsKey(schemeWithAuthority)) {
+Context context = handler.getContext(uri, 
coordJobConf, coordJob.getUser(), false);
+
uriHandlerContextMap.put(schemeWithAuthority, context);
 }
-handler.delete(uri, 
contextMap.get(schemeWithAuthority));
+handler.delete(uri, 
uriHandlerContextMap.get(schemeWithAuthority));
 LOG.info("Cleanup the output data " + 
uri.toString());
 }
 }
@@ -180,18 +173,10 @@ public class CoordRerunXCommand extends 
RerunTransitionXCommand> itr = 
contextMap.entrySet().iterator();
-while (itr.hasNext()) {
-Entry entry = itr.next();
-entry.getValue().destroy();
-itr.remove();
-}
-}
 }
-
 }
 }
+
 }
 else {
 LOG.info("No output-events defined in coordinator xml. Therefore 
nothing to cleanup");
@@ -363,21 +348,42 @@ public class CoordRerunXCommand extends 
RerunTransitionXCommand coordActions = 
CoordUtils.getCoordActions(rerunType, jobId, scope, false);
 if (checkAllActionsRunnable(coordActions)) {
-for (CoordinatorActionBean coordAction : coordActions) {
-String actionXml = coordAction.getActionXml();
-if (!noCleanup) {
-Element eAction = XmlUtils.parseXml(actionXml);
-cleanupOu

oozie git commit: OOZIE-2309 Enable the coord:dateOffset() function in /coordinator-app/datasets/dataset/@initial-instance (kailongs via rohini)

2015-08-10 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 9e4a26602 -> 2d270148f


OOZIE-2309 Enable the coord:dateOffset() function in 
/coordinator-app/datasets/dataset/@initial-instance (kailongs via rohini)


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

Branch: refs/heads/master
Commit: 2d270148f949c92a36e971b6b3e706b234e85b35
Parents: 9e4a266
Author: Rohini Palaniswamy 
Authored: Mon Aug 10 11:01:11 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Aug 10 11:01:11 2015 -0700

--
 .../apache/oozie/coord/CoordELEvaluator.java|  9 
 core/src/main/resources/oozie-default.xml   |  1 +
 .../oozie/coord/TestCoordELEvaluator.java   | 23 +++-
 release-log.txt |  1 +
 4 files changed, 25 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/2d270148/core/src/main/java/org/apache/oozie/coord/CoordELEvaluator.java
--
diff --git a/core/src/main/java/org/apache/oozie/coord/CoordELEvaluator.java 
b/core/src/main/java/org/apache/oozie/coord/CoordELEvaluator.java
index 8b2f456..d495dfa 100644
--- a/core/src/main/java/org/apache/oozie/coord/CoordELEvaluator.java
+++ b/core/src/main/java/org/apache/oozie/coord/CoordELEvaluator.java
@@ -73,7 +73,7 @@ public class CoordELEvaluator {
  Configuration conf) 
throws Exception {
 ELEvaluator eval = 
Services.get().get(ELService.class).createEvaluator(tag);
 setConfigToEval(eval, conf);
-SyncCoordDataset ds = getDSObject(event);
+SyncCoordDataset ds = getDSObject(eval, event);
 CoordELFunctions.configureEvaluator(eval, ds, appInst);
 return eval;
 }
@@ -104,7 +104,7 @@ public class CoordELEvaluator {
 throws Exception {
 ELEvaluator eval = 
Services.get().get(ELService.class).createEvaluator("coord-action-start");
 setConfigToEval(eval, conf);
-SyncCoordDataset ds = getDSObject(dEvent);
+SyncCoordDataset ds = getDSObject(eval, dEvent);
 SyncCoordAction appInst = new SyncCoordAction();
 appInst.setNominalTime(nominalTime);
 appInst.setActualTime(actualTime);
@@ -254,11 +254,12 @@ public class CoordELEvaluator {
  * @return
  * @throws Exception
  */
-private static SyncCoordDataset getDSObject(Element eData) throws 
Exception {
+private static SyncCoordDataset getDSObject(ELEvaluator eval, Element 
eData) throws Exception {
 SyncCoordDataset ds = new SyncCoordDataset();
 Element eDataset = eData.getChild("dataset", eData.getNamespace());
 // System.out.println("eDATA :"+ XmlUtils.prettyPrint(eData));
-Date initInstance = 
DateUtils.parseDateOozieTZ(eDataset.getAttributeValue("initial-instance"));
+String expr = eDataset.getAttributeValue("initial-instance");
+Date initInstance = DateUtils.parseDateOozieTZ(eval.evaluate(expr, 
String.class));
 ds.setInitInstance(initInstance);
 if (eDataset.getAttributeValue("frequency") != null) {
 int frequency = 
Integer.parseInt(eDataset.getAttributeValue("frequency"));

http://git-wip-us.apache.org/repos/asf/oozie/blob/2d270148/core/src/main/resources/oozie-default.xml
--
diff --git a/core/src/main/resources/oozie-default.xml 
b/core/src/main/resources/oozie-default.xml
index 12c5aa0..0a7e250 100644
--- a/core/src/main/resources/oozie-default.xml
+++ b/core/src/main/resources/oozie-default.xml
@@ -1070,6 +1070,7 @@
 
coord:current=org.apache.oozie.coord.CoordELFunctions#ph2_coord_current,
 
coord:currentRange=org.apache.oozie.coord.CoordELFunctions#ph2_coord_currentRange,
 
coord:offset=org.apache.oozie.coord.CoordELFunctions#ph2_coord_offset,
+
coord:dateOffset=org.apache.oozie.coord.CoordELFunctions#ph2_coord_dateOffset,
 
coord:latest=org.apache.oozie.coord.CoordELFunctions#ph2_coord_latest_echo,
 
coord:latestRange=org.apache.oozie.coord.CoordELFunctions#ph2_coord_latestRange_echo,
 
coord:future=org.apache.oozie.coord.CoordELFunctions#ph2_coord_future_echo,

http://git-wip-us.apache.org/repos/asf/oozie/blob/2d270148/core/src/test/java/org/apache/oozie/coord/TestCoordELEvaluator.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/coord/TestCoordELEvaluator.java 
b/

oozie git commit: OOZIE-2308 Add support for bundle:conf() function (kailongs via rohini)

2015-07-28 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master b5c1137ba -> 2d18d935d


OOZIE-2308 Add support for bundle:conf() function (kailongs via rohini)


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

Branch: refs/heads/master
Commit: 2d18d935d6661219613e49dc781a56da1de930b1
Parents: b5c1137
Author: Rohini Palaniswamy 
Authored: Tue Jul 28 10:17:43 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Jul 28 10:17:43 2015 -0700

--
 .../apache/oozie/bundle/BundleELFunctions.java  | 29 ++
 .../command/bundle/BundleSubmitXCommand.java| 24 ++--
 core/src/main/resources/oozie-default.xml   |  7 ++-
 .../bundle/TestBundleSubmitXCommand.java|  3 +-
 .../org/apache/oozie/test/XDataTestCase.java|  1 +
 .../bundle-submit-job-with-functions.xml| 60 
 docs/src/site/twiki/BundleFunctionalSpec.twiki  | 15 ++---
 release-log.txt |  1 +
 8 files changed, 126 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/2d18d935/core/src/main/java/org/apache/oozie/bundle/BundleELFunctions.java
--
diff --git a/core/src/main/java/org/apache/oozie/bundle/BundleELFunctions.java 
b/core/src/main/java/org/apache/oozie/bundle/BundleELFunctions.java
new file mode 100644
index 000..2c31ffa
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/bundle/BundleELFunctions.java
@@ -0,0 +1,29 @@
+/**
+ * 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.bundle;
+
+import org.apache.oozie.util.ELEvaluator;
+
+
+public class BundleELFunctions {
+public static String bundle_conf(String property) {
+ELEvaluator eval = ELEvaluator.getCurrent();
+return (String) eval.getVariable(property);
+}
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/2d18d935/core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java 
b/core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java
index 5d4e0f0..3df37fa 100644
--- 
a/core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java
+++ 
b/core/src/main/java/org/apache/oozie/command/bundle/BundleSubmitXCommand.java
@@ -45,12 +45,13 @@ import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.PreconditionException;
 import org.apache.oozie.command.SubmitTransitionXCommand;
 import org.apache.oozie.executor.jpa.BundleJobQueryExecutor;
+import org.apache.oozie.service.ELService;
 import org.apache.oozie.service.HadoopAccessorException;
 import org.apache.oozie.service.HadoopAccessorService;
 import org.apache.oozie.service.SchemaService;
+import org.apache.oozie.service.SchemaService.SchemaName;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.UUIDService;
-import org.apache.oozie.service.SchemaService.SchemaName;
 import org.apache.oozie.service.UUIDService.ApplicationType;
 import org.apache.oozie.util.ConfigUtils;
 import org.apache.oozie.util.DateUtils;
@@ -60,10 +61,10 @@ import org.apache.oozie.util.IOUtils;
 import org.apache.oozie.util.InstrumentUtils;
 import org.apache.oozie.util.LogUtils;
 import org.apache.oozie.util.ParamChecker;
+import org.apache.oozie.util.ParameterVerifier;
 import org.apache.oozie.util.PropertiesUtils;
 import org.apache.oozie.util.XConfiguration;
 import org.apache.oozie.util.XmlUtils;
-import org.apache.oozie.util.ParameterVerifier;
 import org.jdom.Attribute;
 import org.jdom.Element;
 import org.jdom.JDOMException;
@@ -136,7 +137,7 @@ public class BundleSubmitXCommand extends 
SubmitTransitionXCommand {
 }
 conf = resolvedVarsConf

oozie git commit: OOZIE-2287 Add support for deleting hcat partitions in fs action delete (kailongs via rohini)

2015-07-07 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 9959e2ca0 -> 4d43fe8c4


OOZIE-2287 Add support for deleting hcat partitions in fs action delete 
(kailongs via rohini)


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

Branch: refs/heads/master
Commit: 4d43fe8c4b4f40b97bf6b3d8d135a3606b98c790
Parents: 9959e2c
Author: Rohini Palaniswamy 
Authored: Tue Jul 7 12:50:45 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Jul 7 12:50:45 2015 -0700

--
 .../oozie/action/hadoop/FsActionExecutor.java   | 35 -
 .../action/hadoop/ActionExecutorTestCase.java   |  4 +-
 .../action/hadoop/TestFsActionExecutor.java | 52 
 .../src/site/twiki/WorkflowFunctionalSpec.twiki |  2 +-
 release-log.txt |  1 +
 5 files changed, 81 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/4d43fe8c/core/src/main/java/org/apache/oozie/action/hadoop/FsActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/FsActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/FsActionExecutor.java
index b25ac52..8d96a47 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/FsActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/FsActionExecutor.java
@@ -19,6 +19,7 @@
 package org.apache.oozie.action.hadoop;
 
 import java.io.IOException;
+import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -36,10 +37,13 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.oozie.action.ActionExecutor;
 import org.apache.oozie.action.ActionExecutorException;
 import org.apache.oozie.client.WorkflowAction;
+import org.apache.oozie.dependency.FSURIHandler;
+import org.apache.oozie.dependency.URIHandler;
 import org.apache.oozie.service.ConfigurationService;
 import org.apache.oozie.service.HadoopAccessorException;
 import org.apache.oozie.service.HadoopAccessorService;
 import org.apache.oozie.service.Services;
+import org.apache.oozie.service.URIHandlerService;
 import org.apache.oozie.util.XConfiguration;
 import org.apache.oozie.util.XmlUtils;
 import org.jdom.Element;
@@ -363,20 +367,28 @@ public class FsActionExecutor extends ActionExecutor {
  * @throws ActionExecutorException
  */
 public void delete(Context context, XConfiguration fsConf, Path 
nameNodePath, Path path) throws ActionExecutorException {
+URI uri = path.toUri();
+URIHandler handler;
 try {
-path = resolveToFullPath(nameNodePath, path, true);
-FileSystem fs = getFileSystemFor(path, context, fsConf);
-Path[] pathArr = FileUtil.stat2Paths(fs.globStatus(path));
-if (pathArr != null && pathArr.length > 0) {
-checkGlobMax(pathArr);
-for (Path p : pathArr) {
-if (fs.exists(p)) {
-if (!fs.delete(p, true)) {
-throw new 
ActionExecutorException(ActionExecutorException.ErrorType.ERROR, "FS005",
-"delete, path [{0}] could not delete 
path", p);
+handler = 
Services.get().get(URIHandlerService.class).getURIHandler(uri);
+if (handler instanceof FSURIHandler) {
+// Use legacy code to handle hdfs partition deletion
+path = resolveToFullPath(nameNodePath, path, true);
+FileSystem fs = getFileSystemFor(path, context, fsConf);
+Path[] pathArr = FileUtil.stat2Paths(fs.globStatus(path));
+if (pathArr != null && pathArr.length > 0) {
+checkGlobMax(pathArr);
+for (Path p : pathArr) {
+if (fs.exists(p)) {
+if (!fs.delete(p, true)) {
+throw new 
ActionExecutorException(ActionExecutorException.ErrorType.ERROR, "FS005",
+"delete, path [{0}] could not delete 
path", p);
+}
 }
 }
 }
+} else {
+handler.delete(uri, handler.getContext(uri, fsConf, 
context.getWorkflow().getUser(), false));
 }
 }
 catch (Exception ex) {
@@ -511,8 +523,9 @@ public class FsActionExecutor extends ActionExecutor {
 st = fs.getFileStatus(path);
  

oozie git commit: OOZIE-2285 Change in concurrency should trigger coord action ready command (kailongs via rohini)

2015-07-07 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master d0921f691 -> 9959e2ca0


OOZIE-2285 Change in concurrency should trigger coord action ready command 
(kailongs via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/9959e2ca
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/9959e2ca
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/9959e2ca

Branch: refs/heads/master
Commit: 9959e2ca0b2aa71f86da7b68b4905f670680b20c
Parents: d0921f6
Author: Rohini Palaniswamy 
Authored: Tue Jul 7 10:31:27 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Jul 7 10:31:27 2015 -0700

--
 .../command/coord/CoordChangeXCommand.java  | 14 +---
 .../command/coord/TestCoordChangeXCommand.java  | 37 +---
 release-log.txt |  1 +
 3 files changed, 44 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/9959e2ca/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java 
b/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
index 00c547d..d060859 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
@@ -39,17 +39,17 @@ import org.apache.oozie.client.rest.JsonBean;
 import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.PreconditionException;
 import org.apache.oozie.command.bundle.BundleStatusUpdateXCommand;
+import org.apache.oozie.executor.jpa.BatchQueryExecutor;
+import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.CoordActionGetJPAExecutor;
 import 
org.apache.oozie.executor.jpa.CoordJobGetActionByActionNumberJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
+import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.executor.jpa.SLARegistrationQueryExecutor;
 import org.apache.oozie.executor.jpa.SLARegistrationQueryExecutor.SLARegQuery;
 import org.apache.oozie.executor.jpa.SLASummaryQueryExecutor;
-import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.executor.jpa.SLASummaryQueryExecutor.SLASummaryQuery;
-import org.apache.oozie.executor.jpa.JPAExecutorException;
-import org.apache.oozie.executor.jpa.BatchQueryExecutor;
-import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.sla.SLARegistrationBean;
@@ -64,6 +64,7 @@ import org.apache.oozie.util.StatusUtils;
 public class CoordChangeXCommand extends CoordinatorXCommand {
 private final String jobId;
 private Date newEndTime = null;
+private Integer oldConcurrency = null;
 private Integer newConcurrency = null;
 private Date newPauseTime = null;
 private Date oldPauseTime = null;
@@ -344,6 +345,7 @@ public class CoordChangeXCommand extends 
CoordinatorXCommand {
 LOG.info("STARTED CoordChangeXCommand for jobId=" + jobId);
 
 try {
+oldConcurrency = this.coordJob.getConcurrency();
 if (newEndTime != null) {
 // during coord materialization, nextMaterializedTime is set to
 // startTime + n(actions materialized) * frequency and this 
can be AFTER endTime,
@@ -452,6 +454,10 @@ public class CoordChangeXCommand extends 
CoordinatorXCommand {
 updateList.add(new 
UpdateEntry(CoordJobQuery.UPDATE_COORD_JOB_CHANGE, coordJob));
 
BatchQueryExecutor.getInstance().executeBatchInsertUpdateDelete(null, 
updateList, deleteList);
 
+if (newConcurrency != null && newConcurrency > oldConcurrency) {
+queue(new CoordActionReadyXCommand(jobId));
+}
+
 return null;
 }
 catch (XException ex) {

http://git-wip-us.apache.org/repos/asf/oozie/blob/9959e2ca/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java
 
b/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java
index 7c154c8..3a91aa5 100644
--- 
a/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java
+++ 
b/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java
@@

oozie git commit: OOZIE-2284 HBaseCredentials should only add hbase-default.xml and hbase-site.xml to actionConf (rohini)

2015-07-01 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 665ed3e0d -> d0921f691


OOZIE-2284 HBaseCredentials should only add hbase-default.xml and 
hbase-site.xml to actionConf (rohini)


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

Branch: refs/heads/master
Commit: d0921f6912ec32b1e574598a1a9b8f3ffafffc05
Parents: 665ed3e
Author: Rohini Palaniswamy 
Authored: Wed Jul 1 16:49:52 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jul 1 16:49:52 2015 -0700

--
 .../java/org/apache/oozie/action/hadoop/HbaseCredentials.java | 3 ++-
 release-log.txt   | 1 +
 2 files changed, 3 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/d0921f69/core/src/main/java/org/apache/oozie/action/hadoop/HbaseCredentials.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/HbaseCredentials.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/HbaseCredentials.java
index c4633ed..d716379 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/HbaseCredentials.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/HbaseCredentials.java
@@ -58,7 +58,8 @@ public class HbaseCredentials extends Credentials {
 
 void copyHbaseConfToJobConf(JobConf jobConf, CredentialsProperties props) {
 // Create configuration using hbase-site.xml/hbase-default.xml
-Configuration hbaseConf = HBaseConfiguration.create();
+Configuration hbaseConf = new Configuration(false);
+HBaseConfiguration.addHbaseResources(hbaseConf);
 // copy cred props to hbaseconf and override if values already exists
 addPropsConf(props, hbaseConf);
 // copy cred props to jobconf and override if values already exist

http://git-wip-us.apache.org/repos/asf/oozie/blob/d0921f69/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 7b85fea..521a6ca 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2284 HBaseCredentials should only add hbase-default.xml and 
hbase-site.xml to actionConf (rohini)
 OOZIE-2286 Update Log4j and Log4j-extras to latest 1.2.x release (rkanter)
 OOZIE-2250 Show log for WAITING and TIMEDOUT coord actions (kailongs via 
rohini)
 OOZIE-2262 Fix log streaming from other server with start/end filter (kailongs 
via rohini)



oozie git commit: OOZIE-2250 Show log for WAITING and TIMEDOUT coord actions (kailongs via rohini)

2015-06-29 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 10549ef29 -> 9948829a1


OOZIE-2250 Show log for WAITING and TIMEDOUT coord actions (kailongs via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/9948829a
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/9948829a
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/9948829a

Branch: refs/heads/master
Commit: 9948829a1b4897ddc5ce5e9409ccc91d080b03c1
Parents: 10549ef
Author: Rohini Palaniswamy 
Authored: Mon Jun 29 11:52:13 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Jun 29 11:52:13 2015 -0700

--
 release-log.txt |  1 +
 webapp/src/main/webapp/oozie-console.js | 13 -
 2 files changed, 13 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/9948829a/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index a53fcb6..4fb97f8 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2250 Show log for WAITING and TIMEDOUT coord actions (kailongs via 
rohini)
 OOZIE-2262 Fix log streaming from other server with start/end filter (kailongs 
via rohini)
 OOZIE-2159 'oozie validate' command should be moved server-side (seoeun25 via 
rkanter)
 OOZIE-2271 Upgrade Tomcat to 6.0.44 (rkanter)

http://git-wip-us.apache.org/repos/asf/oozie/blob/9948829a/webapp/src/main/webapp/oozie-console.js
--
diff --git a/webapp/src/main/webapp/oozie-console.js 
b/webapp/src/main/webapp/oozie-console.js
index ab184e3..574832f 100644
--- a/webapp/src/main/webapp/oozie-console.js
+++ b/webapp/src/main/webapp/oozie-console.js
@@ -1037,6 +1037,7 @@ function coordJobDetailsPopup(response, request) {
 var actionsTextBox = new Ext.form.TextField({
  fieldLabel: 'ActionsList',
  name: 'ActionsList',
+ id: 'actions_text_box',
  width: 150,
  value: ''
  });
@@ -1052,6 +1053,7 @@ function coordJobDetailsPopup(response, request) {
 var searchFilterBox = new Ext.form.TextField({
  fieldLabel: 'searchFilterBox',
  name: 'searchFilterBox',
+ id: 'search_filter_box',
  width: 350,
  value: ''
 });
@@ -1278,7 +1280,15 @@ function coordJobDetailsPopup(response, request) {
 function showWorkflowPopup(thisGrid, rowIndex, cellIndex, e) {
 var actionStatus = thisGrid.store.data.items[rowIndex].data;
 var workflowId = actionStatus["externalId"];
-jobDetailsGridWindow(workflowId);
+if(workflowId == null) {
+jobDetailsTab.getComponent('coord_job_log').show();
+
Ext.getCmp('actions_text_box').setValue(actionStatus["id"].split("@")[1]);
+Ext.getCmp('search_filter_box').setValue('recent=5m');
+fetchLogs(coordJobId, actionsTextBox.getValue());
+}
+else {
+jobDetailsGridWindow(workflowId);
+}
 }
 // alert("Coordinator PopUP 4 inside coordDetailsPopup ");
 function showCoordActionContextMenu(thisGrid, rowIndex, cellIndex, e) {
@@ -1500,6 +1510,7 @@ function coordJobDetailsPopup(response, request) {
},{
title: 'Coord Job Log',
items: jobLogArea,
+   id: 'coord_job_log',
tbar: [
   actionsText,actionsTextBox, searchFilter, searchFilterBox, 
getLogButton, {xtype: 'tbfill'}, logStatus]
},



oozie git commit: OOZIE-2262 Fix log streaming from other server with start/end filter (kailongs via rohini)

2015-06-26 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master cc94ad8e5 -> 10549ef29


OOZIE-2262 Fix log streaming from other server with start/end filter (kailongs 
via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/10549ef2
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/10549ef2
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/10549ef2

Branch: refs/heads/master
Commit: 10549ef294b0e08a3802eb58d7c9ef5f1afe15ba
Parents: cc94ad8
Author: Rohini Palaniswamy 
Authored: Fri Jun 26 13:47:04 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Fri Jun 26 13:47:04 2015 -0700

--
 core/src/main/java/org/apache/oozie/util/AuthUrlClient.java | 6 --
 release-log.txt | 3 ++-
 2 files changed, 6 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/10549ef2/core/src/main/java/org/apache/oozie/util/AuthUrlClient.java
--
diff --git a/core/src/main/java/org/apache/oozie/util/AuthUrlClient.java 
b/core/src/main/java/org/apache/oozie/util/AuthUrlClient.java
index 5de8471..b45a96a 100644
--- a/core/src/main/java/org/apache/oozie/util/AuthUrlClient.java
+++ b/core/src/main/java/org/apache/oozie/util/AuthUrlClient.java
@@ -23,8 +23,10 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
 import java.net.HttpURLConnection;
 import java.net.URL;
+import java.net.URLEncoder;
 import java.security.PrivilegedExceptionAction;
 import java.util.Map;
 
@@ -142,7 +144,7 @@ public class AuthUrlClient {
 return reader;
 }
 
-public static String getQueryParamString(Map params) {
+public static String getQueryParamString(Map params) 
throws UnsupportedEncodingException {
 StringBuilder stringBuilder = new StringBuilder();
 if (params == null || params.isEmpty()) {
 return "";
@@ -153,7 +155,7 @@ public class AuthUrlClient {
 String value = params.get(key)[0]; // We don't support multi 
value.
 stringBuilder.append(key);
 stringBuilder.append("=");
-stringBuilder.append(value);
+stringBuilder.append(URLEncoder.encode(value,"UTF-8"));
 }
 }
 return stringBuilder.toString();

http://git-wip-us.apache.org/repos/asf/oozie/blob/10549ef2/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index fe8d1fc..a53fcb6 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,9 +1,10 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2262 Fix log streaming from other server with start/end filter (kailongs 
via rohini)
 OOZIE-2159 'oozie validate' command should be moved server-side (seoeun25 via 
rkanter)
 OOZIE-2271 Upgrade Tomcat to 6.0.44 (rkanter)
 OOZIE-2266 Fix 'total' actions returned in coordinator job info (sai-krish via 
rkanter)
-OOZIE-2264 Fix coord:offset(n,"DAY") to resolve correct data set(kailongs via 
puru)
+OOZIE-2264 Fix coord:offset(n,"DAY") to resolve correct data set (kailongs via 
puru)
 OOZIE-2178 fix javadoc to compile on JDK8 (rkanter)
 OOZIE-2268 Update ActiveMQ version for security and other fixes (rkanter)
 OOZIE-2215 Support glob in FS EL function (ryota)



svn commit: r9213 [1/3] - in /release/oozie/4.2.0: ./ oozie-4.2.0.tar.gz oozie-4.2.0.tar.gz.asc oozie-4.2.0.tar.gz.md5 rat_report.txt release-log.txt

2015-06-03 Thread rohini
Author: rohini
Date: Wed Jun  3 19:12:08 2015
New Revision: 9213

Log:
Adding Oozie 4.2.0 release artifacts (shwethags via rohini)

Added:
release/oozie/4.2.0/
release/oozie/4.2.0/oozie-4.2.0.tar.gz   (with props)
release/oozie/4.2.0/oozie-4.2.0.tar.gz.asc
release/oozie/4.2.0/oozie-4.2.0.tar.gz.md5
release/oozie/4.2.0/rat_report.txt
release/oozie/4.2.0/release-log.txt

Added: release/oozie/4.2.0/oozie-4.2.0.tar.gz
==
Binary file - no diff available.

Propchange: release/oozie/4.2.0/oozie-4.2.0.tar.gz
--
svn:mime-type = application/octet-stream

Added: release/oozie/4.2.0/oozie-4.2.0.tar.gz.asc
==
--- release/oozie/4.2.0/oozie-4.2.0.tar.gz.asc (added)
+++ release/oozie/4.2.0/oozie-4.2.0.tar.gz.asc Wed Jun  3 19:12:08 2015
@@ -0,0 +1,11 @@
+-BEGIN PGP SIGNATURE-
+Comment: GPGTools - https://gpgtools.org
+
+iQEcBAABCgAGBQJVXxJfAAoJEKDm+fXZa/D94RcH/0bqLfx+BRDjP7mScgpLxg3G
+xqkaFbLQU3S/95osMUFbsEgmin/XnZF2SSSxMvrmEM/e4BCUEKsAr6TL3PguK5YK
+VZDUemTp6RI4tQkvEn9Y/vz3pdYmiMEpX3vQ/MSkwxwT7jjtCUk17AS3UOfYrJZ4
+3N/ueYGaa4CWzyGLav8ErwB1Nw9J9nI0qz3RMOtaFnfQ0Kn7g7OkFF7nsNTYLBDA
+mR27Rm1TAR6p5uB8i2TzHMNZJ8zf7QKXhb2kkvSV9aRu5u731NDVVn1OI8kVKoWE
+6YIT87lfd3fL/45EMn8gj/VwoZTRfSxtrkjq116y3z9FW5mk1Q5r7w/o4RrFcCQ=
+=cO1Z
+-END PGP SIGNATURE-

Added: release/oozie/4.2.0/oozie-4.2.0.tar.gz.md5
==
--- release/oozie/4.2.0/oozie-4.2.0.tar.gz.md5 (added)
+++ release/oozie/4.2.0/oozie-4.2.0.tar.gz.md5 Wed Jun  3 19:12:08 2015
@@ -0,0 +1,2 @@
+$ md5 oozie-4.2.0.tar.gz
+MD5 (oozie-4.2.0.tar.gz) = 70066a82b9e93ef28bfec8c6babf0aac




svn commit: r9213 [3/3] - in /release/oozie/4.2.0: ./ oozie-4.2.0.tar.gz oozie-4.2.0.tar.gz.asc oozie-4.2.0.tar.gz.md5 rat_report.txt release-log.txt

2015-06-03 Thread rohini
Added: release/oozie/4.2.0/release-log.txt
==
--- release/oozie/4.2.0/release-log.txt (added)
+++ release/oozie/4.2.0/release-log.txt Wed Jun  3 19:12:08 2015
@@ -0,0 +1,1220 @@
+-- Oozie 4.2.0 release
+
+OOZIE-2240 add configuration to disable email attachment support (egashira via 
shwethags)
+OOZIE-1963 Create a Hive Server 2 example (qwertymaniac via shwethags)
+OOZIE-1993 Rerun fails during join in certain condition (shwethags)
+OOZIE-2236 Need to package hive-hcatalog-server-extensions.jar in the hcatalog 
sharelib (venkatnrangan via bzhang)
+OOZIE-2232 Oozie should invalidate bulk write command when "-filter" is 
missing (venkatnrangan via bzhang)
+OOZIE-2224 Add example worklfow.xml for hive in secure mode (venkatnrangan via 
bzhang)
+OOZIE-2227 PartitionDependencyManagerService keeps on purging delete coord 
actions (puru)
+OOZIE-2163 Remove CoordinatorStore (seoeun25 via bzhang)
+OOZIE-2221 Oozie audit log has null id for some of input request (puru)
+OOZIE-2223 Improve documentation with regard to Java action retries 
(ben.roling via bzhang)
+OOZIE-2218 META-INF directories in the war file have 777 permissions (rkanter)
+OOZIE-2130 Add EL Function for offsetting a date by a timezone amount 
including DST (rkanter)
+OOZIE-2199 Ooziedb.cmd and oozie-setup.ps1 are missing jars in lib/ for 
classpath on Windows (venkatnrangan via bzhang)
+OOZIE-2012 coordinator with an invalid cron frequency throws NPE after 
validation (bzhang)
+OOZIE-2129 Duplicate child jobs per instance (jaydeepvishwakarma via shwethags)
+OOZIE-2214 fix test case TestCoordRerunXCommand.testCoordRerunDateNeg (ryota)
+OOZIE-2213 oozie-setup.ps1 should use "start-process" rather than "cmd /c" to 
invoke OozieSharelibCLI or OozieDBCLI commands (bzhang)
+OOZIE-2210 Update extjs 2.2 link (bzhang)
+OOZIE-2205 add option to load default/site.xml to actionConf on compute node 
(ryota)
+OOZIE-2206 Change Reaper mode on ChildReaper in ZKLocksService (ryota)
+OOZIE-2209 Oozie jobs to set "java.io.tmpdir" to "./tmp" (kailongs via rohini)
+OOZIE-2051 Allows multiple default action configuration files per action 
(venkatnrangan via bzhang)
+OOZIE-2170 Oozie should automatically set configs to make Spark jobs show up 
in the Spark History Server (rkanter)
+OOZIE-2140 Audit Log should be shown in Oozie UI (puru)
+OOZIE-2139 Coord update doesn't work for job which is submitted by bundle 
(puru)
+OOZIE-1726 Oozie does not support _HOST when configuring kerberos security 
(venkatnrangan via bzhang)
+OOZIE-2197 ooziedb.cmd command failed due to classpath being too long on 
windows (me.venkatr via bzhang)
+OOZIE-2182 SLA alert commands are not in sync with doc (puru)
+OOZIE-2191 Upgrade jackson version for hadoop-2 profile (ryota)
+OOZIE-2183 provide an option to disable cleanup for output dataset in rerun 
(ryota)
+OOZIE-2146 Add option to filter sla information by bundle id or name (ryota)
+OOZIE-2188 Fix typos in twiki documentation ( jacobtolar via puru)
+OOZIE-2174 Add missing admin commands to OozieClient and OozieCLI (rkanter)
+OOZIE-2186 Upgrade Tomcat to 6.0.43 (rkanter)
+OOZIE-2181 JsonToBean has some missing and incorrect mappings (rkanter)
+OOZIE-2184 Change default value of action tmp dir removal to true (Viji via 
harsh)
+OOZIE-2006 Hive 2 action should support tez delegation tokens (bzhang)
+OOZIE-2180 Oozie hive2 should explicitly depend on commons-io rather than 
transitive dependency of hive-exec (bzhang)
+OOZIE-2177 Parameterize javadoc plugin configuration (ryota)
+OOZIE-2169 Fix return type for fs:dirSize, fs:fileSize and fs:blockSize in WF 
spec (apivovarov via puru)
+OOZIE-2171 Some error code messages are not parsed correctly due to unescaped 
single quote (rkanter)
+OOZIE-2108 bulk kill, suspend, resume jobs using existing filter, offset, len, 
and jobtype params (bzhang)
+OOZIE-2167 TestCoordMaterializeTransitionXCommand fails (rkanter)
+OOZIE-1964 Hive Server 2 action doesn't return Hadoop Job IDs (rkanter)
+OOZIE-2126 SSH action can be too fast for Oozie sometimes (rkanter)
+OOZIE-2142 Changing the JT whitelist causes running Workflows to stay RUNNING 
forever (rkanter)
+OOZIE-2164 make master parameterizable in Spark action example (wypoon via 
rkanter)
+OOZIE-2155 Incorrect DST Shifts are occurring based on the Database timezone 
(rkanter)
+OOZIE-2156 override hive.querylog.location in hive-site.xml (ryota)
+OOZIE-2153 modify HCatCredentialHelper to use HCatClient (ryota)
+OOZIE-2132 FS actions are not retried (ryota)
+OOZIE-2147 Oozie upgrade from 4.0.0 to 4.1.0 fails with CLOBs of zero length 
with Oracle DB (venkatnrangan via bzhang)
+OOZIE-2158 Overrides in action conf in streaming action do not work (rohini)
+OOZIE-2042 Max substitution for config variables should be configurable 
(seoeun25 via puru)
+OOZIE-1913 Devise a way to turn off SLA alerts for bundle/coordinator flexi

svn commit: r9213 [2/3] - in /release/oozie/4.2.0: ./ oozie-4.2.0.tar.gz oozie-4.2.0.tar.gz.asc oozie-4.2.0.tar.gz.md5 rat_report.txt release-log.txt

2015-06-03 Thread rohini
Added: release/oozie/4.2.0/rat_report.txt
==
--- release/oozie/4.2.0/rat_report.txt (added)
+++ release/oozie/4.2.0/rat_report.txt Wed Jun  3 19:12:08 2015
@@ -0,0 +1,1430 @@
+
+*
+Summary
+---
+Generated at: 2015-05-22T16:56:21+05:30
+Notes: 10
+Binaries: 36
+Archives: 0
+Standards: 1347
+
+Apache Licensed: 1347
+Generated Documents: 0
+
+JavaDocs are generated and so license header is optional
+Generated files do not required license headers
+
+0 Unknown Licenses
+
+***
+
+Unapproved licenses:
+
+
+***
+
+Archives:
+
+*
+  Files with Apache License headers will be marked AL
+  Binary files (which do not require AL headers) will be marked B
+  Compressed archives will be marked A
+  Notices, licenses etc will be marked N
+  ALbin/create-release-artifact
+  ALbin/mkdistro.sh
+  ALbin/test-patch
+  ALbin/test-patch-00-clean
+  ALbin/test-patch-05-patch-raw-analysis
+  ALbin/test-patch-08-rat
+  ALbin/test-patch-09-javadoc
+  ALbin/test-patch-10-compile
+  ALbin/test-patch-15-backwards-compatibility
+  ALbin/test-patch-20-tests
+  ALbin/test-patch-30-distro
+  ALbin/test-patch-find-new-patch-available-jiras
+  ALbin/test-patch-XX-template
+  ALbuilds/runHudsonCIBuild.sh
+  ALclient/pom.xml
+  ALclient/src/main/bin/oozie
+  ALclient/src/main/java/org/apache/oozie/AppType.java
+  ALclient/src/main/java/org/apache/oozie/BuildInfo.java
+  ALclient/src/main/java/org/apache/oozie/cli/CLIParser.java
+  ALclient/src/main/java/org/apache/oozie/cli/OozieCLI.java
+  ALclient/src/main/java/org/apache/oozie/cli/OozieCLIException.java
+  ALclient/src/main/java/org/apache/oozie/client/AuthOozieClient.java
+  ALclient/src/main/java/org/apache/oozie/client/BulkResponse.java
+  ALclient/src/main/java/org/apache/oozie/client/BundleJob.java
+  ALclient/src/main/java/org/apache/oozie/client/CoordinatorAction.java
+  ALclient/src/main/java/org/apache/oozie/client/CoordinatorJob.java
+  ALclient/src/main/java/org/apache/oozie/client/event/Event.java
+  AL
client/src/main/java/org/apache/oozie/client/event/jms/JMSHeaderConstants.java
+  AL
client/src/main/java/org/apache/oozie/client/event/jms/JMSMessagingUtils.java
+  AL
client/src/main/java/org/apache/oozie/client/event/jms/JSONMessageDeserializer.java
+  AL
client/src/main/java/org/apache/oozie/client/event/jms/MessageDeserializer.java
+  ALclient/src/main/java/org/apache/oozie/client/event/JobEvent.java
+  AL
client/src/main/java/org/apache/oozie/client/event/message/CoordinatorActionMessage.java
+  AL
client/src/main/java/org/apache/oozie/client/event/message/EventMessage.java
+  AL
client/src/main/java/org/apache/oozie/client/event/message/JobMessage.java
+  AL
client/src/main/java/org/apache/oozie/client/event/message/SLAMessage.java
+  AL
client/src/main/java/org/apache/oozie/client/event/message/WorkflowJobMessage.java
+  ALclient/src/main/java/org/apache/oozie/client/event/SLAEvent.java
+  ALclient/src/main/java/org/apache/oozie/client/JMSConnectionInfo.java
+  AL
client/src/main/java/org/apache/oozie/client/JMSConnectionInfoWrapper.java
+  ALclient/src/main/java/org/apache/oozie/client/Job.java
+  ALclient/src/main/java/org/apache/oozie/client/OozieClient.java
+  ALclient/src/main/java/org/apache/oozie/client/OozieClientException.java
+  ALclient/src/main/java/org/apache/oozie/client/rest/JsonTags.java
+  ALclient/src/main/java/org/apache/oozie/client/rest/JsonToBean.java
+  ALclient/src/main/java/org/apache/oozie/client/rest/JsonUtils.java
+  ALclient/src/main/java/org/apache/oozie/client/rest/RestConstants.java
+  AL
client/src/main/java/org/apache/oozie/client/retry/ConnectionRetriableClient.java
+  ALclient/src/main/java/org/apache/oozie/client/SLAEvent.java
+  ALclient/src/main/java/org/apache/oozie/client/WorkflowAction.java
+  ALclient/src/main/java/org/apache/oozie/client/WorkflowJob.java
+  ALclient/src/main/java/org/apache/oozie/client/XOozieClient.java
+  ALclient/src/main/resources/distcp-action-0.1.xsd
+  ALclient/src/main/resources/distcp-action-0.2.xsd
+  ALclient/src/main/resources/email-action-0.1.xsd
+  ALclient/src/main/resources/email-action-0.2.xsd
+  ALclient/src/main/resources/gms-oozie-sla-0.1.xsd
+  ALclient/src/main/resources/hive-action-0.2.xsd
+  ALclient/src/main/resources/hive-action-0.3.xsd
+  ALclient/src/main/resources/hive-action-0.4.xsd
+  ALclient/src/main/resources/hive-action-0.5.xsd
+  ALclient/src/main/resources/hive2-action-0.1.xsd
+  ALclient/src/main/resources/oozie-buildinfo.properties
+  ALclient/src/main/resources/oozie-bundle-0.1.xsd
+  A

oozie git commit: OOZIE-2209 Oozie jobs to set java.io.tmpdir to ./tmp (kailongs via rohini)

2015-04-17 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master c37738134 -> e311d1947


OOZIE-2209 Oozie jobs to set java.io.tmpdir to ./tmp (kailongs via rohini)


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

Branch: refs/heads/master
Commit: e311d19474001347db77c9eff4475f276165a274
Parents: c377381
Author: Rohini Palaniswamy 
Authored: Fri Apr 17 12:24:52 2015 -0700
Committer: Rohini Palaniswamy 
Committed: Fri Apr 17 12:24:52 2015 -0700

--
 .../oozie/action/hadoop/JavaActionExecutor.java |  30 +++-
 .../action/hadoop/TestJavaActionExecutor.java   | 138 ++-
 release-log.txt |   1 +
 3 files changed, 135 insertions(+), 34 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/e311d194/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index cf22ed6..163956e 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -98,6 +98,7 @@ public class JavaActionExecutor extends ActionExecutor {
 public static final String HADOOP_MAP_MEMORY_MB = 
"mapreduce.map.memory.mb";
 public static final String HADOOP_CHILD_JAVA_OPTS = 
"mapred.child.java.opts";
 public static final String HADOOP_MAP_JAVA_OPTS = 
"mapreduce.map.java.opts";
+public static final String HADOOP_REDUCE_JAVA_OPTS = 
"mapreduce.reduce.java.opts";
 public static final String HADOOP_CHILD_JAVA_ENV = "mapred.child.env";
 public static final String HADOOP_MAP_JAVA_ENV = "mapreduce.map.env";
 public static final String YARN_AM_RESOURCE_MB = 
"yarn.app.mapreduce.am.resource.mb";
@@ -114,6 +115,7 @@ public class JavaActionExecutor extends ActionExecutor {
 private static final String FAILED_KILLED = "FAILED/KILLED";
 protected XLog LOG = XLog.getLog(getClass());
 private static final Pattern heapPattern = 
Pattern.compile("-Xmx(([0-9]+)[mMgG])");
+private static final String JAVA_TMP_DIR_SETTINGS = "-Djava.io.tmpdir=";
 public static final String CONF_HADOOP_YARN_UBER_MODE = 
"oozie.action.launcher." + HADOOP_YARN_UBER_MODE;
 public static final String HADOOP_JOB_CLASSLOADER = 
"mapreduce.job.classloader";
 public static final String HADOOP_USER_CLASSPATH_FIRST = 
"mapreduce.user.classpath.first";
@@ -389,6 +391,31 @@ public class JavaActionExecutor extends ActionExecutor {
 }
 }
 
+void updateConfForJavaTmpDir(Configuration conf) {
+String mapOpts = conf.get(HADOOP_MAP_JAVA_OPTS);
+String reduceOpts = conf.get(HADOOP_REDUCE_JAVA_OPTS);
+String childOpts = conf.get(HADOOP_CHILD_JAVA_OPTS);
+String amChildOpts = conf.get(YARN_AM_COMMAND_OPTS);
+String oozieJavaTmpDirSetting = "-Djava.io.tmpdir=./tmp";
+if (childOpts == null) {
+conf.set(HADOOP_CHILD_JAVA_OPTS, oozieJavaTmpDirSetting);
+} else {
+conf.set(HADOOP_CHILD_JAVA_OPTS, childOpts + " " + 
oozieJavaTmpDirSetting);
+}
+
+if (mapOpts != null && !mapOpts.contains(JAVA_TMP_DIR_SETTINGS)) {
+conf.set(HADOOP_MAP_JAVA_OPTS, mapOpts + " " + 
oozieJavaTmpDirSetting);
+}
+
+if (reduceOpts != null && !reduceOpts.contains(JAVA_TMP_DIR_SETTINGS)) 
{
+conf.set(HADOOP_REDUCE_JAVA_OPTS, reduceOpts + " " + 
oozieJavaTmpDirSetting);
+}
+
+if (amChildOpts != null && 
!amChildOpts.contains(JAVA_TMP_DIR_SETTINGS)) {
+conf.set(YARN_AM_COMMAND_OPTS, amChildOpts + " " + 
oozieJavaTmpDirSetting);
+}
+}
+
 private HashMap> populateEnvMap(String input) {
 HashMap> envMaps = new HashMap>();
 String[] envEntries = input.split(",");
@@ -495,6 +522,7 @@ public class JavaActionExecutor extends ActionExecutor {
 
 // set cancel.delegation.token in actionConf that child job 
doesn't cancel delegation token
 
actionConf.setBoolean("mapreduce.job.complete.cancel.delegation.tokens", false);
+updateConfForJavaTmpDir(actionConf);
 return actionConf;
 }
 catch (IOException ex) {
@@ -918,7 +946,7 @@ public class Java

oozie git commit: OOZIE-2158 Overrides in action conf in streaming action do not work (rohini)

2015-03-02 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 532dce684 -> 746be5c29


OOZIE-2158 Overrides in action conf in streaming action do not work (rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/746be5c2
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/746be5c2
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/746be5c2

Branch: refs/heads/master
Commit: 746be5c2917310ab7202767a1a6bb2c79ad26dd6
Parents: 532dce6
Author: Rohini Palaniswamy 
Authored: Mon Mar 2 15:03:01 2015 -0800
Committer: Rohini Palaniswamy 
Committed: Mon Mar 2 15:03:01 2015 -0800

--
 release-log.txt |  1 +
 .../apache/oozie/action/hadoop/PipesMain.java   | 20 ---
 .../oozie/action/hadoop/StreamingMain.java  | 21 ---
 .../hadoop/TestMapReduceActionExecutor.java | 60 ++--
 4 files changed, 82 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/746be5c2/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index ca6dfc2..0851d1c 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.2.0 release (trunk - unreleased)
 
+OOZIE-2158 Overrides in action conf in streaming action do not work (rohini)
 OOZIE-2042 Max substitution for config variables should be configurable 
(seoeun25 via puru)
 OOZIE-1913 Devise a way to turn off SLA alerts for bundle/coordinator flexibly 
(puru)
 OOZIE-2071 Add a Spark example (pavan kumar via rkanter)

http://git-wip-us.apache.org/repos/asf/oozie/blob/746be5c2/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PipesMain.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PipesMain.java 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PipesMain.java
index bf91b43..5b5e9db 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PipesMain.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PipesMain.java
@@ -32,32 +32,32 @@ public class PipesMain extends MapReduceMain {
 }
 
 @Override
-protected RunningJob submitJob(JobConf jobConf) throws Exception {
-String value = jobConf.get("oozie.pipes.map");
+protected void addActionConf(JobConf jobConf, Configuration actionConf) {
+String value = actionConf.get("oozie.pipes.map");
 if (value != null) {
 jobConf.setBoolean("hadoop.pipes.java.mapper", true);
 jobConf.set("mapred.mapper.class", value);
 }
-value = jobConf.get("oozie.pipes.reduce");
+value = actionConf.get("oozie.pipes.reduce");
 if (value != null) {
 jobConf.setBoolean("hadoop.pipes.java.reducer", true);
 jobConf.set("mapred.reducer.class", value);
 }
-value = jobConf.get("oozie.pipes.inputformat");
+value = actionConf.get("oozie.pipes.inputformat");
 if (value != null) {
 jobConf.setBoolean("hadoop.pipes.java.recordreader", true);
 jobConf.set("mapred.input.format.class", value);
 }
-value = jobConf.get("oozie.pipes.partitioner");
+value = actionConf.get("oozie.pipes.partitioner");
 if (value != null) {
 jobConf.set("mapred.partitioner.class", value);
 }
-value = jobConf.get("oozie.pipes.writer");
+value = actionConf.get("oozie.pipes.writer");
 if (value != null) {
 jobConf.setBoolean("hadoop.pipes.java.recordwriter", true);
 jobConf.set("mapred.output.format.class", value);
 }
-value = jobConf.get("oozie.pipes.program");
+value = actionConf.get("oozie.pipes.program");
 if (value != null) {
 jobConf.set("hadoop.pipes.executable", value);
 if (value.contains("#")) {
@@ -65,6 +65,12 @@ public class PipesMain extends MapReduceMain {
 }
 }
 
+super.addActionConf(jobConf, actionConf);
+}
+
+@Override
+protected RunningJob submitJob(JobConf jobConf) throws Exception {
+
 //propagate delegation related props from launcher job to MR job
 if (getFilePathFromEnv("HADOOP_TOKEN_FILE_LOCATION") != null) {
 jobConf.set("mapreduce.job.credentials.binary", 
getFilePathFromEnv("HADOOP_TOKEN_FILE_LOCATION"));

http://git-wip-us.apache.org/repos/asf/oozie/blob/746be5c2/sharelib/streaming/src/m

oozie git commit: OOZIE-2133 Support getting ATS delegation tokens for tez jobs (rohini)

2015-02-09 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 9b1f6b653 -> 97b21af4d


OOZIE-2133 Support getting ATS delegation tokens for tez jobs (rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/97b21af4
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/97b21af4
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/97b21af4

Branch: refs/heads/master
Commit: 97b21af4d305bdb12f0642f641c6ad18d3abfa15
Parents: 9b1f6b6
Author: Rohini Palaniswamy 
Authored: Mon Feb 9 12:27:42 2015 -0800
Committer: Rohini Palaniswamy 
Committed: Mon Feb 9 12:27:42 2015 -0800

--
 .../oozie/action/hadoop/JavaActionExecutor.java | 14 +++
 .../oozie/service/ConfigurationService.java | 15 +++
 core/src/main/resources/oozie-default.xml   | 12 +++
 .../action/hadoop/TestJavaActionExecutor.java   | 99 +---
 release-log.txt |  1 +
 5 files changed, 129 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/97b21af4/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index 3383522..5f8646d 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -93,6 +93,7 @@ public class JavaActionExecutor extends ActionExecutor {
 public final static String MAX_EXTERNAL_STATS_SIZE = 
"oozie.external.stats.max.size";
 public static final String ACL_VIEW_JOB = "mapreduce.job.acl-view-job";
 public static final String ACL_MODIFY_JOB = "mapreduce.job.acl-modify-job";
+public static final String HADOOP_YARN_TIMELINE_SERVICE_ENABLED = 
"yarn.timeline-service.enabled";
 public static final String HADOOP_YARN_UBER_MODE = 
"mapreduce.job.ubertask.enable";
 public static final String HADOOP_MAP_MEMORY_MB = 
"mapreduce.map.memory.mb";
 public static final String HADOOP_CHILD_JAVA_OPTS = 
"mapred.child.java.opts";
@@ -279,6 +280,17 @@ public class JavaActionExecutor extends ActionExecutor {
 }
 }
 
+void injectLauncherTimelineServiceEnabled(Configuration launcherConf, 
Configuration actionConf) {
+// Getting delegation token for ATS. If tez-site.xml is present in 
distributed cache, turn on timeline service.
+if (actionConf.get("oozie.launcher." + 
HADOOP_YARN_TIMELINE_SERVICE_ENABLED) == null
+&& ConfigurationService.getBoolean("oozie.action.launcher." + 
HADOOP_YARN_TIMELINE_SERVICE_ENABLED)) {
+String cacheFiles = launcherConf.get("mapred.cache.files");
+if (cacheFiles != null && cacheFiles.contains("tez-site.xml")) {
+launcherConf.setBoolean(HADOOP_YARN_TIMELINE_SERVICE_ENABLED, 
true);
+}
+}
+}
+
 void updateConfForUberMode(Configuration launcherConf) {
 
 // child.env
@@ -907,6 +919,8 @@ public class JavaActionExecutor extends ActionExecutor {
 }
 }
 
+injectLauncherTimelineServiceEnabled(launcherJobConf, actionConf);
+
 // properties from action that are needed by the launcher (e.g. 
QUEUE NAME, ACLs)
 // maybe we should add queue to the WF schema, below job-tracker
 actionConfToLauncherConf(actionConf, launcherJobConf);

http://git-wip-us.apache.org/repos/asf/oozie/blob/97b21af4/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/ConfigurationService.java 
b/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
index d710744..d602e53 100644
--- a/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
+++ b/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
@@ -36,8 +36,11 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.Arrays;
+
 import org.apache.oozie.util.ZKUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Built in service that initializes the services configuration.
  * 
@@ -459,6 +462,18 @@ public class ConfigurationService implements Service, 
Instrumentable {
 }
 }
 
+@VisibleForTesting
+public static void set(String name, String value) {
+Configuration conf = Services.get().getConf();
+conf.set(name, value);
+}
+
+@Visibl

svn commit: r922320 - /websites/production/oozie/content/

2014-09-15 Thread rohini
Author: rohini
Date: Mon Sep 15 21:20:04 2014
New Revision: 922320

Log:
Ordered committers alphabetically and added doap_Oozie.rdf

Added:
websites/production/oozie/content/
  - copied from r922319, websites/staging/oozie/trunk/content/



svn commit: r1625165 - in /oozie/site/trunk: doap_Oozie.rdf pom.xml

2014-09-15 Thread rohini
Author: rohini
Date: Mon Sep 15 21:12:43 2014
New Revision: 1625165

URL: http://svn.apache.org/r1625165
Log:
OOZIE-1795: Please create a DOAP file for your TLP (rohini)

Added:
oozie/site/trunk/doap_Oozie.rdf
Modified:
oozie/site/trunk/pom.xml

Added: oozie/site/trunk/doap_Oozie.rdf
URL: 
http://svn.apache.org/viewvc/oozie/site/trunk/doap_Oozie.rdf?rev=1625165&view=auto
==
--- oozie/site/trunk/doap_Oozie.rdf (added)
+++ oozie/site/trunk/doap_Oozie.rdf Mon Sep 15 21:12:43 2014
@@ -0,0 +1,58 @@
+
+
+http://usefulinc.com/ns/doap#"; 
+ xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"; 
+ xmlns:asfext="http://projects.apache.org/ns/asfext#";
+ xmlns:foaf="http://xmlns.com/foaf/0.1/";>
+
+  http://oozie.apache.org";>
+2014-09-15
+http://spdx.org/licenses/Apache-2.0"; />
+Apache Oozie
+http://oozie.apache.org"; />
+http://oozie.apache.org"; />
+Oozie is a workflow scheduler system to manage Apache Hadoop 
jobs.
+Oozie is a workflow scheduler system to manage Apache Hadoop 
jobs. Oozie is integrated with the rest of the Hadoop stack supporting several 
types of Hadoop jobs out of the box (such as Java map-reduce, Streaming 
map-reduce, Pig, Hive, Sqoop and Distcp) as well as system specific jobs (such 
as Java programs and shell scripts).
+https://issues.apache.org/jira/browse/OOZIE"; />
+http://oozie.apache.org/mail-lists.html"; />
+http://www.apache.org/dyn/closer.cgi/oozie/"; 
/>
+Java
+JavaScript
+http://projects.apache.org/category/big-data"; />
+
+  
+Apache Oozie
+2014-03-31
+4.0.1
+  
+
+
+  
+https://git-wip-us.apache.org/repos/asf/oozie.git"/>
+https://git-wip-us.apache.org/repos/asf?p=oozie.git"/>
+  
+
+
+  
+Mohammad Islam
+  mailto:misla...@yahoo.com"/>
+  
+
+  
+

Modified: oozie/site/trunk/pom.xml
URL: 
http://svn.apache.org/viewvc/oozie/site/trunk/pom.xml?rev=1625165&r1=1625164&r2=1625165&view=diff
==
--- oozie/site/trunk/pom.xml (original)
+++ oozie/site/trunk/pom.xml Mon Sep 15 21:12:43 2014
@@ -191,6 +191,14 @@
 
   
   
+puru
+Purshotam Shah
+Yahoo!
+
+  committer
+
+  
+  
   rkanter
   Robert Kanter
   Cloudera
@@ -217,28 +225,20 @@
 
   
   
-virag
-Virag Kothari
-Yahoo!
+shwethags
+Shwetha GS
+InMobi
 
   committer
-  PMC member
 
   
   
-puru
-Purshotam Shah
+virag
+Virag Kothari
 Yahoo!
 
   committer
-
-  
-  
-shwethags
-Shwetha GS
-InMobi
-
-  committer
+  PMC member
 
   
   




git commit: OOZIE-1906 Service to periodically remove ZK lock (puru via rohini)

2014-07-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.1 283a3c6a8 -> cb911b664


OOZIE-1906 Service to periodically remove ZK lock (puru via rohini)


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

Branch: refs/heads/branch-4.1
Commit: cb911b6642df3a0604f1974f79f0d4e46fcd5309
Parents: 283a3c6
Author: Rohini Palaniswamy 
Authored: Wed Jul 23 23:27:01 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jul 23 23:27:32 2014 -0700

--
 .../apache/oozie/service/ZKLocksService.java| 42 ++--
 core/src/main/resources/oozie-default.xml   | 18 +
 .../oozie/service/TestZKLocksService.java   | 21 ++
 pom.xml |  7 ++--
 release-log.txt |  1 +
 5 files changed, 83 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/cb911b66/core/src/main/java/org/apache/oozie/service/ZKLocksService.java
--
diff --git a/core/src/main/java/org/apache/oozie/service/ZKLocksService.java 
b/core/src/main/java/org/apache/oozie/service/ZKLocksService.java
index d03a899..3c642db 100644
--- a/core/src/main/java/org/apache/oozie/service/ZKLocksService.java
+++ b/core/src/main/java/org/apache/oozie/service/ZKLocksService.java
@@ -17,10 +17,16 @@
  */
 package org.apache.oozie.service;
 
+import java.io.IOException;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.curator.framework.recipes.locks.ChildReaper;
+import org.apache.curator.framework.recipes.locks.Reaper;
 import org.apache.curator.framework.recipes.locks.InterProcessMutex;
 import org.apache.curator.framework.recipes.locks.InterProcessReadWriteLock;
+import org.apache.curator.utils.ThreadUtils;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.util.Instrumentable;
 import org.apache.oozie.util.Instrumentation;
@@ -28,18 +34,30 @@ import org.apache.oozie.lock.LockToken;
 import org.apache.oozie.util.XLog;
 import org.apache.oozie.util.ZKUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Service that provides distributed locks via ZooKeeper.  Requires that a 
ZooKeeper ensemble is available.  The locks will be
  * located under a ZNode named "locks" under the namespace (see {@link 
ZKUtils}).  For example, with default settings, if the
  * resource we're locking is called "foo", then the ZNode backing the lock 
will be at /oozie/locks/foo.
+ * 
+ * ChildReaper is used for deleting unused locks. Only one childreaper will be 
active in cluster.
+ * ZK Path /oozie.zookeeper.namespace/services/locksChildReaperLeaderPath is 
used for leader selection.
  */
+
 public class ZKLocksService extends MemoryLocksService implements Service, 
Instrumentable {
 
 private ZKUtils zk;
 private static XLog LOG = XLog.getLog(ZKLocksService.class);
-private static final String LOCKS_NODE = "/locks/";
+public static final String LOCKS_NODE = "/locks";
 private final AtomicLong lockCount = new AtomicLong();
 
+private static final String REAPING_LEADER_PATH = 
ZKUtils.ZK_BASE_SERVICES_PATH + "/locksChildReaperLeaderPath";
+public static final int DEFAULT_REAPING_THRESHOLD = 300; // In sec
+public static final String REAPING_THRESHOLD = CONF_PREFIX + 
"ZKLocksService.locks.reaper.threshold";
+public static final String REAPING_THREADS = CONF_PREFIX + 
"ZKLocksService.locks.reaper.threads";
+private ChildReaper reaper = null;
+
 /**
  * Initialize the zookeeper locks service
  *
@@ -50,6 +68,9 @@ public class ZKLocksService extends MemoryLocksService 
implements Service, Instr
 super.init(services);
 try {
 zk = ZKUtils.register(this);
+reaper = new ChildReaper(zk.getClient(), LOCKS_NODE, 
Reaper.Mode.REAP_INDEFINITELY, getExecutorService(),
+services.getConf().getInt(REAPING_THRESHOLD, 
DEFAULT_REAPING_THRESHOLD) * 1000, REAPING_LEADER_PATH);
+reaper.start();
 }
 catch (Exception ex) {
 throw new ServiceException(ErrorCode.E1700, ex.getMessage(), ex);
@@ -62,6 +83,15 @@ public class ZKLocksService extends MemoryLocksService 
implements Service, Instr
  */
 @Override
 public void destroy() {
+if (reaper != null) {
+try {
+reaper.close();
+}
+catch (IOException e) {
+LOG.error("

git commit: OOZIE-1906 Service to periodically remove ZK lock (puru via rohini)

2014-07-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 5be0a8705 -> 12ef61470


OOZIE-1906 Service to periodically remove ZK lock (puru via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/12ef6147
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/12ef6147
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/12ef6147

Branch: refs/heads/master
Commit: 12ef61470e12aa9885220de1e453dec1da05b28c
Parents: 5be0a87
Author: Rohini Palaniswamy 
Authored: Wed Jul 23 23:27:01 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jul 23 23:27:01 2014 -0700

--
 .../apache/oozie/service/ZKLocksService.java| 42 ++--
 core/src/main/resources/oozie-default.xml   | 18 +
 .../oozie/service/TestZKLocksService.java   | 21 ++
 pom.xml |  7 ++--
 release-log.txt |  1 +
 5 files changed, 83 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/12ef6147/core/src/main/java/org/apache/oozie/service/ZKLocksService.java
--
diff --git a/core/src/main/java/org/apache/oozie/service/ZKLocksService.java 
b/core/src/main/java/org/apache/oozie/service/ZKLocksService.java
index d03a899..3c642db 100644
--- a/core/src/main/java/org/apache/oozie/service/ZKLocksService.java
+++ b/core/src/main/java/org/apache/oozie/service/ZKLocksService.java
@@ -17,10 +17,16 @@
  */
 package org.apache.oozie.service;
 
+import java.io.IOException;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.curator.framework.recipes.locks.ChildReaper;
+import org.apache.curator.framework.recipes.locks.Reaper;
 import org.apache.curator.framework.recipes.locks.InterProcessMutex;
 import org.apache.curator.framework.recipes.locks.InterProcessReadWriteLock;
+import org.apache.curator.utils.ThreadUtils;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.util.Instrumentable;
 import org.apache.oozie.util.Instrumentation;
@@ -28,18 +34,30 @@ import org.apache.oozie.lock.LockToken;
 import org.apache.oozie.util.XLog;
 import org.apache.oozie.util.ZKUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Service that provides distributed locks via ZooKeeper.  Requires that a 
ZooKeeper ensemble is available.  The locks will be
  * located under a ZNode named "locks" under the namespace (see {@link 
ZKUtils}).  For example, with default settings, if the
  * resource we're locking is called "foo", then the ZNode backing the lock 
will be at /oozie/locks/foo.
+ * 
+ * ChildReaper is used for deleting unused locks. Only one childreaper will be 
active in cluster.
+ * ZK Path /oozie.zookeeper.namespace/services/locksChildReaperLeaderPath is 
used for leader selection.
  */
+
 public class ZKLocksService extends MemoryLocksService implements Service, 
Instrumentable {
 
 private ZKUtils zk;
 private static XLog LOG = XLog.getLog(ZKLocksService.class);
-private static final String LOCKS_NODE = "/locks/";
+public static final String LOCKS_NODE = "/locks";
 private final AtomicLong lockCount = new AtomicLong();
 
+private static final String REAPING_LEADER_PATH = 
ZKUtils.ZK_BASE_SERVICES_PATH + "/locksChildReaperLeaderPath";
+public static final int DEFAULT_REAPING_THRESHOLD = 300; // In sec
+public static final String REAPING_THRESHOLD = CONF_PREFIX + 
"ZKLocksService.locks.reaper.threshold";
+public static final String REAPING_THREADS = CONF_PREFIX + 
"ZKLocksService.locks.reaper.threads";
+private ChildReaper reaper = null;
+
 /**
  * Initialize the zookeeper locks service
  *
@@ -50,6 +68,9 @@ public class ZKLocksService extends MemoryLocksService 
implements Service, Instr
 super.init(services);
 try {
 zk = ZKUtils.register(this);
+reaper = new ChildReaper(zk.getClient(), LOCKS_NODE, 
Reaper.Mode.REAP_INDEFINITELY, getExecutorService(),
+services.getConf().getInt(REAPING_THRESHOLD, 
DEFAULT_REAPING_THRESHOLD) * 1000, REAPING_LEADER_PATH);
+reaper.start();
 }
 catch (Exception ex) {
 throw new ServiceException(ErrorCode.E1700, ex.getMessage(), ex);
@@ -62,6 +83,15 @@ public class ZKLocksService extends MemoryLocksService 
implements Service, Instr
  */
 @Override
 public void destroy() {
+if (reaper != null) {
+try {
+reaper.close();
+}
+catch (IOException e) {
+LOG.error("

git commit: OOZIE-1812 Bulk API with bundle Id should relax regex check for Id (puru via rohini)

2014-07-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.1 0b763d79e -> 283a3c6a8


OOZIE-1812 Bulk API with bundle Id should relax regex check for Id (puru via 
rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/283a3c6a
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/283a3c6a
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/283a3c6a

Branch: refs/heads/branch-4.1
Commit: 283a3c6a84b5ce87b9475c538ee396987c0184f9
Parents: 0b763d7
Author: Rohini Palaniswamy 
Authored: Wed Jul 23 23:18:54 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jul 23 23:19:25 2014 -0700

--
 .../oozie/executor/jpa/BulkJPAExecutor.java |  3 +-
 .../org/apache/oozie/service/UUIDService.java   |  8 ++--
 .../org/apache/oozie/service/ZKUUIDService.java | 23 +-
 core/src/main/resources/oozie-default.xml   |  9 
 .../apache/oozie/service/TestZKUUIDService.java | 46 ++--
 .../org/apache/oozie/test/XDataTestCase.java| 21 -
 release-log.txt |  1 +
 7 files changed, 79 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/283a3c6a/core/src/main/java/org/apache/oozie/executor/jpa/BulkJPAExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/executor/jpa/BulkJPAExecutor.java 
b/core/src/main/java/org/apache/oozie/executor/jpa/BulkJPAExecutor.java
index 8327aee..9074987 100644
--- a/core/src/main/java/org/apache/oozie/executor/jpa/BulkJPAExecutor.java
+++ b/core/src/main/java/org/apache/oozie/executor/jpa/BulkJPAExecutor.java
@@ -42,6 +42,7 @@ import org.apache.oozie.CoordinatorJobBean;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.StringBlob;
 import org.apache.oozie.client.CoordinatorAction;
+import org.apache.oozie.service.Services;
 import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.util.ParamChecker;
 
@@ -140,7 +141,7 @@ public class BulkJPAExecutor implements 
JPAExecutor {
  * @return PARAM_TYPE
  */
 private PARAM_TYPE getParamType(String id, char job) {
-Pattern p = Pattern.compile("\\d{7}-\\d{15}-oozie-[a-z]{4}-" + job);
+Pattern p = Pattern.compile("\\d{7}-\\d{15}-" + 
Services.get().getSystemId() + "-" + job);
 Matcher m = p.matcher(id);
 if (m.matches()) {
 return PARAM_TYPE.ID;

http://git-wip-us.apache.org/repos/asf/oozie/blob/283a3c6a/core/src/main/java/org/apache/oozie/service/UUIDService.java
--
diff --git a/core/src/main/java/org/apache/oozie/service/UUIDService.java 
b/core/src/main/java/org/apache/oozie/service/UUIDService.java
index 7489a53..b5e593b 100644
--- a/core/src/main/java/org/apache/oozie/service/UUIDService.java
+++ b/core/src/main/java/org/apache/oozie/service/UUIDService.java
@@ -56,7 +56,7 @@ public class UUIDService implements Service {
 String genType = services.getConf().get(CONF_GENERATOR, 
"counter").trim();
 if (genType.equals("counter")) {
 counter = new AtomicLong();
-startTime = getStartTime();
+resetStartTime();
 }
 else {
 if (!genType.equals("random")) {
@@ -76,11 +76,11 @@ public class UUIDService implements Service {
 }
 
 /**
- * Get Server start time
+ * reset start time
  * @return
  */
-public String getStartTime() {
-return new SimpleDateFormat("yyMMddHHmmssSSS").format(new Date());
+protected void resetStartTime() {
+startTime = new SimpleDateFormat("yyMMddHHmmssSSS").format(new Date());
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/oozie/blob/283a3c6a/core/src/main/java/org/apache/oozie/service/ZKUUIDService.java
--
diff --git a/core/src/main/java/org/apache/oozie/service/ZKUUIDService.java 
b/core/src/main/java/org/apache/oozie/service/ZKUUIDService.java
index 33d782b..0b2be64 100644
--- a/core/src/main/java/org/apache/oozie/service/ZKUUIDService.java
+++ b/core/src/main/java/org/apache/oozie/service/ZKUUIDService.java
@@ -18,9 +18,6 @@
 
 package org.apache.oozie.service;
 
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
 import org.apache.curator.framework.recipes.atomic.AtomicValue;
 import org.apache.curator.framework.recipes.atomic.DistributedAtomicLong;
 import org.apache.oozie.ErrorCode;
@@ -28,6 +25,8 @@ import org.apache.oozie.lock.LockToken;
 import org.apache.oozie.util.XLog;
 import org.apache.oozie.util.ZKUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Service that provides

git commit: OOZIE-1812 Bulk API with bundle Id should relax regex check for Id (puru via rohini)

2014-07-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 0695c2033 -> 5be0a8705


OOZIE-1812 Bulk API with bundle Id should relax regex check for Id (puru via 
rohini)


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

Branch: refs/heads/master
Commit: 5be0a8705103e5bbabb84e72007643976fa3e9c7
Parents: 0695c20
Author: Rohini Palaniswamy 
Authored: Wed Jul 23 23:18:54 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jul 23 23:18:54 2014 -0700

--
 .../oozie/executor/jpa/BulkJPAExecutor.java |  3 +-
 .../org/apache/oozie/service/UUIDService.java   |  8 ++--
 .../org/apache/oozie/service/ZKUUIDService.java | 23 +-
 core/src/main/resources/oozie-default.xml   |  9 
 .../apache/oozie/service/TestZKUUIDService.java | 46 ++--
 .../org/apache/oozie/test/XDataTestCase.java| 21 -
 release-log.txt |  1 +
 7 files changed, 79 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/5be0a870/core/src/main/java/org/apache/oozie/executor/jpa/BulkJPAExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/executor/jpa/BulkJPAExecutor.java 
b/core/src/main/java/org/apache/oozie/executor/jpa/BulkJPAExecutor.java
index 8327aee..9074987 100644
--- a/core/src/main/java/org/apache/oozie/executor/jpa/BulkJPAExecutor.java
+++ b/core/src/main/java/org/apache/oozie/executor/jpa/BulkJPAExecutor.java
@@ -42,6 +42,7 @@ import org.apache.oozie.CoordinatorJobBean;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.StringBlob;
 import org.apache.oozie.client.CoordinatorAction;
+import org.apache.oozie.service.Services;
 import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.util.ParamChecker;
 
@@ -140,7 +141,7 @@ public class BulkJPAExecutor implements 
JPAExecutor {
  * @return PARAM_TYPE
  */
 private PARAM_TYPE getParamType(String id, char job) {
-Pattern p = Pattern.compile("\\d{7}-\\d{15}-oozie-[a-z]{4}-" + job);
+Pattern p = Pattern.compile("\\d{7}-\\d{15}-" + 
Services.get().getSystemId() + "-" + job);
 Matcher m = p.matcher(id);
 if (m.matches()) {
 return PARAM_TYPE.ID;

http://git-wip-us.apache.org/repos/asf/oozie/blob/5be0a870/core/src/main/java/org/apache/oozie/service/UUIDService.java
--
diff --git a/core/src/main/java/org/apache/oozie/service/UUIDService.java 
b/core/src/main/java/org/apache/oozie/service/UUIDService.java
index 7489a53..b5e593b 100644
--- a/core/src/main/java/org/apache/oozie/service/UUIDService.java
+++ b/core/src/main/java/org/apache/oozie/service/UUIDService.java
@@ -56,7 +56,7 @@ public class UUIDService implements Service {
 String genType = services.getConf().get(CONF_GENERATOR, 
"counter").trim();
 if (genType.equals("counter")) {
 counter = new AtomicLong();
-startTime = getStartTime();
+resetStartTime();
 }
 else {
 if (!genType.equals("random")) {
@@ -76,11 +76,11 @@ public class UUIDService implements Service {
 }
 
 /**
- * Get Server start time
+ * reset start time
  * @return
  */
-public String getStartTime() {
-return new SimpleDateFormat("yyMMddHHmmssSSS").format(new Date());
+protected void resetStartTime() {
+startTime = new SimpleDateFormat("yyMMddHHmmssSSS").format(new Date());
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/oozie/blob/5be0a870/core/src/main/java/org/apache/oozie/service/ZKUUIDService.java
--
diff --git a/core/src/main/java/org/apache/oozie/service/ZKUUIDService.java 
b/core/src/main/java/org/apache/oozie/service/ZKUUIDService.java
index 33d782b..0b2be64 100644
--- a/core/src/main/java/org/apache/oozie/service/ZKUUIDService.java
+++ b/core/src/main/java/org/apache/oozie/service/ZKUUIDService.java
@@ -18,9 +18,6 @@
 
 package org.apache.oozie.service;
 
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
 import org.apache.curator.framework.recipes.atomic.AtomicValue;
 import org.apache.curator.framework.recipes.atomic.DistributedAtomicLong;
 import org.apache.oozie.ErrorCode;
@@ -28,6 +25,8 @@ import org.apache.oozie.lock.LockToken;
 import org.apache.oozie.util.XLog;
 import org.apache.oozie.util.ZKUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Service that provides

git commit: OOZIE-1915 Move system properties to conf properties (puru via rohini)

2014-07-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.1 29c553fe9 -> 0b763d79e


OOZIE-1915 Move system properties to conf properties (puru via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/0b763d79
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/0b763d79
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/0b763d79

Branch: refs/heads/branch-4.1
Commit: 0b763d79ef75ac03cdb20bd9fc5df436a80ce40b
Parents: 29c553f
Author: Rohini Palaniswamy 
Authored: Wed Jul 23 23:07:07 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jul 23 23:08:22 2014 -0700

--
 .../oozie/service/ConfigurationService.java | 22 ++--
 .../oozie/service/JobsConcurrencyService.java   |  8 +++
 .../apache/oozie/servlet/BaseAdminServlet.java  |  5 -
 .../java/org/apache/oozie/util/ConfigUtils.java |  5 +++--
 .../java/org/apache/oozie/util/ZKUtils.java |  2 +-
 core/src/main/resources/oozie-default.xml   | 11 +-
 .../service/TestJobsConcurrencyService.java | 17 ---
 .../java/org/apache/oozie/test/ZKXTestCase.java |  2 +-
 docs/src/site/twiki/AG_Install.twiki| 16 +-
 release-log.txt |  1 +
 10 files changed, 59 insertions(+), 30 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/0b763d79/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/ConfigurationService.java 
b/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
index 798782e..8da50c3 100644
--- a/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
+++ b/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
@@ -81,26 +81,27 @@ public class ConfigurationService implements Service, 
Instrumentable {
 public static final String OOZIE_CONFIG_FILE = "oozie.config.file";
 
 private static final Set IGNORE_SYS_PROPS = new HashSet();
+private static final Set CONF_SYS_PROPS = new HashSet();
+
 private static final String IGNORE_TEST_SYS_PROPS = "oozie.test.";
 private static final Set MASK_PROPS = new HashSet();
 
 static {
-IGNORE_SYS_PROPS.add(CONF_IGNORE_SYS_PROPS);
 
 //all this properties are seeded as system properties, no need to log 
changes
-IGNORE_SYS_PROPS.add("oozie.http.hostname");
-IGNORE_SYS_PROPS.add("oozie.http.port");
-IGNORE_SYS_PROPS.add(ZKUtils.OOZIE_INSTANCE_ID);
-
+IGNORE_SYS_PROPS.add(CONF_IGNORE_SYS_PROPS);
 IGNORE_SYS_PROPS.add(Services.OOZIE_HOME_DIR);
 IGNORE_SYS_PROPS.add(OOZIE_CONFIG_DIR);
 IGNORE_SYS_PROPS.add(OOZIE_CONFIG_FILE);
 IGNORE_SYS_PROPS.add(OOZIE_DATA_DIR);
-
 IGNORE_SYS_PROPS.add(XLogService.OOZIE_LOG_DIR);
 IGNORE_SYS_PROPS.add(XLogService.LOG4J_FILE);
 IGNORE_SYS_PROPS.add(XLogService.LOG4J_RELOAD);
 
+CONF_SYS_PROPS.add("oozie.http.hostname");
+CONF_SYS_PROPS.add("oozie.http.port");
+CONF_SYS_PROPS.add(ZKUtils.OOZIE_INSTANCE_ID);
+
 // These properties should be masked when displayed because they 
contain sensitive info (e.g. password)
 MASK_PROPS.add(JPAService.CONF_PASSWORD);
 MASK_PROPS.add("oozie.authentication.signature.secret");
@@ -255,6 +256,15 @@ public class ConfigurationService implements Service, 
Instrumentable {
 }
 }
 
+//Backward compatible, we should still support -Dparam.
+for (String key : CONF_SYS_PROPS) {
+String sysValue = System.getProperty(key);
+if (sysValue != null && !IGNORE_SYS_PROPS.contains(key)) {
+log.info("Overriding configuration with system property. Key 
[{0}], Value [{1}] ", key, sysValue);
+configuration.set(key, sysValue);
+}
+}
+
 return new LogChangesConfiguration(configuration);
 }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/0b763d79/core/src/main/java/org/apache/oozie/service/JobsConcurrencyService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/JobsConcurrencyService.java 
b/core/src/main/java/org/apache/oozie/service/JobsConcurrencyService.java
index b44d9d7..7508b21 100644
--- a/core/src/main/java/org/apache/oozie/service/JobsConcurrencyService.java
+++ b/core/src/main/java/org/apache/oozie/service/JobsConcurrencyService.java
@@ -33,11 +33,7 @@ import org.apache.oozie.util.ZKUtils;
  */
 public class JobsConcurrencyService implements Service, Instr

Git Push Summary

2014-07-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/remotes/origin/branch-4.1 [created] 29c553fe9


git commit: OOZIE-1915 Move system properties to conf properties (puru via rohini)

2014-07-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 805c5978e -> 0695c2033


OOZIE-1915 Move system properties to conf properties (puru via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/0695c203
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/0695c203
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/0695c203

Branch: refs/heads/master
Commit: 0695c20337283327fe700b136d27b7a2bc0e6846
Parents: 805c597
Author: Rohini Palaniswamy 
Authored: Wed Jul 23 23:07:07 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jul 23 23:07:07 2014 -0700

--
 .../oozie/service/ConfigurationService.java | 22 ++--
 .../oozie/service/JobsConcurrencyService.java   |  8 +++
 .../apache/oozie/servlet/BaseAdminServlet.java  |  5 -
 .../java/org/apache/oozie/util/ConfigUtils.java |  5 +++--
 .../java/org/apache/oozie/util/ZKUtils.java |  2 +-
 core/src/main/resources/oozie-default.xml   | 11 +-
 .../service/TestJobsConcurrencyService.java | 17 ---
 .../java/org/apache/oozie/test/ZKXTestCase.java |  2 +-
 docs/src/site/twiki/AG_Install.twiki| 16 +-
 release-log.txt |  1 +
 10 files changed, 59 insertions(+), 30 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/0695c203/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/ConfigurationService.java 
b/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
index 798782e..8da50c3 100644
--- a/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
+++ b/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
@@ -81,26 +81,27 @@ public class ConfigurationService implements Service, 
Instrumentable {
 public static final String OOZIE_CONFIG_FILE = "oozie.config.file";
 
 private static final Set IGNORE_SYS_PROPS = new HashSet();
+private static final Set CONF_SYS_PROPS = new HashSet();
+
 private static final String IGNORE_TEST_SYS_PROPS = "oozie.test.";
 private static final Set MASK_PROPS = new HashSet();
 
 static {
-IGNORE_SYS_PROPS.add(CONF_IGNORE_SYS_PROPS);
 
 //all this properties are seeded as system properties, no need to log 
changes
-IGNORE_SYS_PROPS.add("oozie.http.hostname");
-IGNORE_SYS_PROPS.add("oozie.http.port");
-IGNORE_SYS_PROPS.add(ZKUtils.OOZIE_INSTANCE_ID);
-
+IGNORE_SYS_PROPS.add(CONF_IGNORE_SYS_PROPS);
 IGNORE_SYS_PROPS.add(Services.OOZIE_HOME_DIR);
 IGNORE_SYS_PROPS.add(OOZIE_CONFIG_DIR);
 IGNORE_SYS_PROPS.add(OOZIE_CONFIG_FILE);
 IGNORE_SYS_PROPS.add(OOZIE_DATA_DIR);
-
 IGNORE_SYS_PROPS.add(XLogService.OOZIE_LOG_DIR);
 IGNORE_SYS_PROPS.add(XLogService.LOG4J_FILE);
 IGNORE_SYS_PROPS.add(XLogService.LOG4J_RELOAD);
 
+CONF_SYS_PROPS.add("oozie.http.hostname");
+CONF_SYS_PROPS.add("oozie.http.port");
+CONF_SYS_PROPS.add(ZKUtils.OOZIE_INSTANCE_ID);
+
 // These properties should be masked when displayed because they 
contain sensitive info (e.g. password)
 MASK_PROPS.add(JPAService.CONF_PASSWORD);
 MASK_PROPS.add("oozie.authentication.signature.secret");
@@ -255,6 +256,15 @@ public class ConfigurationService implements Service, 
Instrumentable {
 }
 }
 
+//Backward compatible, we should still support -Dparam.
+for (String key : CONF_SYS_PROPS) {
+String sysValue = System.getProperty(key);
+if (sysValue != null && !IGNORE_SYS_PROPS.contains(key)) {
+log.info("Overriding configuration with system property. Key 
[{0}], Value [{1}] ", key, sysValue);
+configuration.set(key, sysValue);
+}
+}
+
 return new LogChangesConfiguration(configuration);
 }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/0695c203/core/src/main/java/org/apache/oozie/service/JobsConcurrencyService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/JobsConcurrencyService.java 
b/core/src/main/java/org/apache/oozie/service/JobsConcurrencyService.java
index b44d9d7..7508b21 100644
--- a/core/src/main/java/org/apache/oozie/service/JobsConcurrencyService.java
+++ b/core/src/main/java/org/apache/oozie/service/JobsConcurrencyService.java
@@ -33,11 +33,7 @@ import org.apache.oozie.util.ZKUtils;
  */
 public class JobsConcurrencyService implements Service, Instrumentable {
 
-pr

git commit: OOZIE-1886 Queue operation talking longer time (shwethags via rohini)

2014-07-07 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 10e8ecc20 -> 0439fbb05


OOZIE-1886 Queue operation talking longer time (shwethags via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/0439fbb0
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/0439fbb0
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/0439fbb0

Branch: refs/heads/master
Commit: 0439fbb057d1ecd32a5cea250481c63a6cfc3a96
Parents: 10e8ecc
Author: Rohini Palaniswamy 
Authored: Mon Jul 7 14:59:34 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Jul 7 14:59:34 2014 -0700

--
 .../apache/oozie/util/PriorityDelayQueue.java   | 20 +++-
 release-log.txt |  1 +
 2 files changed, 8 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/0439fbb0/core/src/main/java/org/apache/oozie/util/PriorityDelayQueue.java
--
diff --git a/core/src/main/java/org/apache/oozie/util/PriorityDelayQueue.java 
b/core/src/main/java/org/apache/oozie/util/PriorityDelayQueue.java
index a3f2148..1aad92e 100644
--- a/core/src/main/java/org/apache/oozie/util/PriorityDelayQueue.java
+++ b/core/src/main/java/org/apache/oozie/util/PriorityDelayQueue.java
@@ -349,20 +349,14 @@ public class PriorityDelayQueue extends 
AbstractQueue= 
maxSize) {
 return false;
 }
-boolean accepted;
-lock.lock();
-try {
-accepted = queues[queueElement.getPriority()].offer(queueElement);
-debug("offer([{0}]), to P[{1}] delay[{2}ms] accepted[{3}]", 
queueElement.getElement().toString(),
-  queueElement.getPriority(), 
queueElement.getDelay(TimeUnit.MILLISECONDS), accepted);
-if (accepted) {
-if (currentSize != null) {
-currentSize.incrementAndGet();
-}
-queueElement.inQueue = true;
+boolean accepted = 
queues[queueElement.getPriority()].offer(queueElement);
+debug("offer([{0}]), to P[{1}] delay[{2}ms] accepted[{3}]", 
queueElement.getElement().toString(),
+  queueElement.getPriority(), 
queueElement.getDelay(TimeUnit.MILLISECONDS), accepted);
+if (accepted) {
+if (currentSize != null) {
+currentSize.incrementAndGet();
 }
-} finally {
-lock.unlock();
+queueElement.inQueue = true;
 }
 return accepted;
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/0439fbb0/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index eee06a4..5ac93f2 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1886 Queue operation talking longer time (shwethags via rohini)
 OOZIE-1865 Oozie servers can't talk to each other with Oozie HA and Kerberos 
(rkanter)
 OOZIE-1821 Oozie java action fails due to AlreadyBeingCreatedException 
(abhishek.agarwal via rkanter)
 OOZIE-1532 Purging should remove completed children job for long running 
coordinator jobs (bzhang)



svn commit: r914790 - /websites/production/oozie/content/

2014-07-02 Thread rohini
Author: rohini
Date: Wed Jul  2 17:57:38 2014
New Revision: 914790

Log:
Update role for Rohini to PMC and order team list alphabetically

Added:
websites/production/oozie/content/
  - copied from r914789, websites/staging/oozie/trunk/content/



git commit: OOZIE-1527 Fix scalability issues with coordinator materialization (puru via rohini)

2014-06-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 06a2241c0 -> b1d79f050


OOZIE-1527 Fix scalability issues with coordinator materialization (puru via 
rohini)


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

Branch: refs/heads/master
Commit: b1d79f050c247225cfc0d3d671497d236af8d08a
Parents: 06a2241
Author: Rohini Palaniswamy 
Authored: Mon Jun 23 16:19:58 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Jun 23 16:19:58 2014 -0700

--
 .../org/apache/oozie/service/CoordMaterializeTriggerService.java  | 3 +--
 core/src/main/resources/oozie-default.xml | 3 +++
 2 files changed, 4 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/b1d79f05/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
 
b/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
index d2b5a6c..1dac7e8 100644
--- 
a/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
+++ 
b/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
@@ -61,7 +61,6 @@ public class CoordMaterializeTriggerService implements 
Service {
 private static final String INSTRUMENTATION_GROUP = "coord_job_mat";
 private static final String INSTR_MAT_JOBS_COUNTER = "jobs";
 public static final int CONF_LOOKUP_INTERVAL_DEFAULT = 300;
-private static final int CONF_SCHEDULING_INTERVAL_DEFAULT = 300;
 private static final int CONF_MATERIALIZATION_WINDOW_DEFAULT = 3600;
 private static final int CONF_MATERIALIZATION_SYSTEM_LIMIT_DEFAULT = 50;
 
@@ -210,7 +209,7 @@ public class CoordMaterializeTriggerService implements 
Service {
 // default is 300sec (5min)
 int lookupInterval = 
Services.get().getConf().getInt(CONF_LOOKUP_INTERVAL, 
CONF_LOOKUP_INTERVAL_DEFAULT);
 // default is 300sec (5min)
-int schedulingInterval = 
Services.get().getConf().getInt(CONF_SCHEDULING_INTERVAL, 
CONF_SCHEDULING_INTERVAL_DEFAULT);
+int schedulingInterval = 
Services.get().getConf().getInt(CONF_SCHEDULING_INTERVAL, lookupInterval);
 
 Runnable lookupTriggerJobsRunnable = new 
CoordMaterializeTriggerRunnable(materializationWindow, lookupInterval);
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/b1d79f05/core/src/main/resources/oozie-default.xml
--
diff --git a/core/src/main/resources/oozie-default.xml 
b/core/src/main/resources/oozie-default.xml
index 4dedeef..61ac388 100644
--- a/core/src/main/resources/oozie-default.xml
+++ b/core/src/main/resources/oozie-default.xml
@@ -417,12 +417,15 @@
 
 
 
+
 
 
 
oozie.service.CoordMaterializeTriggerService.materialization.window



git commit: OOZIE-1877 Setting to fail oozie server startup in case of sharelib misconfiguration (puru via rohini)

2014-06-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 1add349e0 -> 06a2241c0


OOZIE-1877 Setting to fail oozie server startup in case of sharelib 
misconfiguration (puru via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/06a2241c
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/06a2241c
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/06a2241c

Branch: refs/heads/master
Commit: 06a2241c0d33c0604a005d57c6bd32f816424891
Parents: 1add349
Author: Rohini Palaniswamy 
Authored: Mon Jun 23 15:13:21 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Jun 23 15:13:21 2014 -0700

--
 .../apache/oozie/service/ShareLibService.java   | 25 ++--
 core/src/main/resources/oozie-default.xml   |  8 +++
 .../oozie/service/TestShareLibService.java  | 19 +++
 release-log.txt |  1 +
 4 files changed, 46 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/06a2241c/core/src/main/java/org/apache/oozie/service/ShareLibService.java
--
diff --git a/core/src/main/java/org/apache/oozie/service/ShareLibService.java 
b/core/src/main/java/org/apache/oozie/service/ShareLibService.java
index 3ef5e07..c31d587 100644
--- a/core/src/main/java/org/apache/oozie/service/ShareLibService.java
+++ b/core/src/main/java/org/apache/oozie/service/ShareLibService.java
@@ -39,7 +39,6 @@ import java.util.Set;
 import java.util.TimeZone;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -53,6 +52,7 @@ import org.apache.oozie.util.Instrumentation;
 import org.apache.oozie.util.XLog;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.oozie.ErrorCode;
 
 public class ShareLibService implements Service, Instrumentable {
@@ -65,6 +65,8 @@ public class ShareLibService implements Service, 
Instrumentable {
 
 public static final String PURGE_INTERVAL = CONF_PREFIX + 
"ShareLibService.purge.interval";
 
+public static final String FAIL_FAST_ON_STARTUP = CONF_PREFIX + 
"ShareLibService.fail.fast.on.startup";
+
 private static final String PERMISSION_STRING = "-rwxr-xr-x";
 
 public static final String LAUNCHER_PREFIX = "launcher_";
@@ -102,6 +104,7 @@ public class ShareLibService implements Service, 
Instrumentable {
 this.services = services;
 sharelibMappingFile = services.getConf().get(SHARELIB_MAPPING_FILE, 
"");
 isShipLauncherEnabled = 
services.getConf().getBoolean(SHIP_LAUNCHER_JAR, false);
+boolean failOnfailure = 
services.getConf().getBoolean(FAIL_FAST_ON_STARTUP, false);
 Path launcherlibPath = getLauncherlibPath();
 HadoopAccessorService has = 
Services.get().get(HadoopAccessorService.class);
 URI uri = launcherlibPath.toUri();
@@ -109,12 +112,20 @@ public class ShareLibService implements Service, 
Instrumentable {
 fs = FileSystem.get(has.createJobConf(uri.getAuthority()));
 updateLauncherLib();
 updateShareLib();
-} catch(IOException ioe) {
-// We don't want to actually fail init by throwing an Exception, 
so only create the ServiceException and log it
-ServiceException se = new ServiceException(ErrorCode.E0104, 
getClass().getName(),
-"Not able to cache sharelib.  An Admin needs to install 
the sharelib with oozie-setup.sh and issue the "
-+ "'oozie admin' CLI command to update the 
sharelib", ioe);
-LOG.error(se);
+} catch(Throwable e) {
+if (failOnfailure) {
+LOG.error("Sharelib initialization fails", e);
+throw new ServiceException(ErrorCode.E0104, 
getClass().getName(), "Sharelib initialization fails. "
++ e.getMessage());
+}
+else {
+// We don't want to actually fail init by throwing an 
Exception, so only create the ServiceException and
+// log it
+ServiceException se = new ServiceException(ErrorCode.E0104, 
getClass().getName(),
+"Not able to cache sharelib. An Admin needs to install 
the sharelib with oozie-setup.sh and issue the "
++ "'oozie admin' CLI command to update the 
sharelib", e);
+LOG.error(se);
+}
 }
 Runnable purgeLibsRunnable 

git commit: OOZIE-1388 Add a admin servlet to show thread stack trace and CPU usage per thread (rohini)

2014-06-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master b84a3e45c -> 1add349e0


OOZIE-1388 Add a admin servlet to show thread stack trace and CPU usage per 
thread (rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/1add349e
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/1add349e
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/1add349e

Branch: refs/heads/master
Commit: 1add349e0fd74653bf0b8821a88c7171fd656c16
Parents: b84a3e4
Author: Rohini Palaniswamy 
Authored: Mon Jun 23 14:54:51 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Jun 23 14:54:51 2014 -0700

--
 .../java/org/apache/oozie/servlet/JVMInfo.java  | 275 +++
 docs/src/site/twiki/AG_Monitoring.twiki |   7 +
 release-log.txt |   1 +
 webapp/src/main/webapp/WEB-INF/web-common.xml   |   5 +
 webapp/src/main/webapp/admin/jvminfo.jsp|  91 ++
 5 files changed, 379 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/1add349e/core/src/main/java/org/apache/oozie/servlet/JVMInfo.java
--
diff --git a/core/src/main/java/org/apache/oozie/servlet/JVMInfo.java 
b/core/src/main/java/org/apache/oozie/servlet/JVMInfo.java
new file mode 100644
index 000..9a8248e
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/servlet/JVMInfo.java
@@ -0,0 +1,275 @@
+/**
+ * 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.servlet;
+
+import java.io.Serializable;
+import java.lang.Thread.State;
+import java.lang.management.ClassLoadingMXBean;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+import java.lang.management.MemoryUsage;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+@SuppressWarnings("serial")
+public class JVMInfo implements Serializable {
+
+private static Map threadStateOrdering;
+private final MemoryMXBean memoryMXBean;
+private final ClassLoadingMXBean classLoadingMXBean;
+private final ThreadMXBean threadMXBean;
+private List jThreadInfos;
+private String threadSortOrder;
+private Integer cpuMonitorTime = 0;
+private int blockedThreads = 0;
+private int runnableThreads = 0;
+private int waitingThreads = 0;
+private int timedWaitingThreads = 0;
+private int newThreads = 0;
+private int terminatedThreads = 0;
+
+static {
+threadStateOrdering = new HashMap();
+threadStateOrdering.put(State.RUNNABLE, 1);
+threadStateOrdering.put(State.BLOCKED, 2);
+threadStateOrdering.put(State.WAITING, 3);
+threadStateOrdering.put(State.TIMED_WAITING, 4);
+threadStateOrdering.put(State.NEW, 5);
+threadStateOrdering.put(State.TERMINATED, 6);
+}
+
+public JVMInfo() {
+memoryMXBean = ManagementFactory.getMemoryMXBean();
+classLoadingMXBean = ManagementFactory.getClassLoadingMXBean();
+threadMXBean = ManagementFactory.getThreadMXBean();
+}
+
+public String getThreadSortOrder() {
+return threadSortOrder;
+}
+
+public void setThreadSortOrder(String threadSortOrder) {
+this.threadSortOrder = threadSortOrder;
+}
+
+public String getCpuMonitorTime() {
+return cpuMonitorTime.toString();
+}
+
+public void setCpuMonitorTime(String sleepTime) {
+if (sleepTime != null) {
+this.cpuMonitorTime = Integer.parseInt(sleepTime);
+}
+}
+
+public String getHeapMemoryUsage() {
+MemoryUsage hmu = memoryMXBean.getHeapMemoryUsage();
+StringBuffer sb = new StringBuffer(60);
+sb.append("INIT=").append(hmu.getInit());
+sb.append("  U

git commit: OOZIE-1893 Recovery service will never recover bundle action if CoordSubmitXCommand command is lost (puru via rohini)

2014-06-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master db11ab356 -> b84a3e45c


OOZIE-1893 Recovery service will never recover bundle action if 
CoordSubmitXCommand command is lost (puru via rohini)


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

Branch: refs/heads/master
Commit: b84a3e45ce15e7deab260d2ca8afaa72e0bceddc
Parents: db11ab3
Author: Rohini Palaniswamy 
Authored: Mon Jun 23 14:16:38 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Jun 23 14:16:38 2014 -0700

--
 core/src/main/java/org/apache/oozie/service/RecoveryService.java | 2 +-
 release-log.txt  | 1 +
 2 files changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/b84a3e45/core/src/main/java/org/apache/oozie/service/RecoveryService.java
--
diff --git a/core/src/main/java/org/apache/oozie/service/RecoveryService.java 
b/core/src/main/java/org/apache/oozie/service/RecoveryService.java
index 2749bc4..b22a98d 100644
--- a/core/src/main/java/org/apache/oozie/service/RecoveryService.java
+++ b/core/src/main/java/org/apache/oozie/service/RecoveryService.java
@@ -183,7 +183,7 @@ public class RecoveryService implements Service {
 log.error("CoordId is null for Bundle action " + 
baction.getBundleActionId());
 continue;
 }
-if 
(Services.get().get(JobsConcurrencyService.class).isJobIdForThisServer(baction.getCoordId()))
 {
+if 
(Services.get().get(JobsConcurrencyService.class).isJobIdForThisServer(baction.getBundleId()))
 {
 if (baction.getStatus() == Job.Status.PREP && 
baction.getCoordId() == null) {
 BundleJobBean bundleJob = null;
 if (jpaService != null) {

http://git-wip-us.apache.org/repos/asf/oozie/blob/b84a3e45/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index fef3453..5f4602b 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1893 Recovery service will never recover bundle action if 
CoordSubmitXCommand command is lost (puru via rohini)
 OOZIE-1878 Can't execute dryrun on the CLI (puru via rohini)
 OOZIE-1741 Add new coord EL function to get input partitions value string 
(satish.mittal via rohini) 
 OOZIE-1817 Oozie timers are not biased (rkanter)



git commit: OOZIE-1878 Can't execute dryrun on the CLI (puru via rohini)

2014-06-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 6f0d303c9 -> db11ab356


OOZIE-1878 Can't execute dryrun on the CLI (puru via rohini)


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

Branch: refs/heads/master
Commit: db11ab356af7c29478e41441bd2325dfdb5130c0
Parents: 6f0d303
Author: Rohini Palaniswamy 
Authored: Mon Jun 23 13:52:06 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Jun 23 13:52:06 2014 -0700

--
 .../java/org/apache/oozie/cli/CLIParser.java| 50 -
 .../org/apache/oozie/client/TestOozieCLI.java   | 79 +++-
 .../servlet/MockCoordinatorEngineService.java   | 16 
 release-log.txt |  1 +
 4 files changed, 144 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/db11ab35/client/src/main/java/org/apache/oozie/cli/CLIParser.java
--
diff --git a/client/src/main/java/org/apache/oozie/cli/CLIParser.java 
b/client/src/main/java/org/apache/oozie/cli/CLIParser.java
index 6d490f2..c8b1681 100644
--- a/client/src/main/java/org/apache/oozie/cli/CLIParser.java
+++ b/client/src/main/java/org/apache/oozie/cli/CLIParser.java
@@ -17,12 +17,15 @@
  */
 package org.apache.oozie.cli;
 
+import org.apache.commons.cli.MissingOptionException;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.UnrecognizedOptionException;
 
+import java.util.Arrays;
 import java.util.Map;
 import java.util.LinkedHashMap;
 import java.text.MessageFormat;
@@ -114,9 +117,18 @@ public class CLIParser {
 }
 else {
 if (commands.containsKey(args[0])) {
-GnuParser parser = new GnuParser();
+GnuParser parser ;
 String[] minusCommand = new String[args.length - 1];
 System.arraycopy(args, 1, minusCommand, 0, 
minusCommand.length);
+
+if (args[0].equals(OozieCLI.JOB_CMD)) {
+validdateArgs(args, minusCommand);
+parser = new OozieGnuParser(true);
+}
+else {
+parser = new OozieGnuParser(false);
+}
+
 return new Command(args[0], 
parser.parse(commands.get(args[0]), minusCommand,
  
commandWithArgs.get(args[0])));
 }
@@ -126,6 +138,23 @@ public class CLIParser {
 }
 }
 
+public void validdateArgs(final String[] args, String[] minusCommand) 
throws ParseException {
+try {
+GnuParser parser = new OozieGnuParser(false);
+parser.parse(commands.get(args[0]), minusCommand, 
commandWithArgs.get(args[0]));
+}
+catch (MissingOptionException e) {
+if (Arrays.toString(args).contains("-dryrun")) {
+// ignore this, else throw exception
+//Dryrun is also part of update sub-command. CLI parses dryrun 
as sub-command and throws
+//Missing Option Exception, if -dryrun is used as command. 
It's ok to skip exception only for dryrun.
+}
+else {
+throw e;
+}
+}
+}
+
 public String shortHelp() {
 return "use 'help [sub-command]' for help details";
 }
@@ -164,5 +193,24 @@ public class CLIParser {
 pw.flush();
 }
 
+static class OozieGnuParser extends GnuParser {
+private boolean ignoreMissingOption;
+
+public OozieGnuParser(final boolean ignoreMissingOption) {
+this.ignoreMissingOption = ignoreMissingOption;
+}
+
+@Override
+protected void checkRequiredOptions() throws MissingOptionException {
+if (ignoreMissingOption) {
+return;
+}
+else {
+super.checkRequiredOptions();
+}
+}
+}
+
 }
 
+

http://git-wip-us.apache.org/repos/asf/oozie/blob/db11ab35/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
--
diff --git a/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java 
b/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
index 8a85cd2..a8f8cf9 100644
--- a/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
+++ b/core/src/test/java/org/apache/oozie/client

git commit: OOZIE-1741 Add new coord EL function to get input partitions value string (satish.mittal via rohini)

2014-06-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master e16448289 -> 6f0d303c9


OOZIE-1741 Add new coord EL function to get input partitions value string 
(satish.mittal via rohini)


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

Branch: refs/heads/master
Commit: 6f0d303c9086214a472cbace5362f21aec0b943b
Parents: e164482
Author: Rohini Palaniswamy 
Authored: Mon Jun 23 13:35:50 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Jun 23 13:35:50 2014 -0700

--
 .../org/apache/oozie/coord/HCatELFunctions.java |  47 +
 core/src/main/resources/oozie-default.xml   |   2 +
 .../apache/oozie/coord/TestHCatELFunctions.java |  57 ++
 .../site/twiki/CoordinatorFunctionalSpec.twiki  | 186 +++
 release-log.txt |   1 +
 .../java/org/apache/oozie/util/HCatURI.java |  29 +++
 6 files changed, 322 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/6f0d303c/core/src/main/java/org/apache/oozie/coord/HCatELFunctions.java
--
diff --git a/core/src/main/java/org/apache/oozie/coord/HCatELFunctions.java 
b/core/src/main/java/org/apache/oozie/coord/HCatELFunctions.java
index e5f0146..9a36af0 100644
--- a/core/src/main/java/org/apache/oozie/coord/HCatELFunctions.java
+++ b/core/src/main/java/org/apache/oozie/coord/HCatELFunctions.java
@@ -115,6 +115,12 @@ public class HCatELFunctions {
 return echoUnResolved("dataOutPartitions", "'" + dataOutName + "'");
 }
 
+public static String ph1_coord_dataInPartitions_echo(String dataInName, 
String type) {
+// Checking if the dataIn/dataOut is correct?
+isValidDataEvent(dataInName);
+return echoUnResolved("dataInPartitions", "'" + dataInName + "', '" + 
type + "'");
+}
+
 public static String ph1_coord_dataOutPartitionValue_echo(String 
dataOutName, String partition) {
 // Checking if the dataIn/dataOut is correct?
 isValidDataEvent(dataOutName);
@@ -266,6 +272,47 @@ public class HCatELFunctions {
 }
 
 /**
+ * Used to specify the entire HCat partition defining input for workflow 
job.  Look for two evaluator-level
+ * variables  A) .datain. B) 
.datain..unresolved  A defines the data-in HCat URI.
+ *  B defines whether there are any unresolved EL-function (i.e 
latest)  If there are something unresolved,
+ * this function will echo back the original function  otherwise it 
sends the partition.
+ *
+ * @param dataInName : DataIn name
+ * @param type : for action type: hive-export
+ */
+public static String ph3_coord_dataInPartitions(String dataInName, String 
type) {
+ELEvaluator eval = ELEvaluator.getCurrent();
+String uri = (String) eval.getVariable(".datain." + dataInName);
+Boolean unresolved = (Boolean) eval.getVariable(".datain." + 
dataInName + ".unresolved");
+if (unresolved != null && unresolved.booleanValue() == true) {
+return "${coord:dataInPartitions('" + dataInName + "', '" + type + 
"')}";
+}
+String partitionValue = null;
+if (uri != null) {
+if (type.equals("hive-export")) {
+String[] uriList = uri.split(CoordELFunctions.DIR_SEPARATOR);
+if (uriList.length > 1) {
+throw new RuntimeException("Multiple partitions not 
supported for hive-export type. Dataset name: "
++ dataInName + " URI: " + uri);
+}
+try {
+partitionValue = new 
HCatURI(uri).toPartitionValueString(type);
+}
+catch (URISyntaxException e) {
+throw new RuntimeException("Parsing exception for HCatURI 
" + uri, e);
+}
+} else {
+  throw new RuntimeException("Unsupported type: " + type + " 
dataset name: " + dataInName);
+}
+}
+else {
+XLog.getLog(HCatELFunctions.class).warn("URI is null");
+return null;
+}
+return partitionValue;
+}
+
+/**
  * Used to specify the MAXIMUM value of an HCat partition which is input 
dependency for workflow job. Look for two evaluator-level
  * variables  A) .datain. B) 
.

svn commit: r1603981 - /oozie/site/trunk/pom.xml

2014-06-19 Thread rohini
Author: rohini
Date: Thu Jun 19 18:02:54 2014
New Revision: 1603981

URL: http://svn.apache.org/r1603981
Log:
Update role for Rohini to PMC and order names alphabetically. update maven-site 
to 3.3 to fix build failure

Modified:
oozie/site/trunk/pom.xml

Modified: oozie/site/trunk/pom.xml
URL: 
http://svn.apache.org/viewvc/oozie/site/trunk/pom.xml?rev=1603981&r1=1603980&r2=1603981&view=diff
==
--- oozie/site/trunk/pom.xml (original)
+++ oozie/site/trunk/pom.xml Thu Jun 19 18:02:54 2014
@@ -46,7 +46,7 @@
 
   org.apache.maven.plugins
   maven-site-plugin
-  3.0
+  3.3
   
 ${basedir}/content
   
@@ -121,6 +121,14 @@
 
   
   
+bzhang
+Bowen Zhang
+Hortonworks
+
+  committer
+
+  
+  
 brookwc
 Chao Wang
 Microsoft
@@ -174,15 +182,6 @@
 
   
   
-virag
-Virag Kothari
-Yahoo!
-
-  committer
-  PMC member
-
-  
-  
 mona
 Mona Chitnis
 Yahoo!
@@ -206,6 +205,7 @@
 Yahoo!
 
   committer
+  PMC member
 
   
   
@@ -217,11 +217,12 @@
 
   
   
-bzhang
-Bowen Zhang
-Hortonworks
+virag
+Virag Kothari
+Yahoo!
 
   committer
+  PMC member
 
   
   




git commit: OOZIE-1807 Make bundle change command synchronous (puru via rohini)

2014-06-18 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 4e015d45e -> d361ee4d4


OOZIE-1807 Make bundle change command synchronous (puru via rohini)


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

Branch: refs/heads/master
Commit: d361ee4d4f065eae4473300b1a961a464f2365b1
Parents: 4e015d4
Author: Rohini Palaniswamy 
Authored: Wed Jun 18 09:55:36 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jun 18 09:55:36 2014 -0700

--
 .../main/java/org/apache/oozie/ErrorCode.java   |   1 +
 .../command/bundle/BundleJobChangeXCommand.java |  25 ++-
 .../bundle/BundleStatusUpdateXCommand.java  |  12 +-
 .../command/coord/CoordChangeXCommand.java  |   9 +-
 .../bundle/TestBundleChangeXCommand.java| 165 ++-
 release-log.txt |   1 +
 6 files changed, 198 insertions(+), 15 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/d361ee4d/core/src/main/java/org/apache/oozie/ErrorCode.java
--
diff --git a/core/src/main/java/org/apache/oozie/ErrorCode.java 
b/core/src/main/java/org/apache/oozie/ErrorCode.java
index d3c1b03..88a2c67 100644
--- a/core/src/main/java/org/apache/oozie/ErrorCode.java
+++ b/core/src/main/java/org/apache/oozie/ErrorCode.java
@@ -232,6 +232,7 @@ public enum ErrorCode {
 E1317(XLog.STD, "Invalid bundle job change value {0}, {1}"),
 E1318(XLog.STD, "No coord jobs for the bundle=[{0}], fail the bundle"),
 E1319(XLog.STD, "Invalid bundle coord job namespace, [{0}]"),
+E1320(XLog.STD, "Bundle Job change error, [{0}]"),
 
 E1400(XLog.STD, "doAs (proxyuser) failure"),
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/d361ee4d/core/src/main/java/org/apache/oozie/command/bundle/BundleJobChangeXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/bundle/BundleJobChangeXCommand.java
 
b/core/src/main/java/org/apache/oozie/command/bundle/BundleJobChangeXCommand.java
index 1962748..41ad8ae 100644
--- 
a/core/src/main/java/org/apache/oozie/command/bundle/BundleJobChangeXCommand.java
+++ 
b/core/src/main/java/org/apache/oozie/command/bundle/BundleJobChangeXCommand.java
@@ -162,6 +162,7 @@ public class BundleJobChangeXCommand extends XCommand 
{
  */
 @Override
 protected Void execute() throws CommandException {
+StringBuffer changeReport = new StringBuffer();
 try {
 if (isChangePauseTime || isChangeEndTime) {
 if (isChangePauseTime) {
@@ -179,18 +180,28 @@ public class BundleJobChangeXCommand extends 
XCommand {
 for (BundleActionBean action : this.bundleActions) {
 // queue coord change commands;
 if (action.getStatus() != Job.Status.KILLED && 
action.getCoordId() != null) {
-queue(new CoordChangeXCommand(action.getCoordId(), 
changeValue));
-LOG.info("Queuing CoordChangeXCommand coord job = " + 
action.getCoordId() + " to change "
-+ changeValue);
-action.setPending(action.getPending() + 1);
-updateList.add(new UpdateEntry(
-
BundleActionQuery.UPDATE_BUNDLE_ACTION_PENDING_MODTIME, action));
+try {
+new CoordChangeXCommand(action.getCoordId(), 
changeValue).call();
+}
+catch (Exception e) {
+String errorMsg = action.getCoordId() + " : " + 
e.getMessage();
+LOG.info("Change command failed " + errorMsg);
+changeReport.append("[ 
").append(errorMsg).append(" ]");
+}
+}
+else {
+String errorMsg = action.getCoordId() + " : Coord is 
in killed state";
+LOG.info("Change command failed " + errorMsg);
+changeReport.append("[ ").append(errorMsg).append(" 
]");
 }
 }
 updateList.add(new 
UpdateEntry(BundleJobQuery.UPDATE_BUNDLE_JOB_STATUS_PAUSE_ENDTIME,
 bundleJob));
 
BatchQueryExecutor.getInstance().executeBatchInsertUpdateDelete(null, 
updateList, null);
 }
+if(!ch

git commit: OOZIE-1685 Oozie doesn’t process correctly workflows with a non-default name node (benjzh via rohini)

2014-06-17 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 6d4a9d0ea -> d5f1e3864


OOZIE-1685 Oozie doesn’t process correctly workflows with a non-default name 
node (benjzh via rohini)


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

Branch: refs/heads/master
Commit: d5f1e3864e96c23133304a12adc5a14aebba854b
Parents: 6d4a9d0
Author: Rohini Palaniswamy 
Authored: Tue Jun 17 13:58:39 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Jun 17 13:58:39 2014 -0700

--
 .../oozie/action/hadoop/JavaActionExecutor.java |  62 +++---
 .../oozie/service/HadoopAccessorService.java|   9 +-
 .../apache/oozie/service/ShareLibService.java   |  11 ++
 .../java/org/apache/oozie/util/JobUtils.java|   9 +-
 .../action/hadoop/ActionExecutorTestCase.java   |   4 +
 .../action/hadoop/TestJavaActionExecutor.java   | 190 ---
 .../java/org/apache/oozie/test/XFsTestCase.java |  54 --
 .../java/org/apache/oozie/test/XTestCase.java   | 121 +---
 .../src/site/twiki/WorkflowFunctionalSpec.twiki |   6 +
 release-log.txt |   1 +
 10 files changed, 358 insertions(+), 109 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/d5f1e386/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index 40add2c..7a0d0e3 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -349,11 +349,27 @@ public class JavaActionExecutor extends ActionExecutor {
 throws IOException, ActionExecutorException, 
HadoopAccessorException, URISyntaxException {
 Namespace ns = element.getNamespace();
 Iterator it = element.getChildren("job-xml", ns).iterator();
+HashMap filesystemsMap = new HashMap();
+HadoopAccessorService has = 
Services.get().get(HadoopAccessorService.class);
 while (it.hasNext()) {
 Element e = it.next();
 String jobXml = e.getTextTrim();
-Path path = new Path(appPath, jobXml);
-FileSystem fs = context.getAppFileSystem();
+Path pathSpecified = new Path(jobXml);
+Path path = pathSpecified.isAbsolute() ? pathSpecified : new 
Path(appPath, jobXml);
+FileSystem fs;
+if (filesystemsMap.containsKey(path.toUri().getAuthority())) {
+  fs = filesystemsMap.get(path.toUri().getAuthority());
+}
+else {
+  if (path.toUri().getAuthority() != null) {
+fs = has.createFileSystem(context.getWorkflow().getUser(), 
path.toUri(),
+has.createJobConf(path.toUri().getAuthority()));
+  }
+  else {
+fs = context.getAppFileSystem();
+  }
+  filesystemsMap.put(path.toUri().getAuthority(), fs);
+}
 Configuration jobXmlConf = new XConfiguration(fs.open(path));
 try {
 String jobXmlConfString = 
XmlUtils.prettyPrint(jobXmlConf).toString();
@@ -432,22 +448,11 @@ public class JavaActionExecutor extends ActionExecutor {
 else if (fileName.endsWith(".jar")) { // .jar files
 if (!fileName.contains("#")) {
 String user = conf.get("user.name");
-Path pathToAdd;
-// if filePath and appPath belong to same cluster, add 
URI path component else add absolute URI
-if (uri.getScheme() != null && uri.getHost() != null &&
-uri.getPort() > 0 && baseUri.getScheme() != null &&
-baseUri.getHost() != null && baseUri.getPort() > 0 
&&
-
uri.getScheme().equalsIgnoreCase(baseUri.getScheme()) &&
-uri.getHost().equalsIgnoreCase(baseUri.getHost()) 
&&
-uri.getPort() == baseUri.getPort()) {
-  pathToAdd = new Path(uri.getPath());
-} else {
-  pathToAdd = new Path(uri.normalize());
-}
+Path pathToAdd = new Path(uri.normalize());
  

git commit: OOZIE-1861 Pig action should work with tez mode (rohini)

2014-06-11 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 11c665998 -> b7f6cc319


OOZIE-1861 Pig action should work with tez mode (rohini)


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

Branch: refs/heads/master
Commit: b7f6cc319b842347e775fd53316e8c3a09445037
Parents: 11c6659
Author: Rohini Palaniswamy 
Authored: Wed Jun 11 11:37:41 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jun 11 11:37:41 2014 -0700

--
 release-log.txt   | 1 +
 .../pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java | 3 +--
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/b7f6cc31/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 5090b5d..08ecebe 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1861 Pig action should work with tez mode (rohini)
 OOZIE-1703 User should be able to set coord end-time before start time (puru 
via rohini)
 OOZIE-1715 Distributed ID sequence for HA (puru via rkanter)
 OOZIE-1870 Workflow action doen't resolve retry-max and retry-interval (puru 
via rohini)

http://git-wip-us.apache.org/repos/asf/oozie/blob/b7f6cc31/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
--
diff --git 
a/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java 
b/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
index 48820f7..382e8b9 100644
--- a/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
+++ b/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
@@ -67,8 +67,6 @@ public class PigMain extends LauncherMain {
 DISALLOWED_PIG_OPTIONS.add("-logfile");
 DISALLOWED_PIG_OPTIONS.add("-r");
 DISALLOWED_PIG_OPTIONS.add("-dryrun");
-DISALLOWED_PIG_OPTIONS.add("-x");
-DISALLOWED_PIG_OPTIONS.add("-exectype");
 DISALLOWED_PIG_OPTIONS.add("-P");
 DISALLOWED_PIG_OPTIONS.add("-propertyFile");
 }
@@ -106,6 +104,7 @@ public class PigMain extends LauncherMain {
 String jobTokenFile = getFilePathFromEnv("HADOOP_TOKEN_FILE_LOCATION");
 if (jobTokenFile != null) {
 pigProperties.setProperty("mapreduce.job.credentials.binary", 
jobTokenFile);
+pigProperties.setProperty("tez.credentials.path", jobTokenFile);
 System.out.println("");
 System.out.println("Setting env property for 
mapreduce.job.credentials.binary to:"
 + jobTokenFile);



git commit: OOZIE-1703 User should be able to set coord end-time before start time (puru via rohini)

2014-06-11 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master ecedf6dc6 -> 11c665998


OOZIE-1703 User should be able to set coord end-time before start time (puru 
via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/11c66599
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/11c66599
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/11c66599

Branch: refs/heads/master
Commit: 11c665998f0ae665cddd24cf0fb3bde4d2080d94
Parents: ecedf6d
Author: Rohini Palaniswamy 
Authored: Wed Jun 11 06:27:29 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Jun 11 06:27:29 2014 -0700

--
 .../command/coord/CoordChangeXCommand.java  | 58 +++-
 .../command/coord/TestCoordChangeXCommand.java  | 57 +--
 docs/src/site/twiki/DG_CommandLineTool.twiki| 17 ++
 release-log.txt |  1 +
 4 files changed, 80 insertions(+), 53 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/11c66599/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java 
b/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
index ea7df17..436b999 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
@@ -186,10 +186,7 @@ public class CoordChangeXCommand extends 
CoordinatorXCommand {
  */
 private void checkPauseTime(CoordinatorJobBean coordJob, Date newPauseTime)
 throws CommandException {
-// New pauseTime has to be a non-past time.
-if (newPauseTime.before(coordJob.getStartTime())) {
-throw new CommandException(ErrorCode.E1015, newPauseTime, "must be 
a non-past time");
-}
+//no check.
 }
 
 /**
@@ -349,33 +346,44 @@ public class CoordChangeXCommand extends 
CoordinatorXCommand {
 if (!dontChange) {
 coordJob.setEndTime(newEndTime);
 // OOZIE-1703, we should SUCCEEDED the coord, if it's in 
PREP and new endtime is before start time
-if (coordJob.getStatus() == CoordinatorJob.Status.PREP && 
coordJob.getStartTime().after(newEndTime)) {
-LOG.info("Changing coord status to SUCCEEDED, because 
it's in PREP and new end time is before start time. "
-+ "Startime is " + coordJob.getStartTime() + " 
and new end time is " + newEndTime);
-coordJob.setStatus(CoordinatorJob.Status.SUCCEEDED);
+if (coordJob.getStartTime().compareTo(newEndTime) >= 0) {
+if (coordJob.getStatus() != 
CoordinatorJob.Status.PREP) {
+processLookaheadActions(coordJob, newEndTime);
+}
+if (coordJob.getStatus() == CoordinatorJob.Status.PREP
+|| coordJob.getStatus() == 
CoordinatorJob.Status.RUNNING) {
+LOG.info("Changing coord status to SUCCEEDED, 
because it's in " + coordJob.getStatus()
++ " and new end time is before start time. 
Startime is " + coordJob.getStartTime()
++ " and new end time is " + newEndTime);
+
coordJob.setStatus(CoordinatorJob.Status.SUCCEEDED);
+coordJob.resetPending();
+}
 coordJob.setDoneMaterialization();
-coordJob.resetPending();
 }
 else {
-//move it to running iff neendtime is after starttime.
-if (newEndTime.after(coordJob.getStartTime())) {
-if (coordJob.getStatus() == 
CoordinatorJob.Status.SUCCEEDED) {
-
coordJob.setStatus(CoordinatorJob.Status.RUNNING);
-}
-if (coordJob.getStatus() == 
CoordinatorJob.Status.DONEWITHERROR
-|| coordJob.getStatus() == 
CoordinatorJob.Status.FAILED) {
-// Check for backward compatibility for Oozie 
versions (3.2 and before)
-// when RUNNINGWITHERROR, SUSPENDEDWITHERROR 
and
-// PAUSEDWITHERROR is not supported
-coordJob.setStatus(StatusUtils
-

git commit: Correct typo in checkin from OOZIE-1807 to OOZIE-1870

2014-06-09 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 9f16ce1d2 -> 659c45c4d


Correct typo in checkin from OOZIE-1807 to OOZIE-1870


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/659c45c4
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/659c45c4
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/659c45c4

Branch: refs/heads/master
Commit: 659c45c4d3ee07512d30c5e9187d65e59bd0d615
Parents: 9f16ce1
Author: Rohini Palaniswamy 
Authored: Mon Jun 9 16:25:18 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Jun 9 16:25:18 2014 -0700

--
 release-log.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/659c45c4/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index deead7b..8071b46 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,6 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
-OOZIE-1807 Workflow action doen't resolve retry-max and retry-interval (puru 
via rohini)
+OOZIE-1870 Workflow action doen't resolve retry-max and retry-interval (puru 
via rohini)
 OOZIE-1686 Typo in DG_CommandLineTool (anbu78 via ryota)
 OOZIE-1804 Improve documentation for Coordinator Specification (lars_francke 
via rkanter)
 OOZIE-1828 Introduce counters JobStatus terminal states metrics (rkanter)



git commit: OOZIE-1807 Workflow action doen't resolve retry-max and retry-interval (puru via rohini)

2014-06-09 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 052187f79 -> 9f16ce1d2


OOZIE-1807 Workflow action doen't resolve retry-max and retry-interval (puru 
via rohini)


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

Branch: refs/heads/master
Commit: 9f16ce1d2fe45e3a783e895c3414d1926c388633
Parents: 052187f
Author: Rohini Palaniswamy 
Authored: Mon Jun 9 15:42:50 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Jun 9 15:42:50 2014 -0700

--
 .../apache/oozie/command/wf/SignalXCommand.java |  2 ++
 .../workflow/lite/LiteWorkflowAppParser.java| 21 +--
 .../lite/TestLiteWorkflowAppParser.java | 22 +++-
 release-log.txt |  1 +
 4 files changed, 43 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/9f16ce1d/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
--
diff --git a/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java 
b/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
index 7968973..4a7283f 100644
--- a/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
@@ -58,6 +58,7 @@ import org.apache.oozie.util.InstrumentUtils;
 import org.apache.oozie.util.LogUtils;
 import org.apache.oozie.util.XConfiguration;
 import org.apache.oozie.util.ParamChecker;
+import org.apache.oozie.util.XLog;
 import org.apache.oozie.util.XmlUtils;
 import org.apache.oozie.util.db.SLADbXOperations;
 import org.jdom.Element;
@@ -190,6 +191,7 @@ public class SignalXCommand extends WorkflowXCommand {
 completed = 
workflowInstance.signal(wfAction.getExecutionPath(), wfAction.getSignalValue());
 }
 catch (WorkflowException e) {
+   LOG.error("Workflow action failed : " + e.getMessage(), e);
 
wfJob.setStatus(WorkflowJob.Status.valueOf(workflowInstance.getStatus().toString()));
 completed = true;
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/9f16ce1d/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java 
b/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
index 685503a..e47e619 100644
--- 
a/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
+++ 
b/core/src/main/java/org/apache/oozie/workflow/lite/LiteWorkflowAppParser.java
@@ -18,6 +18,7 @@
 package org.apache.oozie.workflow.lite;
 
 import org.apache.oozie.workflow.WorkflowException;
+import org.apache.oozie.util.ELUtils;
 import org.apache.oozie.util.IOUtils;
 import org.apache.oozie.util.XConfiguration;
 import org.apache.oozie.util.XmlUtils;
@@ -28,6 +29,7 @@ import org.apache.oozie.ErrorCode;
 import org.apache.oozie.action.ActionExecutor;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.ActionService;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.jdom.Element;
 import org.jdom.JDOMException;
@@ -152,7 +154,7 @@ public class LiteWorkflowAppParser {
 
 Element wfDefElement = XmlUtils.parseXml(strDef);
 ParameterVerifier.verifyParameters(jobConf, wfDefElement);
-LiteWorkflowApp app = parse(strDef, wfDefElement, configDefault);
+LiteWorkflowApp app = parse(strDef, wfDefElement, configDefault, 
jobConf);
 Map traversed = new HashMap();
 traversed.put(app.getNode(StartNodeDef.START).getName(), 
VisitStatus.VISITING);
 validate(app, app.getNode(StartNodeDef.START), traversed);
@@ -376,11 +378,14 @@ public class LiteWorkflowAppParser {
  *
  * @param strDef
  * @param root
+ * @param configDefault
+ * @param jobConf
  * @return LiteWorkflowApp
  * @throws WorkflowException
  */
 @SuppressWarnings({"unchecked"})
-private LiteWorkflowApp parse(String strDef, Element root, Configuration 
configDefault) throws WorkflowException {
+private LiteWorkflowApp parse(String strDef, Element root, Configuration 
configDefault, Configuration jobConf)
+throws WorkflowException {
 Namespace ns = root.getNamespace();
 LiteWorkflowApp def = null;
 Element global = null;
@@ -451,6 +456,18 @@ public

git commit: OOZIE-1812 Bundle status is always in RUNNING if one of the action status is in PREP (puru via rohini)

2014-05-31 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 74cb3b0bc -> 3637874e9


OOZIE-1812 Bundle status is always in RUNNING if one of the action status is in 
PREP (puru via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/3637874e
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/3637874e
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/3637874e

Branch: refs/heads/master
Commit: 3637874e911b5b15103015285b6ac7b7703ea8cc
Parents: 74cb3b0
Author: Rohini Palaniswamy 
Authored: Sat May 31 20:00:27 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Sat May 31 20:00:27 2014 -0700

--
 .../oozie/service/StatusTransitService.java | 24 +---
 .../oozie/service/TestStatusTransitService.java | 29 
 release-log.txt |  1 +
 3 files changed, 44 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/3637874e/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/StatusTransitService.java 
b/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
index e140e64..21ac25f 100644
--- a/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
+++ b/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
@@ -426,7 +426,7 @@ public class StatusTransitService implements Service {
 if (bundleActionStatus.containsKey(Job.Status.PREP)) {
 // If all the bundle actions are PREP then bundle job should 
be RUNNING.
 if (bundleActions.size() > 
bundleActionStatus.get(Job.Status.PREP)) {
-bundleStatus[0] = Job.Status.RUNNING;
+bundleStatus[0] = getRunningStatus(bundleActionStatus);
 ret = true;
 }
 }
@@ -618,21 +618,25 @@ public class StatusTransitService implements Service {
 List bundleActions, Job.Status[] 
bundleStatus) {
 boolean ret = false;
 if (bundleStatus[0] != Job.Status.PREP) {
-if (bundleActionStatus.containsKey(Job.Status.FAILED)
-|| bundleActionStatus.containsKey(Job.Status.KILLED)
-|| 
bundleActionStatus.containsKey(Job.Status.DONEWITHERROR)
-|| 
bundleActionStatus.containsKey(Job.Status.RUNNINGWITHERROR)) {
-bundleStatus[0] = Job.Status.RUNNINGWITHERROR;
-}
-else {
-bundleStatus[0] = Job.Status.RUNNING;
-}
+bundleStatus[0] = getRunningStatus(bundleActionStatus);
 ret = true;
 }
 return ret;
 
 }
 
+private Job.Status getRunningStatus(HashMap 
bundleActionStatus) {
+if (bundleActionStatus.containsKey(Job.Status.FAILED)
+|| bundleActionStatus.containsKey(Job.Status.KILLED)
+|| bundleActionStatus.containsKey(Job.Status.DONEWITHERROR)
+|| 
bundleActionStatus.containsKey(Job.Status.RUNNINGWITHERROR)) {
+return Job.Status.RUNNINGWITHERROR;
+}
+else {
+return Job.Status.RUNNING;
+}
+}
+
 private void updateBundleJob(boolean isPending, BundleJobBean 
bundleJob, Job.Status bundleStatus)
 throws JPAExecutorException {
 String jobId = bundleJob.getId();

http://git-wip-us.apache.org/repos/asf/oozie/blob/3637874e/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java 
b/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java
index b767b69..bb99138 100644
--- a/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java
+++ b/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java
@@ -1475,4 +1475,33 @@ public class TestStatusTransitService extends 
XDataTestCase {
 bundleJob = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
 assertEquals(CoordinatorJob.Status.RUNNING, bundleJob.getStatus());
 }
+
+// Test bundle transition from running to runningwitherror when one action 
is killed.
+public void testBundleStatusTransitRunningWithError() throws Exception {
+Services.get().destroy();
+
setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR,
 "false");
+new Services().init();
+
+BundleJ

git commit: OOZIE-1844 HA - Lock mechanism for CoordMaterializeTriggerService (puru via rohini)

2014-05-28 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 373a52ff2 -> 853a4af9f


OOZIE-1844 HA - Lock mechanism for CoordMaterializeTriggerService (puru via 
rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/853a4af9
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/853a4af9
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/853a4af9

Branch: refs/heads/master
Commit: 853a4af9f06348e2bd95793e1d1b6c1e6a9949e1
Parents: 373a52f
Author: Rohini Palaniswamy 
Authored: Wed May 28 10:38:55 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed May 28 10:38:55 2014 -0700

--
 .../service/CoordMaterializeTriggerService.java | 81 +---
 release-log.txt |  1 +
 2 files changed, 55 insertions(+), 27 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/853a4af9/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
 
b/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
index ef3c3f4..d2b5a6c 100644
--- 
a/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
+++ 
b/core/src/main/java/org/apache/oozie/service/CoordMaterializeTriggerService.java
@@ -27,6 +27,7 @@ import 
org.apache.oozie.command.coord.CoordMaterializeTransitionXCommand;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
+import org.apache.oozie.lock.LockToken;
 import org.apache.oozie.util.XCallable;
 import org.apache.oozie.util.XLog;
 import org.apache.oozie.util.DateUtils;
@@ -73,6 +74,8 @@ public class CoordMaterializeTriggerService implements 
Service {
 private long delay = 0;
 private List> callables;
 private List> delayedCallables;
+private XLog LOG = XLog.getLog(getClass());
+
 
 public CoordMaterializeTriggerRunnable(int materializationWindow, int 
lookupInterval) {
 this.materializationWindow = materializationWindow;
@@ -81,29 +84,54 @@ public class CoordMaterializeTriggerService implements 
Service {
 
 @Override
 public void run() {
-runCoordJobMatLookup();
+LockToken lock = null;
 
-if (null != callables) {
-boolean ret = 
Services.get().get(CallableQueueService.class).queueSerial(callables);
-if (ret == false) {
-XLog.getLog(getClass()).warn(
-"Unable to queue the callables commands for 
CoordMaterializeTriggerRunnable. "
-+ "Most possibly command queue is full. 
Queue size is :"
-+ 
Services.get().get(CallableQueueService.class).queueSize());
+// first check if there is some other running instance from the 
same service;
+try {
+lock = Services.get().get(MemoryLocksService.class)
+
.getWriteLock(CoordMaterializeTriggerService.class.getName(), lockTimeout);
+
+if (lock != null) {
+runCoordJobMatLookup();
+if (null != callables) {
+boolean ret = 
Services.get().get(CallableQueueService.class).queueSerial(callables);
+if (ret == false) {
+XLog.getLog(getClass()).warn(
+"Unable to queue the callables commands 
for CoordMaterializeTriggerRunnable. "
++ "Most possibly command queue is 
full. Queue size is :"
++ 
Services.get().get(CallableQueueService.class).queueSize());
+}
+callables = null;
+}
+if (null != delayedCallables) {
+boolean ret = 
Services.get().get(CallableQueueService.class)
+.queueSerial(delayedCallables, this.delay);
+if (ret == false) {
+XLog.getLog(getClass()).warn(
+"Unable to queue the delayedCallables 
commands for CoordMaterializeTriggerRunnable. "
++ "Most possibly Callable queue is 
full. Queue size is :"
++ 
Services.get().get(CallableQueueService.class).queueSize());
+ 

git commit: OOZIE-1819 Avoid early queueing of CoordActionInputCheckXCommand (shwethags via rohini)

2014-05-27 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master b9c3e0b0c -> 252300c62


OOZIE-1819 Avoid early queueing of CoordActionInputCheckXCommand (shwethags via 
rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/252300c6
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/252300c6
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/252300c6

Branch: refs/heads/master
Commit: 252300c62c0a0f5d4efb483fbc0fdb135dd27469
Parents: b9c3e0b
Author: Rohini Palaniswamy 
Authored: Tue May 27 10:45:00 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Tue May 27 10:45:00 2014 -0700

--
 .../CoordMaterializeTransitionXCommand.java | 31 
 release-log.txt |  1 +
 2 files changed, 19 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/252300c6/core/src/main/java/org/apache/oozie/command/coord/CoordMaterializeTransitionXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/coord/CoordMaterializeTransitionXCommand.java
 
b/core/src/main/java/org/apache/oozie/command/coord/CoordMaterializeTransitionXCommand.java
index 57cbb34..7b46ad6 100644
--- 
a/core/src/main/java/org/apache/oozie/command/coord/CoordMaterializeTransitionXCommand.java
+++ 
b/core/src/main/java/org/apache/oozie/command/coord/CoordMaterializeTransitionXCommand.java
@@ -17,13 +17,6 @@
  */
 package org.apache.oozie.command.coord;
 
-import java.io.IOException;
-import java.io.StringReader;
-import java.sql.Timestamp;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.TimeZone;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.AppType;
 import org.apache.oozie.CoordinatorActionBean;
@@ -39,8 +32,8 @@ import org.apache.oozie.command.MaterializeTransitionXCommand;
 import org.apache.oozie.command.PreconditionException;
 import org.apache.oozie.command.bundle.BundleStatusUpdateXCommand;
 import org.apache.oozie.coord.TimeUnit;
-import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.BatchQueryExecutor;
+import org.apache.oozie.executor.jpa.BatchQueryExecutor.UpdateEntry;
 import org.apache.oozie.executor.jpa.CoordActionsActiveCountJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
@@ -50,11 +43,11 @@ import org.apache.oozie.service.EventHandlerService;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Service;
 import org.apache.oozie.service.Services;
+import org.apache.oozie.sla.SLAOperations;
 import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.util.Instrumentation;
 import org.apache.oozie.util.LogUtils;
 import org.apache.oozie.util.ParamChecker;
-import org.apache.oozie.sla.SLAOperations;
 import org.apache.oozie.util.StatusUtils;
 import org.apache.oozie.util.XConfiguration;
 import org.apache.oozie.util.XmlUtils;
@@ -62,6 +55,13 @@ import org.apache.oozie.util.db.SLADbOperations;
 import org.jdom.Element;
 import org.jdom.JDOMException;
 
+import java.io.IOException;
+import java.io.StringReader;
+import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.TimeZone;
+
 /**
  * Materialize actions for specified start and end time for coordinator job.
  */
@@ -130,6 +130,15 @@ public class CoordMaterializeTransitionXCommand extends 
MaterializeTransitionXCo
 if (EventHandlerService.isEnabled()) {
 CoordinatorXCommand.generateEvent(coordAction, 
coordJob.getUser(), coordJob.getAppName(), null);
 }
+
+// TODO: time 100s should be configurable
+queue(new CoordActionNotificationXCommand(coordAction), 
100);
+
+//Delay for input check = (nominal time - now)
+long checkDelay = coordAction.getNominalTime().getTime() - 
new Date().getTime();
+queue(new 
CoordActionInputCheckXCommand(coordAction.getId(), coordAction.getJobId()),
+Math.max(checkDelay, 0));
+
 if (coordAction.getPushMissingDependencies() != null) {
 // TODO: Delay in catchup mode?
 queue(new 
CoordPushDependencyCheckXCommand(coordAction.getId(), true), 100);
@@ -482,10 +491,6 @@ public class CoordMaterializeTransitionXCommand extends 
MaterializeTransitionXCo
 
 insertList.add(actionBean);
 writeActionSlaRegistration(actionXml, actionBean);
-
-// TODO: time 100s should be configurable
-queue(new CoordActionNotificationXCommand(actionBean), 100);
-qu

git commit: OOZIE-1737 Oozie log streaming is slow (puru via rohini) - amend patch

2014-05-16 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 7f5398830 -> 2dfaa4a74


OOZIE-1737 Oozie log streaming is slow (puru via rohini) - amend patch


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

Branch: refs/heads/master
Commit: 2dfaa4a74f9336cf27769483783760950bca0b94
Parents: 7f53988
Author: Rohini Palaniswamy 
Authored: Fri May 16 09:08:00 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Fri May 16 09:08:00 2014 -0700

--
 .../src/main/java/org/apache/oozie/util/XLogFilter.java | 10 ++
 .../java/org/apache/oozie/util/XLogUserFilterParam.java | 12 +---
 .../java/org/apache/oozie/util/TestLogStreamer.java |  8 
 3 files changed, 19 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/2dfaa4a7/core/src/main/java/org/apache/oozie/util/XLogFilter.java
--
diff --git a/core/src/main/java/org/apache/oozie/util/XLogFilter.java 
b/core/src/main/java/org/apache/oozie/util/XLogFilter.java
index 0007c76..a1a7fbc 100644
--- a/core/src/main/java/org/apache/oozie/util/XLogFilter.java
+++ b/core/src/main/java/org/apache/oozie/util/XLogFilter.java
@@ -52,6 +52,7 @@ public class XLogFilter {
 private Date startDate;
 private boolean isActionList = false;
 private String formattedEndDate;
+private String formattedStartDate;
 
 // TODO Patterns to be read from config file
 private static final String DEFAULT_REGEX = "[^\\]]*";
@@ -121,6 +122,11 @@ public class XLogFilter {
  * @return
  */
 public boolean matches(ArrayList logParts) {
+if (getStartDate() != null) {
+if (logParts.get(0).substring(0, 
19).compareTo(getFormattedStartDate()) < 0) {
+return false;
+}
+}
 String logLevel = logParts.get(1);
 String logMessage = logParts.get(2);
 if (this.logLevels == null || 
this.logLevels.containsKey(logLevel.toUpperCase())) {
@@ -206,6 +212,9 @@ public class XLogFilter {
 return formattedEndDate;
 }
 
+public String getFormattedStartDate() {
+return formattedStartDate;
+}
 
 public Date getStartDate() {
 return startDate;
@@ -274,6 +283,7 @@ public class XLogFilter {
 }
 
 formattedEndDate = XLogUserFilterParam.dt.get().format(getEndDate());
+formattedStartDate = 
XLogUserFilterParam.dt.get().format(getStartDate());
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/oozie/blob/2dfaa4a7/core/src/main/java/org/apache/oozie/util/XLogUserFilterParam.java
--
diff --git a/core/src/main/java/org/apache/oozie/util/XLogUserFilterParam.java 
b/core/src/main/java/org/apache/oozie/util/XLogUserFilterParam.java
index 6e49e11..e3ed55d 100644
--- a/core/src/main/java/org/apache/oozie/util/XLogUserFilterParam.java
+++ b/core/src/main/java/org/apache/oozie/util/XLogUserFilterParam.java
@@ -56,6 +56,8 @@ public class XLogUserFilterParam {
 private boolean isDebug = false;
 private String searchText;
 
+private String params;
+
 public static final ThreadLocal dt = new 
ThreadLocal() {
 @Override
 protected SimpleDateFormat initialValue() {
@@ -104,6 +106,8 @@ public class XLogUserFilterParam {
  * @throws Exception
  */
 private void parseFilterParam(String param) throws Exception {
+this.params = param;
+
 if (StringUtils.isEmpty(param)) {
 return;
 }
@@ -264,13 +268,7 @@ public class XLogUserFilterParam {
 
 @Override
 public String toString() {
-StringBuffer sb = new StringBuffer();
-sb.append(START_TIME).append("=").append(getStartDate()).append(";");
-sb.append(END_TIME).append("=").append(getEndDate()).append(";");
-sb.append(LOG_LEVEL).append("=").append(getLogLevel()).append(";");
-sb.append(LIMIT).append("=").append(getLimit()).append(";");
-
sb.append(RECENT_LOG_OFFSET).append("=").append(getRecent()).append(";");
-return sb.toString();
+return params;
 }
 
 private int getOffsetInMinute(String offset) throws IOException {

http://git-wip-us.apache.org/repos/asf/oozie/blob/2dfaa4a7/core/src/test/java/org/apache/oozie/util/TestLogStreamer.java
--
diff --git a/core/src/test/java/org/apache/oozie/util/TestLogStreamer.java 
b/core/src/test/java/org/apache/oozie/util/Te

git commit: OOZIE-1825 Optimize wf_jobs protoconf storage (puru via rohini)

2014-05-13 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 83dec8b0e -> a0aa6fbdf


OOZIE-1825 Optimize wf_jobs protoconf storage (puru via rohini)


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

Branch: refs/heads/master
Commit: a0aa6fbdfab47e803d36f8540ca28063907cb472
Parents: 83dec8b
Author: Rohini Palaniswamy 
Authored: Tue May 13 13:58:48 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Tue May 13 13:58:48 2014 -0700

--
 .../oozie/service/WorkflowAppService.java   | 14 +++---
 .../oozie/command/wf/TestSubmitXCommand.java| 52 +++-
 release-log.txt |  1 +
 3 files changed, 59 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/a0aa6fbd/core/src/main/java/org/apache/oozie/service/WorkflowAppService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/WorkflowAppService.java 
b/core/src/main/java/org/apache/oozie/service/WorkflowAppService.java
index c2f3836..05efefb 100644
--- a/core/src/main/java/org/apache/oozie/service/WorkflowAppService.java
+++ b/core/src/main/java/org/apache/oozie/service/WorkflowAppService.java
@@ -151,7 +151,6 @@ public abstract class WorkflowAppService implements Service 
{
 throw new WorkflowException(ErrorCode.E0710, ex.getMessage(), ex);
 }
 }
-
 /**
  * Create proto configuration.  The proto configuration includes the 
user,group and the paths which need to be
  * added to distributed cache. These paths include .jar,.so and the 
resource file paths.
@@ -168,9 +167,12 @@ public abstract class WorkflowAppService implements 
Service {
 URI uri = new URI(jobConf.get(OozieClient.APP_PATH));
 
 Configuration conf = has.createJobConf(uri.getAuthority());
+XConfiguration protoConf = new XConfiguration();
+
 
 String user = jobConf.get(OozieClient.USER_NAME);
 conf.set(OozieClient.USER_NAME, user);
+protoConf.set(OozieClient.USER_NAME, user);
 
 FileSystem fs = has.createFileSystem(user, uri, conf);
 
@@ -227,7 +229,7 @@ public abstract class WorkflowAppService implements Service 
{
 }
 }
 
-conf.setStrings(APP_LIB_PATH_LIST, filePaths.toArray(new 
String[filePaths.size()]));
+protoConf.setStrings(APP_LIB_PATH_LIST, filePaths.toArray(new 
String[filePaths.size()]));
 
 //Add all properties start with 'oozie.'
 for (Map.Entry entry : jobConf) {
@@ -235,14 +237,12 @@ public abstract class WorkflowAppService implements 
Service {
 String name = entry.getKey();
 String value = entry.getValue();
 // if property already exists, should not overwrite
-if(conf.get(name) == null) {
-conf.set(name, value);
+if(protoConf.get(name) == null) {
+protoConf.set(name, value);
 }
 }
 }
-XConfiguration retConf = new XConfiguration();
-XConfiguration.copy(conf, retConf);
-return retConf;
+return protoConf;
 }
 catch (IOException ex) {
 throw new WorkflowException(ErrorCode.E0712, 
jobConf.get(OozieClient.APP_PATH), ex.getMessage(), ex);

http://git-wip-us.apache.org/repos/asf/oozie/blob/a0aa6fbd/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java 
b/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java
index 5aff988..e8f77df 100644
--- a/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java
+++ b/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java
@@ -21,15 +21,19 @@ import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.io.StringReader;
 import java.net.URI;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.WorkflowJobBean;
 import org.apache.oozie.local.LocalOozie;
 import org.apache.oozie.client.OozieClient;
+import org.apache.oozie.client.WorkflowJob;
 import org.apache.oozie.command.CommandException;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobQueryExecutor.WorkflowJobQuery;
 import org.apache.oozie.ser

git commit: OOZIE-1831 Oozie upgrade fails if workflow jobs are in running or suspended state (satish.mittal via rohini)

2014-05-13 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master d77934462 -> 83dec8b0e


OOZIE-1831 Oozie upgrade fails if workflow jobs are in running or suspended 
state (satish.mittal via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/83dec8b0
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/83dec8b0
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/83dec8b0

Branch: refs/heads/master
Commit: 83dec8b0ebe37c84e2aabce35b60ed865b51b028
Parents: d779344
Author: Rohini Palaniswamy 
Authored: Tue May 13 10:03:01 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Tue May 13 10:03:01 2014 -0700

--
 release-log.txt |  5 +--
 .../java/org/apache/oozie/tools/OozieDBCLI.java | 38 ++--
 2 files changed, 39 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/83dec8b0/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 1ef64b7..c35ca7d 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,8 +1,9 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1831 Oozie upgrade fails if workflow jobs are in running or suspended 
state (satish.mittal via rohini)
 OOZIE-1690 TestShellActionExecutor#testEnvVar failed for Windows 
(omaliuvanchuk via rkanter)
-OOZIE-1243 libtools dir should not include hadoop JARs (sathish.mittal via 
rohini)
-OOZIE-1806 Java Action type jobs are failing with hadoop-0.20.0 and earlier 
versions on oozie trunk (sathish.mittal via rohini)
+OOZIE-1243 libtools dir should not include hadoop JARs (satish.mittal via 
rohini)
+OOZIE-1806 Java Action type jobs are failing with hadoop-0.20.0 and earlier 
versions on oozie trunk (satish.mittal via rohini)
 OOZIE-1823 OozieSharelibCLI shouldn't load ext services (rkanter)
 OOZIE-1762 Sharelib with oozie.action.ship.launcher.jar=true should copy 
oozie-hadoop-utils.jar (puru via mona)
 OOZIE-1709 CoordELFunctions.getCurrentInstance() is expensive (shwethags via 
rohini) 

http://git-wip-us.apache.org/repos/asf/oozie/blob/83dec8b0/tools/src/main/java/org/apache/oozie/tools/OozieDBCLI.java
--
diff --git a/tools/src/main/java/org/apache/oozie/tools/OozieDBCLI.java 
b/tools/src/main/java/org/apache/oozie/tools/OozieDBCLI.java
index a116984..0733e6f 100644
--- a/tools/src/main/java/org/apache/oozie/tools/OozieDBCLI.java
+++ b/tools/src/main/java/org/apache/oozie/tools/OozieDBCLI.java
@@ -194,7 +194,6 @@ public class OozieDBCLI {
 private void upgradeDB(String sqlFile, boolean run) throws Exception {
 validateConnection();
 validateDBSchema(true);
-verifyDBState();
 String version = 
BuildInfo.getBuildInfo().getProperty(BuildInfo.BUILD_VERSION);
 
 if (!verifyOozieSysTable(false, false)) { // If OOZIE_SYS table doesn't
@@ -231,7 +230,7 @@ public class OozieDBCLI {
 
 private void upgradeDBTo40(String sqlFile, boolean run) throws Exception {
 upgradeOozieDBVersion(sqlFile, run, DB_VERSION_FOR_4_0);
-postUpgradeTasks(sqlFile, run, false);
+postUpgradeTasksFor40(sqlFile, run);
 ddlTweaks(sqlFile, run);
 }
 
@@ -405,6 +404,41 @@ public class OozieDBCLI {
 }
 }
 
+private void postUpgradeTasksFor40(String sqlFile, boolean run) throws 
Exception {
+PrintWriter writer = new PrintWriter(new FileWriter(sqlFile, true));
+writer.println();
+Connection conn = (run) ? createConnection() : null;
+try {
+if (!getDBVendor().equals("derby")) {
+String  updateMissingDependenciesQuery;
+if (getDBVendor().equals("sqlserver")){
+updateMissingDependenciesQuery = 
UPDATE_DELIMITER_VER_TWO_MSSQL;
+} else {
+updateMissingDependenciesQuery = UPDATE_DELIMITER_VER_TWO;
+}
+
+writer.println(updateMissingDependenciesQuery + ";");
+System.out.println("Post-upgrade MISSING_DEPENDENCIES column");
+if (run) {
+Statement st = conn.createStatement();
+st.executeUpdate(updateMissingDependenciesQuery);
+st.close();
+}
+}
+else {
+System.out.println("Post-upgrade MISSING_DEPENDENCIES column 
in Derby");
+replaceForDerby(";", "!!");
+}
+System.out.println("DONE");
+writer.close();
+}
+finally {
+if (run) {
+conn.close();
+}
+  }
+}
+
 

git commit: OOZIE-1243 libtools dir should not include hadoop JARs (sathish.mittal via rohini)

2014-05-10 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 6e2e46a98 -> 3e73f4e9a


OOZIE-1243 libtools dir should not include hadoop JARs (sathish.mittal via 
rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/3e73f4e9
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/3e73f4e9
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/3e73f4e9

Branch: refs/heads/master
Commit: 3e73f4e9aff0c3964067930355f7601c3c45cd89
Parents: 6e2e46a
Author: Rohini Palaniswamy 
Authored: Wed May 7 08:54:50 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed May 7 08:54:50 2014 -0700

--
 release-log.txt   | 1 +
 src/main/assemblies/tools.xml | 3 ++-
 tools/pom.xml | 8 +++-
 3 files changed, 10 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/3e73f4e9/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 795311a..b6cd061 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1243 libtools dir should not include hadoop JARs (sathish.mittal via 
rohini)
 OOZIE-1806 Java Action type jobs are failing with hadoop-0.20.0 and earlier 
versions on oozie trunk (sathish.mittal via rohini)
 OOZIE-1823 OozieSharelibCLI shouldn't load ext services (rkanter)
 OOZIE-1762 Sharelib with oozie.action.ship.launcher.jar=true should copy 
oozie-hadoop-utils.jar (puru via mona)

http://git-wip-us.apache.org/repos/asf/oozie/blob/3e73f4e9/src/main/assemblies/tools.xml
--
diff --git a/src/main/assemblies/tools.xml b/src/main/assemblies/tools.xml
index 9b65773..a2b4482 100644
--- a/src/main/assemblies/tools.xml
+++ b/src/main/assemblies/tools.xml
@@ -39,7 +39,8 @@
 true
 
oozie-tools-${project.version}/libtools
 false
-compile
+
+runtime
 
  *:*:pom:*
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/3e73f4e9/tools/pom.xml
--
diff --git a/tools/pom.xml b/tools/pom.xml
index 1245e47..487783d 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -71,6 +71,12 @@
 
 org.apache.oozie
 oozie-client
+
+
+org.apache.hadoop
+hadoop-auth
+
+
 
 
 commons-cli
@@ -138,7 +144,7 @@
 
 org.apache.oozie
 oozie-hadoop
-compile
+provided
 
 
 



git commit: OOZIE-1806 Java Action type jobs are failing with hadoop-0.20.0 and earlier versions on oozie trunk (sathish.mittal via rohini)

2014-05-06 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 9c91c814c -> 6e2e46a98


OOZIE-1806 Java Action type jobs are failing with hadoop-0.20.0 and earlier 
versions on oozie trunk (sathish.mittal via rohini)


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

Branch: refs/heads/master
Commit: 6e2e46a98fa59377134ec0f5303ead1b6f1c7532
Parents: 9c91c81
Author: Rohini Palaniswamy 
Authored: Tue May 6 10:16:45 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Tue May 6 10:16:45 2014 -0700

--
 .../oozie/action/hadoop/JavaActionExecutor.java |  7 +++--
 .../java/org/apache/oozie/util/JobUtils.java| 30 
 release-log.txt |  1 +
 3 files changed, 35 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/6e2e46a9/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index d4b4f5e..40add2c 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -66,6 +66,7 @@ import org.apache.oozie.service.URIHandlerService;
 import org.apache.oozie.service.WorkflowAppService;
 import org.apache.oozie.servlet.CallbackServlet;
 import org.apache.oozie.util.ELEvaluator;
+import org.apache.oozie.util.JobUtils;
 import org.apache.oozie.util.LogUtils;
 import org.apache.oozie.util.PropertiesUtils;
 import org.apache.oozie.util.XConfiguration;
@@ -524,7 +525,7 @@ public class JavaActionExecutor extends ActionExecutor {
 if (listOfPaths != null && !listOfPaths.isEmpty()) {
 
 for (Path actionLibPath : listOfPaths) {
-
DistributedCache.addFileToClassPath(actionLibPath, conf, fs);
+JobUtils.addFileToClassPath(actionLibPath, 
conf, fs);
 DistributedCache.createSymlink(conf);
 }
 }
@@ -554,13 +555,13 @@ public class JavaActionExecutor extends ActionExecutor {
 }
 FileSystem fs = listOfPaths.get(0).getFileSystem(conf);
 for (Path actionLibPath : listOfPaths) {
-DistributedCache.addFileToClassPath(actionLibPath, conf, 
fs);
+JobUtils.addFileToClassPath(actionLibPath, conf, fs);
 DistributedCache.createSymlink(conf);
 }
 listOfPaths = shareLibService.getSystemLibJars(getType());
 if (listOfPaths != null) {
 for (Path actionLibPath : listOfPaths) {
-DistributedCache.addFileToClassPath(actionLibPath, 
conf, fs);
+JobUtils.addFileToClassPath(actionLibPath, conf, fs);
 DistributedCache.createSymlink(conf);
 }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/6e2e46a9/core/src/main/java/org/apache/oozie/util/JobUtils.java
--
diff --git a/core/src/main/java/org/apache/oozie/util/JobUtils.java 
b/core/src/main/java/org/apache/oozie/util/JobUtils.java
index 25eb735..135b096 100644
--- a/core/src/main/java/org/apache/oozie/util/JobUtils.java
+++ b/core/src/main/java/org/apache/oozie/util/JobUtils.java
@@ -23,6 +23,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.oozie.ErrorCode;
@@ -133,4 +134,33 @@ public class JobUtils {
 
 return map;
 }
+
+/**
+ * This method provides a wrapper around hadoop 0.20/1.x and 0.23/2.x 
implementations.
+ * TODO: Remove the workaround when we drop the support for hadoop 0.20.
+ * @param file Path of the file to be added
+ * @param conf Configuration that contains the classpath setting
+ * @param fs FileSystem with respect to which path should be interpreted
+ * @throws IOException
+ */
+public static void addFileToClassPath(Path file, Configuration conf, 
FileSystem fs) throws IOException {
+  Configuration defaultConf = new Configuration();
+  XConfiguration.copy(conf, defaultConf);
+  DistributedCache.addFileToCla

git commit: OOZIE-1709 CoordELFunctions.getCurrentInstance() is expensive (shwethags via rohini)

2014-05-01 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master de7e72413 -> ef513d285


OOZIE-1709 CoordELFunctions.getCurrentInstance() is expensive (shwethags via 
rohini)


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

Branch: refs/heads/master
Commit: ef513d2853ce943ce3fea6aa6a3f6c9bdec25f63
Parents: de7e724
Author: Rohini Palaniswamy 
Authored: Thu May 1 10:16:07 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Thu May 1 10:16:07 2014 -0700

--
 .../apache/oozie/coord/CoordELFunctions.java| 119 ++-
 .../oozie/coord/TestCoordELFunctions.java   |  15 ++-
 release-log.txt |   1 +
 3 files changed, 102 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/ef513d28/core/src/main/java/org/apache/oozie/coord/CoordELFunctions.java
--
diff --git a/core/src/main/java/org/apache/oozie/coord/CoordELFunctions.java 
b/core/src/main/java/org/apache/oozie/coord/CoordELFunctions.java
index d73bc7d..db3259b 100644
--- a/core/src/main/java/org/apache/oozie/coord/CoordELFunctions.java
+++ b/core/src/main/java/org/apache/oozie/coord/CoordELFunctions.java
@@ -17,25 +17,28 @@
  */
 package org.apache.oozie.coord;
 
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.List;
-import java.util.TimeZone;
-
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.command.CommandException;
-import org.apache.oozie.dependency.URIHandler.Context;
 import org.apache.oozie.dependency.URIHandler;
+import org.apache.oozie.dependency.URIHandler.Context;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.service.URIHandlerService;
 import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.util.ELEvaluator;
 import org.apache.oozie.util.ParamChecker;
 import org.apache.oozie.util.XLog;
-import org.apache.oozie.service.Services;
-import org.apache.oozie.service.URIHandlerService;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import java.util.TimeZone;
 
 /**
  * This class implements the EL function related to coordinator
@@ -52,6 +55,12 @@ public class CoordELFunctions {
 // TODO: in next release, support flexibility
 private static String END_OF_OPERATION_INDICATOR_FILE = "_SUCCESS";
 
+public static final long MINUTE_MSEC = 60 * 1000L;
+public static final long HOUR_MSEC = 60 * MINUTE_MSEC;
+public static final long DAY_MSEC = 24 * HOUR_MSEC;
+public static final long MONTH_MSEC = 30 * DAY_MSEC;
+public static final long YEAR_MSEC = 365 * DAY_MSEC;
+
 /**
  * Used in defining the frequency in 'day' unit.  domain:  val 
> 0 and should be integer.
  *
@@ -922,7 +931,6 @@ public class CoordELFunctions {
 TimeUnit dsTimeUnit = getDSTimeUnit();
 int[] instCount = new int[1];// used as pass by ref
 Calendar nominalInstanceCal = 
getCurrentInstance(getActionCreationtime(), instCount);
-StringBuilder instanceList = new StringBuilder();
 if (nominalInstanceCal == null) {
 LOG.warn("If the initial instance of the dataset is later than the 
nominal time, an empty string is"
 + " returned. This means that no data is available at the 
current-instance specified by the user"
@@ -930,33 +938,25 @@ public class CoordELFunctions {
 return "";
 } else {
 Calendar initInstance = getInitialInstanceCal();
-instCount[0] = instCount[0] + end;
 // Add in the reverse order - newest instance first.
-for (int i = end; i >= start; i--) {
-// Tried to avoid the clone. But subtracting datasetFrequency 
gives different results than multiplying
-// and Spring DST transition test in 
TestCoordELfunctions.testCurrent() fails
-//nominalInstanceCal.add(dsTimeUnit.getCalendarUnit(), 
-datasetFrequency);
-nominalInstanceCal = (Calendar) initInstance.clone();
-nominalInstanceCal.add(dsTimeUnit.getCalendarUnit(), 
instCount[0] * datasetFrequency);
-instCount[0]--;
+nominalInstanceCal = (Calendar) initInstance.clone();
+nominalInstanceC

git commit: OOZIE-1785 Add oozie email action xsd to OozieCli.java (jagatsingh via rohini)

2014-04-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 14599a0a5 -> aff5941f4


OOZIE-1785 Add oozie email action xsd to OozieCli.java (jagatsingh via rohini)


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

Branch: refs/heads/master
Commit: aff5941f4d5724dbc07ed0629db03e6a9ff9a62d
Parents: 14599a0
Author: Rohini Palaniswamy 
Authored: Wed Apr 23 10:03:16 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Apr 23 10:03:16 2014 -0700

--
 client/src/main/java/org/apache/oozie/cli/OozieCLI.java | 2 ++
 release-log.txt | 1 +
 2 files changed, 3 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/aff5941f/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
--
diff --git a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java 
b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
index d964889..9929040 100644
--- a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
+++ b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
@@ -1730,6 +1730,8 @@ public class OozieCLI {
 sources.add(new 
StreamSource(Thread.currentThread().getContextClassLoader().getResourceAsStream(
 "email-action-0.1.xsd")));
 sources.add(new 
StreamSource(Thread.currentThread().getContextClassLoader().getResourceAsStream(
+"email-action-0.2.xsd")));
+sources.add(new 
StreamSource(Thread.currentThread().getContextClassLoader().getResourceAsStream(
 "distcp-action-0.1.xsd")));
 sources.add(new 
StreamSource(Thread.currentThread().getContextClassLoader().getResourceAsStream(
 "distcp-action-0.2.xsd")));

http://git-wip-us.apache.org/repos/asf/oozie/blob/aff5941f/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 0a58977..b7daf42 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1785 Add oozie email action xsd to OozieCli.java (jagatsingh via rohini)
 OOZIE-1527 Fix scalability issues with coordinator materialization (puru via 
rohini)
 OOZIE-1797 Workflow rerun command should use existing workflow properties 
(puru via rohini)
 OOZIE-1769 An option to update coord properties/definition (puru via rohini)



git commit: OOZIE-1527 Fix scalability issues with coordinator materialization (puru via rohini)

2014-04-23 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master c707867b7 -> 14599a0a5


 OOZIE-1527 Fix scalability issues with coordinator materialization (puru via 
rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/14599a0a
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/14599a0a
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/14599a0a

Branch: refs/heads/master
Commit: 14599a0a5c6cde5209b5f99574d65860dd9bffef
Parents: c707867
Author: Rohini Palaniswamy 
Authored: Wed Apr 23 09:45:09 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Apr 23 09:45:09 2014 -0700

--
 .../org/apache/oozie/CoordinatorJobBean.java|  5 +-
 .../CoordMaterializeTransitionXCommand.java | 65 +-
 .../executor/jpa/CoordJobQueryExecutor.java | 16 +++-
 .../CoordJobsToBeMaterializedJPAExecutor.java   |  2 +-
 .../service/CoordMaterializeTriggerService.java | 48 --
 core/src/main/resources/oozie-default.xml   | 36 +---
 .../TestCoordMaterializeTransitionXCommand.java | 92 +++-
 ...estCoordJobsToBeMaterializedJPAExecutor.java |  2 +
 .../TestCoordMaterializeTriggerService.java | 83 ++
 release-log.txt |  1 +
 10 files changed, 297 insertions(+), 53 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/14599a0a/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
--
diff --git a/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java 
b/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
index 4a2ea39..7915698 100644
--- a/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
+++ b/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
@@ -91,7 +91,7 @@ import org.json.simple.JSONObject;
 
 @NamedQuery(name = "GET_COORD_JOB_ACTION_KILL", query = "select w.id, 
w.user, w.group, w.appName, w.statusStr from CoordinatorJobBean w where w.id = 
:id"),
 
-@NamedQuery(name = "GET_COORD_JOB_MATERIALIZE", query = "select w.id, 
w.user, w.group, w.appName, w.statusStr, w.frequency, w.matThrottling, 
w.timeOut, w.timeZone, w.startTimestamp, w.endTimestamp, w.pauseTimestamp, 
w.nextMaterializedTimestamp, w.lastActionTimestamp, w.lastActionNumber, 
w.doneMaterialization, w.bundleId, w.conf, w.jobXml, w.appNamespace from 
CoordinatorJobBean w where w.id = :id"),
+@NamedQuery(name = "GET_COORD_JOB_MATERIALIZE", query = "select w.id, 
w.user, w.group, w.appName, w.statusStr, w.frequency, w.matThrottling, 
w.timeOut, w.timeZone, w.startTimestamp, w.endTimestamp, w.pauseTimestamp, 
w.nextMaterializedTimestamp, w.lastActionTimestamp, w.lastActionNumber, 
w.doneMaterialization, w.bundleId, w.conf, w.jobXml, w.appNamespace, 
w.timeUnitStr from CoordinatorJobBean w where w.id = :id"),
 
 @NamedQuery(name = "GET_COORD_JOB_SUSPEND_KILL", query = "select w.id, 
w.user, w.group, w.appName, w.statusStr, w.bundleId, w.appNamespace, 
w.doneMaterialization from CoordinatorJobBean w where w.id = :id"),
 
@@ -103,8 +103,11 @@ import org.json.simple.JSONObject;
 
 @NamedQuery(name = "GET_COORD_JOBS_COLUMNS", query = "select w.id, 
w.appName, w.statusStr, w.user, w.group, w.startTimestamp, w.endTimestamp, 
w.appPath, w.concurrency, w.frequency, w.lastActionTimestamp, 
w.nextMaterializedTimestamp, w.createdTimestamp, w.timeUnitStr, w.timeZone, 
w.timeOut from CoordinatorJobBean w order by w.createdTimestamp desc"),
 
+//TODO need to remove.
 @NamedQuery(name = "GET_COORD_JOBS_OLDER_THAN", query = "select 
OBJECT(w) from CoordinatorJobBean w where w.startTimestamp <= :matTime AND 
(w.statusStr = 'PREP' OR w.statusStr = 'RUNNING' or w.statusStr = 
'RUNNINGWITHERROR') AND (w.nextMaterializedTimestamp < :matTime OR 
w.nextMaterializedTimestamp IS NULL) AND (w.nextMaterializedTimestamp IS NULL 
OR (w.endTimestamp > w.nextMaterializedTimestamp AND (w.pauseTimestamp IS NULL 
OR w.pauseTimestamp > w.nextMaterializedTimestamp))) order by 
w.lastModifiedTimestamp"),
 
+@NamedQuery(name = "GET_COORD_JOBS_OLDER_FOR_MATERILZATION", query = 
"select w.id from CoordinatorJobBean w where w.startTimestamp <= :matTime AND 
(w.statusStr = 'PREP' OR w.statusStr = 'RUNNING' or w.statusStr = 
'RUNNINGWITHERROR') AND (w.nextMaterializedTimestamp < :matTime OR 
w.nextMaterializedTimestamp IS NULL) AND (w.nextMaterializedTimestamp IS NULL 
OR (w.endTimestamp > w.nextMaterializedTimestamp AND (w.pauseTimestamp IS NULL 
OR w.pauseTimestamp > w.nextMaterializedTimestamp))) a

git commit: OOZIE-1797 Workflow rerun command should use existing workflow properties (puru via rohini)

2014-04-21 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master cc4b4398f -> c707867b7


OOZIE-1797 Workflow rerun command should use existing workflow properties (puru 
via rohini)


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

Branch: refs/heads/master
Commit: c707867b75319cf2a3209ed7f664e78fde75541d
Parents: cc4b439
Author: Rohini Palaniswamy 
Authored: Mon Apr 21 16:57:05 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Apr 21 16:57:05 2014 -0700

--
 .../java/org/apache/oozie/cli/OozieCLI.java |  37 ++-
 .../main/java/org/apache/oozie/DagEngine.java   |  18 +++-
 .../apache/oozie/servlet/BaseJobServlet.java|   6 +-
 .../oozie/command/wf/TestReRunXCommand.java | 101 ++-
 .../apache/oozie/servlet/TestV0JobServlet.java  |   9 --
 docs/src/site/twiki/DG_WorkflowReRun.twiki  |   2 +
 release-log.txt |   1 +
 7 files changed, 152 insertions(+), 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/c707867b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
--
diff --git a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java 
b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
index c31c8ee..d964889 100644
--- a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
+++ b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
@@ -707,12 +707,12 @@ public class OozieCLI {
 }
 
 private Properties getConfiguration(OozieClient wc, CommandLine 
commandLine) throws IOException {
+if (!isConfigurationSpecified(wc, commandLine)) {
+throw new IOException("configuration is not specified");
+}
 Properties conf = wc.createConfiguration();
 String configFile = commandLine.getOptionValue(CONFIG_OPTION);
-if (configFile == null) {
-throw new IOException("configuration file not specified");
-}
-else {
+if (configFile != null) {
 File file = new File(configFile);
 if (!file.exists()) {
 throw new IOException("configuration file [" + configFile + "] 
not found");
@@ -735,6 +735,28 @@ public class OozieCLI {
 }
 
 /**
+ * Check if configuration has specified
+ * @param wc
+ * @param commandLine
+ * @return
+ * @throws IOException
+ */
+private boolean isConfigurationSpecified(OozieClient wc, CommandLine 
commandLine) throws IOException {
+boolean isConf = false;
+String configFile = commandLine.getOptionValue(CONFIG_OPTION);
+if (configFile == null) {
+isConf = false;
+}
+else {
+isConf = new File(configFile).exists();
+}
+if (commandLine.hasOption("D")) {
+isConf = true;
+}
+return isConf;
+}
+
+/**
  * @param commandLine command line string.
  * @return change value specified by -value.
  * @throws OozieCLIException
@@ -904,7 +926,12 @@ public class OozieCLI {
 }
 else if (options.contains(RERUN_OPTION)) {
 if (commandLine.getOptionValue(RERUN_OPTION).contains("-W")) {
-wc.reRun(commandLine.getOptionValue(RERUN_OPTION), 
getConfiguration(wc, commandLine));
+if (isConfigurationSpecified(wc, commandLine)) {
+wc.reRun(commandLine.getOptionValue(RERUN_OPTION), 
getConfiguration(wc, commandLine));
+}
+else {
+wc.reRun(commandLine.getOptionValue(RERUN_OPTION), new 
Properties());
+}
 }
 else if 
(commandLine.getOptionValue(RERUN_OPTION).contains("-B")) {
 String bundleJobId = 
commandLine.getOptionValue(RERUN_OPTION);

http://git-wip-us.apache.org/repos/asf/oozie/blob/c707867b/core/src/main/java/org/apache/oozie/DagEngine.java
--
diff --git a/core/src/main/java/org/apache/oozie/DagEngine.java 
b/core/src/main/java/org/apache/oozie/DagEngine.java
index 7d316b8..cad5ddd 100644
--- a/core/src/main/java/org/apache/oozie/DagEngine.java
+++ b/core/src/main/java/org/apache/oozie/DagEngine.java
@@ -42,13 +42,18 @@ import org.apache.oozie.command.wf.SubmitSqoopXCommand;
 import org.apache.oozie.command.wf.SubmitXCommand;
 import org.apache.oozie.command.wf.SuspendXCommand;
 import org.apache.oozie.command.wf.WorkflowActionInfoXCom

git commit: OOZIE-1796 Job status should not transition from KILLED (puru via rohini)

2014-04-16 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master e7bf02a8f -> d750451bc


OOZIE-1796 Job status should not transition from KILLED (puru via rohini)


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

Branch: refs/heads/master
Commit: d750451bc2dc3ed22af2537b2cc4f9994b110df5
Parents: e7bf02a
Author: Rohini Palaniswamy 
Authored: Wed Apr 16 10:00:52 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Wed Apr 16 10:00:52 2014 -0700

--
 .../oozie/service/StatusTransitService.java |  11 ++
 .../oozie/service/TestStatusTransitService.java | 100 +--
 release-log.txt |   1 +
 3 files changed, 101 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/d750451b/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/service/StatusTransitService.java 
b/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
index e093c7d..0789d84 100644
--- a/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
+++ b/core/src/main/java/org/apache/oozie/service/StatusTransitService.java
@@ -337,6 +337,11 @@ public class StatusTransitService implements Service {
 }
 
 if (bundleActions.size() == (totalValuesSucceed + 
totalValuesFailed + totalValuesKilled + totalValuesDoneWithError)) {
+// If all bundle action is done and bundle is killed, then 
don't change the status.
+if (bundleStatus[0].equals(Job.Status.KILLED)) {
+bundleStatus[0] = Job.Status.KILLED;
+return true;
+}
 // If all the bundle actions are succeeded then bundle job 
should be succeeded.
 if (bundleActions.size() == totalValuesSucceed) {
 bundleStatus[0] = Job.Status.SUCCEEDED;
@@ -382,6 +387,12 @@ public class StatusTransitService implements Service {
 }
 
 if (coordActionsCount == (totalValuesSucceed + totalValuesFailed + 
totalValuesKilled + totalValuesTimeOut)) {
+
+// If all coord action is done and coord is killed, then don't 
change the status.
+if (coordStatus[0].equals(Job.Status.KILLED)) {
+coordStatus[0] = Job.Status.KILLED;
+return true;
+}
 // If all the coordinator actions are succeeded then 
coordinator job should be succeeded.
 if (coordActionsCount == totalValuesSucceed && 
isDoneMaterialization) {
 coordStatus[0] = Job.Status.SUCCEEDED;

http://git-wip-us.apache.org/repos/asf/oozie/blob/d750451b/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java
--
diff --git 
a/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java 
b/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java
index d5c2c4b..b767b69 100644
--- a/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java
+++ b/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java
@@ -38,12 +38,18 @@ import org.apache.oozie.command.coord.CoordKillXCommand;
 import org.apache.oozie.command.coord.CoordResumeXCommand;
 import org.apache.oozie.command.coord.CoordSuspendXCommand;
 import org.apache.oozie.executor.jpa.BundleActionGetJPAExecutor;
+import org.apache.oozie.executor.jpa.BundleActionQueryExecutor;
+import 
org.apache.oozie.executor.jpa.BundleActionQueryExecutor.BundleActionQuery;
 import org.apache.oozie.executor.jpa.BundleJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.BundleJobInsertJPAExecutor;
+import org.apache.oozie.executor.jpa.BundleJobQueryExecutor;
+import org.apache.oozie.executor.jpa.BundleJobQueryExecutor.BundleJobQuery;
 import org.apache.oozie.executor.jpa.CoordActionGetJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordActionQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobInsertJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
+import org.apache.oozie.executor.jpa.CoordActionQueryExecutor.CoordActionQuery;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
@@ -714,12 +720,11 @@ public class TestStatusTransitService e

git commit: OOZIE-1792 Ability to kill bundle stuck in RUNNING due to inconsistent pending states (rohini)

2014-04-15 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master f6b3f0e8c -> 1e0ed571f


OOZIE-1792 Ability to kill bundle stuck in RUNNING due to inconsistent pending 
states (rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/1e0ed571
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/1e0ed571
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/1e0ed571

Branch: refs/heads/master
Commit: 1e0ed571f72348fb47b96dd93f6f35c1a09dcb3f
Parents: f6b3f0e
Author: Rohini Palaniswamy 
Authored: Tue Apr 15 10:48:08 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Apr 15 10:48:08 2014 -0700

--
 .../command/bundle/BundleKillXCommand.java  | 25 
 release-log.txt |  1 +
 2 files changed, 16 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/1e0ed571/core/src/main/java/org/apache/oozie/command/bundle/BundleKillXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/bundle/BundleKillXCommand.java 
b/core/src/main/java/org/apache/oozie/command/bundle/BundleKillXCommand.java
index cc0d9d5..3ab3273 100644
--- a/core/src/main/java/org/apache/oozie/command/bundle/BundleKillXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/bundle/BundleKillXCommand.java
@@ -129,17 +129,22 @@ public class BundleKillXCommand extends 
KillTransitionXCommand {
 else {
 // Due to race condition bundle action pending might be true
 // while coordinator is killed.
-if (action.isPending() && action.getCoordId() != null) {
-try {
-CoordinatorJobBean coordJob = 
CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB,
-action.getCoordId());
-if (!coordJob.isPending() && coordJob.isTerminalStatus()) {
-action.decrementAndGetPending();
-action.setStatus(coordJob.getStatus());
-}
+if (action.isPending()) {
+if (action.getCoordId() == null) {
+action.setPending(0);
 }
-catch (JPAExecutorException e) {
-LOG.warn("Error in checking coord job status:" + 
action.getCoordId(), e);
+else {
+try {
+CoordinatorJobBean coordJob = 
CoordJobQueryExecutor.getInstance().get(
+CoordJobQuery.GET_COORD_JOB, 
action.getCoordId());
+if (!coordJob.isPending() && 
coordJob.isTerminalStatus()) {
+action.setPending(0);
+action.setStatus(coordJob.getStatus());
+}
+}
+catch (JPAExecutorException e) {
+LOG.warn("Error in checking coord job status:" + 
action.getCoordId(), e);
+}
 }
 }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/1e0ed571/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 413932b..ecc4441 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1792 Ability to kill bundle stuck in RUNNING due to inconsistent pending 
states (rohini)
 OOZIE-1703 User should be able to set coord end-time before start time (puru 
via rohini)
 OOZIE-1719 v1/jobs api returns null for parentId even when it exists (ryota)
 OOZIE-1773 bulk API returns total = 0 when it's not (ryota)



git commit: OOZIE-1703 User should be able to set coord end-time before start time (puru via rohini)

2014-04-15 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 885676214 -> f6b3f0e8c


OOZIE-1703 User should be able to set coord end-time before start time (puru 
via rohini)


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

Branch: refs/heads/master
Commit: f6b3f0e8c38e185b34bc2cd3b4f16533cfa505fc
Parents: 8856762
Author: Rohini Palaniswamy 
Authored: Tue Apr 15 10:25:22 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Apr 15 10:25:22 2014 -0700

--
 .../command/coord/CoordChangeXCommand.java  | 170 -
 .../TestCoordActionInputCheckXCommand.java  |   2 +-
 .../command/coord/TestCoordChangeXCommand.java  | 181 +++
 .../org/apache/oozie/test/XDataTestCase.java|  43 -
 docs/src/site/twiki/DG_CommandLineTool.twiki|  22 ++-
 release-log.txt |   1 +
 6 files changed, 273 insertions(+), 146 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/f6b3f0e8/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java 
b/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
index 5ecd5d5..fb31e9a 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
@@ -18,14 +18,12 @@
 package org.apache.oozie.command.coord;
 
 import java.util.ArrayList;
-import java.util.Calendar;
 import java.util.Date;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.oozie.CoordinatorActionBean;
 import org.apache.oozie.CoordinatorJobBean;
@@ -34,13 +32,11 @@ import org.apache.oozie.XException;
 import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.CoordinatorJob;
 import org.apache.oozie.client.Job;
-import org.apache.oozie.client.Job.Status;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.rest.JsonBean;
 import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.PreconditionException;
 import org.apache.oozie.command.bundle.BundleStatusUpdateXCommand;
-import org.apache.oozie.coord.TimeUnit;
 import org.apache.oozie.executor.jpa.CoordActionGetJPAExecutor;
 import 
org.apache.oozie.executor.jpa.CoordJobGetActionByActionNumberJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
@@ -170,21 +166,6 @@ public class CoordChangeXCommand extends 
CoordinatorXCommand {
 }
 
 /**
- * Returns the actual last action time(one instance before 
coordJob.lastActionTime)
- * @return Date - last action time if coordJob.getLastActionTime() is not 
null, null otherwise
- */
-private Date getLastActionTime() {
-if(coordJob.getLastActionTime() == null)
-return null;
-
-Calendar d = 
Calendar.getInstance(DateUtils.getTimeZone(coordJob.getTimeZone()));
-d.setTime(coordJob.getLastActionTime());
-TimeUnit timeUnit = TimeUnit.valueOf(coordJob.getTimeUnitStr());
-d.add(timeUnit.getCalendarUnit(), 
-Integer.valueOf(coordJob.getFrequency()));
-return d.getTime();
-}
-
-/**
  * Check if new end time is valid.
  *
  * @param coordJob coordinator job id.
@@ -192,21 +173,7 @@ public class CoordChangeXCommand extends 
CoordinatorXCommand {
  * @throws CommandException thrown if new end time is not valid.
  */
 private void checkEndTime(CoordinatorJobBean coordJob, Date newEndTime) 
throws CommandException {
-// New endTime cannot be before coordinator job's start time.
-Date startTime = coordJob.getStartTime();
-if (newEndTime.before(startTime)) {
-throw new CommandException(ErrorCode.E1015, newEndTime, "cannot be 
before coordinator job's start time ["
-+ startTime + "]");
-}
-
-// New endTime cannot be before coordinator job's last action time.
-Date lastActionTime = getLastActionTime();
-if (lastActionTime != null) {
-if (!newEndTime.after(lastActionTime)) {
-throw new CommandException(ErrorCode.E1015, newEndTime,
-"must be after coordinator job's last action time [" + 
lastActionTime + "]");
-}
-}
+//It's ok to set end date before start date.
  

git commit: OOZIE-1761 Improve sharelib purging logic (puru via rohini)

2014-04-04 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 5045f89cf -> a015a45ee


OOZIE-1761 Improve sharelib purging logic (puru via rohini)


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

Branch: refs/heads/master
Commit: a015a45ee50a23add71ee84e8ef6a6bd00d8aa45
Parents: 5045f89
Author: Rohini Palaniswamy 
Authored: Fri Apr 4 12:22:38 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Fri Apr 4 12:22:38 2014 -0700

--
 .../apache/oozie/service/ShareLibService.java   |  45 +---
 .../oozie/service/TestShareLibService.java  | 113 ---
 release-log.txt |   1 +
 3 files changed, 100 insertions(+), 59 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/a015a45e/core/src/main/java/org/apache/oozie/service/ShareLibService.java
--
diff --git a/core/src/main/java/org/apache/oozie/service/ShareLibService.java 
b/core/src/main/java/org/apache/oozie/service/ShareLibService.java
index 89939f1..73ca492 100644
--- a/core/src/main/java/org/apache/oozie/service/ShareLibService.java
+++ b/core/src/main/java/org/apache/oozie/service/ShareLibService.java
@@ -92,6 +92,8 @@ public class ShareLibService implements Service, 
Instrumentable {
 
 FileSystem fs;
 
+final long retentionTime = 1000 * 60 * 60 * 24 * 
Services.get().getConf().getInt(LAUNCHERJAR_LIB_RETENTION, 7);
+
 @Override
 public void init(Services services) throws ServiceException {
 this.services = services;
@@ -106,8 +108,9 @@ public class ShareLibService implements Service, 
Instrumentable {
 updateShareLib();
 //Only one server should purge sharelib
 if 
(Services.get().get(JobsConcurrencyService.class).isFirstServer()) {
-purgeLibs(fs, LAUNCHER_PREFIX);
-purgeLibs(fs, SHARED_LIB_PREFIX);
+final Date current = 
Calendar.getInstance(TimeZone.getTimeZone("GMT")).getTime();
+purgeLibs(fs, LAUNCHER_PREFIX, current);
+purgeLibs(fs, SHARED_LIB_PREFIX, current);
 }
 }
 catch (Exception e) {
@@ -359,14 +362,26 @@ public class ShareLibService implements Service, 
Instrumentable {
  * @throws IOException Signals that an I/O exception has occurred.
  * @throws ParseException the parse exception
  */
-private void purgeLibs(FileSystem fs, final String prefix) throws 
IOException, ParseException {
-Configuration conf = services.getConf();
+private void purgeLibs(FileSystem fs, final String prefix, final Date 
current) throws IOException, ParseException {
 Path executorLibBasePath = 
services.get(WorkflowAppService.class).getSystemLibPath();
-
 PathFilter directoryFilter = new PathFilter() {
 @Override
 public boolean accept(Path path) {
-return path.getName().startsWith(prefix);
+if (path.getName().startsWith(prefix)) {
+String name = path.getName().toString();
+String time = name.substring(prefix.length());
+Date d = null;
+try {
+d = dateFormat.parse(time);
+}
+catch (ParseException e) {
+return false;
+}
+return (current.getTime() - d.getTime()) > retentionTime;
+}
+else {
+return false;
+}
 }
 };
 FileStatus[] dirList = fs.listStatus(executorLibBasePath, 
directoryFilter);
@@ -376,20 +391,14 @@ public class ShareLibService implements Service, 
Instrumentable {
 public int compare(FileStatus o1, FileStatus o2) {
 return 
o2.getPath().getName().compareTo(o1.getPath().getName());
 }
-
 });
-Date current = 
Calendar.getInstance(TimeZone.getTimeZone("GMT")).getTime();
-// Always keep top two, so start counter from 3
-long retentionTime = 1000 * 60 * 60 * 24 * 
conf.getInt(LAUNCHERJAR_LIB_RETENTION, 7);
-for (int i = 2; i < dirList.length; i++) {
+
+//Logic is to keep all share-lib between current timestamp and 7days 
old + 1 latest sharelib older than 7 days.
+// refer OOZIE-1761
+for (int i = 1; i < dirList.length; i++) {
 Path dirPath = dirList[i].getPath();
-String name = dirPath.getName().toString();
-String time = name.substring(prefix.length());
-  

git commit: OOZIE-1735 Support resuming of failed coordinator job and rerun of a failed coordinator action (puru via rohini)

2014-03-27 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master dc22c9022 -> 3de3c523d


OOZIE-1735 Support resuming of failed coordinator job and rerun of a failed 
coordinator action (puru via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/3de3c523
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/3de3c523
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/3de3c523

Branch: refs/heads/master
Commit: 3de3c523d4645b2150e21c12f599de0cb5b333b4
Parents: dc22c90
Author: Rohini Palaniswamy 
Authored: Thu Mar 27 17:04:29 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Thu Mar 27 17:04:29 2014 -0700

--
 .../java/org/apache/oozie/cli/OozieCLI.java |  2 +-
 .../org/apache/oozie/client/OozieClient.java|  2 +
 .../command/coord/CoordChangeXCommand.java  | 62 ++--
 .../oozie/command/coord/CoordRerunXCommand.java | 10 --
 .../command/coord/TestCoordChangeXCommand.java  | 99 
 .../command/coord/TestCoordRerunXCommand.java   | 37 
 docs/src/site/twiki/DG_CommandLineTool.twiki| 12 ++-
 release-log.txt |  1 +
 8 files changed, 205 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/3de3c523/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
--
diff --git a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java 
b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
index 6dc4a3b..e1f551d 100644
--- a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
+++ b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
@@ -720,7 +720,7 @@ public class OozieCLI {
  * @return change value specified by -value.
  * @throws OozieCLIException
  */
-private String getChangeValue(CommandLine commandLine) throws 
OozieCLIException {
+   private String getChangeValue(CommandLine commandLine) throws 
OozieCLIException {
 String changeValue = commandLine.getOptionValue(CHANGE_VALUE_OPTION);
 
 if (changeValue == null) {

http://git-wip-us.apache.org/repos/asf/oozie/blob/3de3c523/client/src/main/java/org/apache/oozie/client/OozieClient.java
--
diff --git a/client/src/main/java/org/apache/oozie/client/OozieClient.java 
b/client/src/main/java/org/apache/oozie/client/OozieClient.java
index 6164447..46c4288 100644
--- a/client/src/main/java/org/apache/oozie/client/OozieClient.java
+++ b/client/src/main/java/org/apache/oozie/client/OozieClient.java
@@ -148,6 +148,8 @@ public class OozieClient {
 
 public static final String CHANGE_VALUE_CONCURRENCY = "concurrency";
 
+public static final String CHANGE_VALUE_STATUS = "status";
+
 public static final String LIBPATH = "oozie.libpath";
 
 public static final String USE_SYSTEM_LIBPATH = "oozie.use.system.libpath";

http://git-wip-us.apache.org/repos/asf/oozie/blob/3de3c523/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java 
b/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
index 4957330..5ecd5d5 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordChangeXCommand.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.oozie.CoordinatorActionBean;
 import org.apache.oozie.CoordinatorJobBean;
 import org.apache.oozie.ErrorCode;
@@ -33,6 +34,7 @@ import org.apache.oozie.XException;
 import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.CoordinatorJob;
 import org.apache.oozie.client.Job;
+import org.apache.oozie.client.Job.Status;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.rest.JsonBean;
 import org.apache.oozie.command.CommandException;
@@ -66,6 +68,7 @@ public class CoordChangeXCommand extends 
CoordinatorXCommand {
 private Date newPauseTime = null;
 private Date oldPauseTime = null;
 private boolean resetPauseTime = false;
+private CoordinatorJob.Status jobStatus = null;
 private CoordinatorJobBean coordJob;
 private JPAService jpaService = null;
 private Job.Status prevStatus;
@@ -77,6 +80,8 @@ public class CoordChangeXCommand extends 
CoordinatorXCommand {
 ALLOWED_CHANGE_OPTIONS.add("endtime");
 ALLOWED_CHANGE_OPTIONS.add("concurrency");
 ALLOWED_CHANGE_OPTIONS.add("pausetime&qu

git commit: OOZIE-1713 Avoid creating dummy input file for each launcher job (puru via rohini)

2014-03-25 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 359d08d22 -> 4c28591b7


OOZIE-1713 Avoid creating dummy input file for each launcher job (puru via 
rohini)


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

Branch: refs/heads/master
Commit: 4c28591b76e1927af3a2f41b4b295ca5cd3eeb00
Parents: 359d08d
Author: Rohini Palaniswamy 
Authored: Tue Mar 25 13:35:58 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Tue Mar 25 13:35:58 2014 -0700

--
 .../oozie/action/hadoop/JavaActionExecutor.java |   1 +
 .../action/hadoop/LauncherMapperHelper.java |   9 +-
 release-log.txt |   1 +
 .../action/hadoop/OozieLauncherInputFormat.java | 118 +++
 4 files changed, 121 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/4c28591b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index 860eb51..52a0953 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -134,6 +134,7 @@ public class JavaActionExecutor extends ActionExecutor {
 public static List getCommonLauncherClasses() {
 List classes = new ArrayList();
 classes.add(LauncherMapper.class);
+classes.add(OozieLauncherInputFormat.class);
 
classes.addAll(Services.get().get(URIHandlerService.class).getClassesForLauncher());
 return classes;
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/4c28591b/core/src/main/java/org/apache/oozie/action/hadoop/LauncherMapperHelper.java
--
diff --git 
a/core/src/main/java/org/apache/oozie/action/hadoop/LauncherMapperHelper.java 
b/core/src/main/java/org/apache/oozie/action/hadoop/LauncherMapperHelper.java
index eeda715..b893200 100644
--- 
a/core/src/main/java/org/apache/oozie/action/hadoop/LauncherMapperHelper.java
+++ 
b/core/src/main/java/org/apache/oozie/action/hadoop/LauncherMapperHelper.java
@@ -155,14 +155,7 @@ public class LauncherMapperHelper {
 OutputStream os = fs.create(new Path(actionDir, 
LauncherMapper.ACTION_CONF_XML));
 actionConf.writeXml(os);
 os.close();
-
-Path inputDir = new Path(actionDir, "input");
-fs.mkdirs(inputDir);
-Writer writer = new OutputStreamWriter(fs.create(new Path(inputDir, 
"dummy.txt")));
-writer.write("dummy");
-writer.close();
-
-launcherConf.set("mapred.input.dir", inputDir.toString());
+launcherConf.setInputFormat(OozieLauncherInputFormat.class);
 launcherConf.set("mapred.output.dir", new Path(actionDir, 
"output").toString());
 }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/4c28591b/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 1358824..0fbdde5 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1713 Avoid creating dummy input file for each launcher job (puru via 
rohini)
 OOZIE-1701 TestXTestCase#testWaitFor and TestXTestCase#testBaseDir have the 
errors (omaliuvanchuk via rohini)
 OOZIE-1751 Default authentication type using env variable for oozie CLI (puru 
via rohini)
 OOZIE-1753 Update create-release-artifact script for git (rkanter)

http://git-wip-us.apache.org/repos/asf/oozie/blob/4c28591b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherInputFormat.java
--
diff --git 
a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherInputFormat.java
 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherInputFormat.java
new file mode 100644
index 000..cf43225
--- /dev/null
+++ 
b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/OozieLauncherInputFormat.java
@@ -0,0 +1,118 @@
+/**
+ * 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 

git commit: OOZIE-1701 TestXTestCase#testWaitFor and TestXTestCase#testBaseDir have the errors (omaliuvanchuk via rohini)

2014-03-24 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master b94bcf4d1 -> 9a82b0f82


OOZIE-1701 TestXTestCase#testWaitFor and TestXTestCase#testBaseDir have the 
errors (omaliuvanchuk via rohini)


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

Branch: refs/heads/master
Commit: 9a82b0f82d4d01f7020b52010df24d83508dcdc6
Parents: b94bcf4
Author: Rohini Palaniswamy 
Authored: Mon Mar 24 12:15:13 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Mar 24 12:15:13 2014 -0700

--
 .../test/java/org/apache/oozie/test/TestXTestCase.java  | 12 
 release-log.txt |  1 +
 2 files changed, 13 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/9a82b0f8/core/src/test/java/org/apache/oozie/test/TestXTestCase.java
--
diff --git a/core/src/test/java/org/apache/oozie/test/TestXTestCase.java 
b/core/src/test/java/org/apache/oozie/test/TestXTestCase.java
index f6a75e2..eeac8ac 100644
--- a/core/src/test/java/org/apache/oozie/test/TestXTestCase.java
+++ b/core/src/test/java/org/apache/oozie/test/TestXTestCase.java
@@ -93,6 +93,18 @@ public class TestXTestCase extends TestCase {
 
 
 public static class MyXTestCase extends XTestCase {
+protected void setUp() throws Exception {
+if(TESTING){
+super.setUp();
+}
+}
+
+protected void tearDown() throws Exception {
+if(TESTING){
+super.tearDown();
+}
+}
+
 
 public void testDummy() {
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/9a82b0f8/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index f97c3f8..b68bae2 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1701 TestXTestCase#testWaitFor and TestXTestCase#testBaseDir have the 
errors (omaliuvanchuk via rohini)
 OOZIE-1751 Default authentication type using env variable for oozie CLI (puru 
via rohini)
 OOZIE-1753 Update create-release-artifact script for git (rkanter)
 OOZIE-1704 Add ability to use Bulk API with bundle ID (mona)



git commit: OOZIE-1751 Default authentication type using env variable for oozie CLI (puru via rohini)

2014-03-24 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 50fd33c5f -> b94bcf4d1


OOZIE-1751 Default authentication type using env variable for oozie CLI (puru 
via rohini)


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

Branch: refs/heads/master
Commit: b94bcf4d1efc13edb0be0ffd8697b9cf28266253
Parents: 50fd33c
Author: Rohini Palaniswamy 
Authored: Mon Mar 24 12:04:12 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Mar 24 12:04:12 2014 -0700

--
 client/src/main/java/org/apache/oozie/cli/OozieCLI.java | 8 
 docs/src/site/twiki/DG_CommandLineTool.twiki| 1 +
 release-log.txt | 1 +
 3 files changed, 10 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/b94bcf4d/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
--
diff --git a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java 
b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
index 87e2f27..a6203c6 100644
--- a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
+++ b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
@@ -80,6 +80,7 @@ public class OozieCLI {
 public static final String ENV_OOZIE_URL = "OOZIE_URL";
 public static final String ENV_OOZIE_DEBUG = "OOZIE_DEBUG";
 public static final String ENV_OOZIE_TIME_ZONE = "OOZIE_TIMEZONE";
+public static final String ENV_OOZIE_AUTH = "OOZIE_AUTH";
 public static final String WS_HEADER_PREFIX = "header:";
 
 public static final String HELP_CMD = "help";
@@ -154,6 +155,7 @@ public class OozieCLI {
 private static final String[] OOZIE_HELP = {
 "the env variable '" + ENV_OOZIE_URL + "' is used as default value 
for the '-" + OOZIE_OPTION + "' option",
 "the env variable '" + ENV_OOZIE_TIME_ZONE + "' is used as default 
value for the '-" + TIME_ZONE_OPTION + "' option",
+"the env variable '" + ENV_OOZIE_AUTH + "' is used as default 
value for the '-" + AUTH_OPTION + "' option",
 "custom headers for Oozie web services can be specified using '-D" 
+ WS_HEADER_PREFIX + "NAME=VALUE'" };
 
 private static final String RULER;
@@ -741,6 +743,12 @@ public class OozieCLI {
  */
 protected String getAuthOption(CommandLine commandLine) {
 String authOpt = commandLine.getOptionValue(AUTH_OPTION);
+if (authOpt == null) {
+authOpt = System.getenv(ENV_OOZIE_AUTH);
+}
+if (commandLine.hasOption(DEBUG_OPTION)) {
+System.out.println(" Auth type : " + authOpt);
+}
 return authOpt;
 }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/b94bcf4d/docs/src/site/twiki/DG_CommandLineTool.twiki
--
diff --git a/docs/src/site/twiki/DG_CommandLineTool.twiki 
b/docs/src/site/twiki/DG_CommandLineTool.twiki
index 0748ff8..c13b78a 100644
--- a/docs/src/site/twiki/DG_CommandLineTool.twiki
+++ b/docs/src/site/twiki/DG_CommandLineTool.twiki
@@ -19,6 +19,7 @@ The =oozie= CLI interacts with Oozie via its WS API.
 usage:
   the env variable 'OOZIE_URL' is used as default value for the '-oozie' 
option
   the env variable 'OOZIE_TIMEZONE' is used as default value for the 
'-timezone' option
+  the env variable 'OOZIE_AUTH' is used as default value for the '-auth' 
option
   custom headers for Oozie web services can be specified using 
'-Dheader:NAME=VALUE'
 
   oozie help : display usage

http://git-wip-us.apache.org/repos/asf/oozie/blob/b94bcf4d/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index fe5bd8c..f97c3f8 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1751 Default authentication type using env variable for oozie CLI (puru 
via rohini)
 OOZIE-1753 Update create-release-artifact script for git (rkanter)
 OOZIE-1704 Add ability to use Bulk API with bundle ID (mona)
 OOZIE-1718 Coord Job Query UPDATE_COORD_JOB_CHANGE does not update last 
modified time (mona)



git commit: OOZIE-1549 Update hcat documentation to mention hcatalog-pig-adapter jar (bowenzhangusa via rohini)

2014-03-20 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/branch-4.0 aa8dba73e -> eeb2c7822


OOZIE-1549 Update hcat documentation to mention hcatalog-pig-adapter jar 
(bowenzhangusa via rohini)


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

Branch: refs/heads/branch-4.0
Commit: eeb2c78224705b63daea41668575b81dfde24d32
Parents: aa8dba7
Author: Rohini Palaniswamy 
Authored: Thu Mar 20 16:33:43 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Thu Mar 20 16:33:43 2014 -0700

--
 hcataloglibs/hcatalog-0.5/pom.xml | 6 ++
 1 file changed, 6 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/eeb2c782/hcataloglibs/hcatalog-0.5/pom.xml
--
diff --git a/hcataloglibs/hcatalog-0.5/pom.xml 
b/hcataloglibs/hcatalog-0.5/pom.xml
index c18cebc..41dad5c 100644
--- a/hcataloglibs/hcatalog-0.5/pom.xml
+++ b/hcataloglibs/hcatalog-0.5/pom.xml
@@ -193,6 +193,12 @@
 compile
 
 
+
+org.apache.hcatalog
+hcatalog-pig-adapter
+0.5.0-incubating
+
+
 
 
 



git commit: OOZIE-1549 Update hcat documentation to mention hcatalog-pig-adapter jar (bowenzhangusa via rohini)

2014-03-20 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master addcaaf4c -> 85919caab


OOZIE-1549 Update hcat documentation to mention hcatalog-pig-adapter jar 
(bowenzhangusa via rohini)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/85919caa
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/85919caa
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/85919caa

Branch: refs/heads/master
Commit: 85919caab4b5cf3231cfaaf7bf0622240f31735c
Parents: addcaaf
Author: Rohini Palaniswamy 
Authored: Thu Mar 20 16:33:07 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Thu Mar 20 16:33:07 2014 -0700

--
 hcataloglibs/hcatalog-0.5/pom.xml | 6 ++
 1 file changed, 6 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/85919caa/hcataloglibs/hcatalog-0.5/pom.xml
--
diff --git a/hcataloglibs/hcatalog-0.5/pom.xml 
b/hcataloglibs/hcatalog-0.5/pom.xml
index 457874d..2a65da9 100644
--- a/hcataloglibs/hcatalog-0.5/pom.xml
+++ b/hcataloglibs/hcatalog-0.5/pom.xml
@@ -193,6 +193,12 @@
 compile
 
 
+
+org.apache.hcatalog
+hcatalog-pig-adapter
+0.5.0-incubating
+
+
 
 
 



git commit: OOZIE-1693 UI timeout while loading job table (puru via rohini)

2014-03-17 Thread rohini
Repository: oozie
Updated Branches:
  refs/heads/master 776af774c -> addcaaf4c


OOZIE-1693 UI timeout while loading job table (puru via rohini)


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

Branch: refs/heads/master
Commit: addcaaf4c7e29cf833db332b59105cee77de748c
Parents: 776af77
Author: Rohini Palaniswamy 
Authored: Mon Mar 17 14:25:28 2014 -0700
Committer: Rohini Palaniswamy 
Committed: Mon Mar 17 14:25:28 2014 -0700

--
 release-log.txt | 1 +
 webapp/src/main/webapp/oozie-console.js | 4 
 2 files changed, 5 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/oozie/blob/addcaaf4/release-log.txt
--
diff --git a/release-log.txt b/release-log.txt
index 59a955e..807e6be 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1693 UI timeout while loading job table (puru via rohini)
 OOZIE-1698 Action sharelib configuration document lacks the "oozie." prefix 
(qwertymaniac via rohini)
 OOZIE-1712 Oozie page does not render in Internet Explorer 9 on Windows 
(omaliuvanchuk via rohini)
 OOZIE-1720 Oozie Hive action doesn't honour mapred.job.name (mithun via rohini)

http://git-wip-us.apache.org/repos/asf/oozie/blob/addcaaf4/webapp/src/main/webapp/oozie-console.js
--
diff --git a/webapp/src/main/webapp/oozie-console.js 
b/webapp/src/main/webapp/oozie-console.js
index b1cd612..2742485 100644
--- a/webapp/src/main/webapp/oozie-console.js
+++ b/webapp/src/main/webapp/oozie-console.js
@@ -1596,6 +1596,7 @@ var coord_jobs_store = new Ext.data.JsonStore({
 url: getOozieBase() + 'jobs'
 })
 });
+coord_jobs_store.proxy.conn.timeout = 30;
 coord_jobs_store.proxy.conn.method = "GET";
 
 /* 
@@ -1615,6 +1616,7 @@ var jobs_store = new Ext.data.JsonStore({
 url: getOozieBase() + 'jobs'
 })
 });
+jobs_store.proxy.conn.timeout = 30;
 jobs_store.proxy.conn.method = "GET";
 
 /* 
@@ -1636,6 +1638,7 @@ var bundle_jobs_store = new Ext.data.JsonStore({
 url: getOozieBase() + 'jobs'
 })
 });
+bundle_jobs_store.proxy.conn.timeout = 30;
 bundle_jobs_store.proxy.conn.method = "GET";
 
 var configGridData = new Ext.data.JsonStore({
@@ -2060,6 +2063,7 @@ var timeZones_store = new Ext.data.JsonStore({
 url: getOozieBase() + 'admin' + "/available-timezones"
 })
 });
+timeZones_store.proxy.conn.timeout = 30;
 timeZones_store.proxy.conn.method = "GET";
 
 function showCoordJobContextMenu(thisGrid, rowIndex, cellIndex, e) {



  1   2   >