You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by mo...@apache.org on 2014/03/21 00:34:00 UTC

[1/2] git commit: OOZIE-1718 Coord Job Query UPDATE_COORD_JOB_CHANGE does not update last modified time (mona)

Repository: oozie
Updated Branches:
  refs/heads/master 85919caab -> e7f002073


OOZIE-1718 Coord Job Query UPDATE_COORD_JOB_CHANGE does not update last modified time (mona)


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

Branch: refs/heads/master
Commit: cb9cbe8b2a3a5acccc9deaa0ef5826a671d9e5a2
Parents: addcaaf
Author: Mona Chitnis <ch...@yahoo-inc.com>
Authored: Thu Mar 20 16:33:28 2014 -0700
Committer: Mona Chitnis <ch...@yahoo-inc.com>
Committed: Thu Mar 20 16:33:28 2014 -0700

----------------------------------------------------------------------
 .../org/apache/oozie/CoordinatorJobBean.java    |  2 +-
 .../oozie/command/coord/CoordKillXCommand.java  |  1 +
 .../executor/jpa/CoordJobQueryExecutor.java     |  1 +
 .../oozie/service/PauseTransitService.java      |  9 ++-
 .../bundle/TestBundleChangeXCommand.java        | 63 +++++++++++++++++++-
 .../command/coord/TestCoordKillXCommand.java    |  3 +
 .../executor/jpa/TestCoordJobQueryExecutor.java | 10 ++++
 release-log.txt                                 |  1 +
 8 files changed, 84 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/cb9cbe8b/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 95f6991..5eb134b 100644
--- a/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
+++ b/core/src/main/java/org/apache/oozie/CoordinatorJobBean.java
@@ -75,7 +75,7 @@ import org.json.simple.JSONObject;
 
         @NamedQuery(name = "UPDATE_COORD_JOB_MATERIALIZE", query = "update CoordinatorJobBean w set w.statusStr = :status, w.pending = :pending, w.doneMaterialization = :doneMaterialization, w.lastActionTimestamp = :lastActionTime, w.lastActionNumber = :lastActionNumber, w.nextMaterializedTimestamp = :nextMatdTime where w.id = :id"),
 
-        @NamedQuery(name = "UPDATE_COORD_JOB_CHANGE", query = "update CoordinatorJobBean w set w.endTimestamp = :endTime, w.statusStr = :status, w.pending = :pending, w.doneMaterialization = :doneMaterialization, w.concurrency = :concurrency, w.pauseTimestamp = :pauseTime, w.lastActionNumber = :lastActionNumber, w.lastActionTimestamp = :lastActionTime, w.nextMaterializedTimestamp = :nextMatdTime where w.id = :id"),
+        @NamedQuery(name = "UPDATE_COORD_JOB_CHANGE", query = "update CoordinatorJobBean w set w.endTimestamp = :endTime, w.statusStr = :status, w.pending = :pending, w.doneMaterialization = :doneMaterialization, w.concurrency = :concurrency, w.pauseTimestamp = :pauseTime, w.lastActionNumber = :lastActionNumber, w.lastActionTimestamp = :lastActionTime, w.nextMaterializedTimestamp = :nextMatdTime, w.lastModifiedTimestamp = :lastModifiedTime where w.id = :id"),
 
         @NamedQuery(name = "DELETE_COORD_JOB", query = "delete from CoordinatorJobBean w where w.id = :id"),
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/cb9cbe8b/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java b/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java
index 5d5df86..2e5f6b4 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordKillXCommand.java
@@ -157,6 +157,7 @@ public class CoordKillXCommand extends KillTransitionXCommand {
             }
         }
         coordJob.setDoneMaterialization();
+        coordJob.setLastModifiedTime(new Date());
         LOG.debug("Killed coord actions for the coordinator=[{0}]", jobId);
     }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/cb9cbe8b/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobQueryExecutor.java b/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobQueryExecutor.java
index 240b352..5f7744c 100644
--- a/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobQueryExecutor.java
+++ b/core/src/main/java/org/apache/oozie/executor/jpa/CoordJobQueryExecutor.java
@@ -179,6 +179,7 @@ public class CoordJobQueryExecutor extends QueryExecutor<CoordinatorJobBean, Coo
                 query.setParameter("lastActionNumber", cjBean.getLastActionNumber());
                 query.setParameter("lastActionTime", cjBean.getLastActionTimestamp());
                 query.setParameter("nextMatdTime", cjBean.getNextMaterializedTimestamp());
+                query.setParameter("lastModifiedTime", cjBean.getLastModifiedTimestamp());
                 query.setParameter("id", cjBean.getId());
                 break;
             default:

http://git-wip-us.apache.org/repos/asf/oozie/blob/cb9cbe8b/core/src/main/java/org/apache/oozie/service/PauseTransitService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/service/PauseTransitService.java b/core/src/main/java/org/apache/oozie/service/PauseTransitService.java
index f479f23..e1bfa2b 100644
--- a/core/src/main/java/org/apache/oozie/service/PauseTransitService.java
+++ b/core/src/main/java/org/apache/oozie/service/PauseTransitService.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *      http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -39,6 +39,8 @@ import org.apache.oozie.service.Services;
 import org.apache.oozie.lock.LockToken;
 import org.apache.oozie.util.XLog;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * PauseTransitService is the runnable which is scheduled to run at the configured interval, it checks all bundles
  * to see if they should be paused, un-paused or started.
@@ -52,7 +54,8 @@ public class PauseTransitService implements Service {
      * PauseTransitRunnable is the runnable which is scheduled to run at the configured interval, it checks all
      * bundles to see if they should be paused, un-paused or started.
      */
-    static class PauseTransitRunnable implements Runnable {
+    @VisibleForTesting
+    public static class PauseTransitRunnable implements Runnable {
         private JPAService jpaService = null;
         private LockToken lock;
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/cb9cbe8b/core/src/test/java/org/apache/oozie/command/bundle/TestBundleChangeXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/bundle/TestBundleChangeXCommand.java b/core/src/test/java/org/apache/oozie/command/bundle/TestBundleChangeXCommand.java
index ea07cb8..6560db6 100644
--- a/core/src/test/java/org/apache/oozie/command/bundle/TestBundleChangeXCommand.java
+++ b/core/src/test/java/org/apache/oozie/command/bundle/TestBundleChangeXCommand.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *      http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -17,20 +17,28 @@
  */
 package org.apache.oozie.command.bundle;
 
+import java.util.Date;
+
 import org.apache.oozie.BundleActionBean;
 import org.apache.oozie.BundleJobBean;
 import org.apache.oozie.CoordinatorJobBean;
 import org.apache.oozie.ErrorCode;
 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.executor.jpa.BundleActionInsertJPAExecutor;
+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.BundleJobQueryExecutor;
+import org.apache.oozie.executor.jpa.BundleJobQueryExecutor.BundleJobQuery;
 import org.apache.oozie.executor.jpa.CoordJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor;
 import org.apache.oozie.executor.jpa.CoordJobQueryExecutor.CoordJobQuery;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
+import org.apache.oozie.service.PauseTransitService.PauseTransitRunnable;
 import org.apache.oozie.test.XDataTestCase;
 import org.apache.oozie.util.DateUtils;
 
@@ -182,4 +190,55 @@ public class TestBundleChangeXCommand extends XDataTestCase {
         job = jpaService.execute(bundleJobGetCmd);
         assertEquals(job.getEndTime(), DateUtils.parseDateOozieTZ(dateStr));
     }
+
+    public void testBundlePauseExtendMaterializesCoordinator() throws Exception {
+        BundleJobBean bundle = this.addRecordToBundleJobTable(Job.Status.PAUSED, false);
+        Date startTime = new Date();
+        Date endTime = new Date(startTime.getTime() + (20 * 60 * 1000));
+        // coord job with num actions materialized = 1
+        CoordinatorJobBean coord = addRecordToCoordJobTable(CoordinatorJob.Status.PAUSED, startTime, endTime, true,
+                false, 1);
+        coord.setPauseTime(startTime); // setting dummy old pause time
+        CoordJobQueryExecutor.getInstance().executeUpdate(CoordJobQuery.UPDATE_COORD_JOB_CHANGE, coord);
+        // persist corresponding bundle action bean
+        BundleActionBean bundleAction = this.addRecordToBundleActionTable(bundle.getId(), "COORD-TEST", 1,
+                Job.Status.PAUSED);
+
+        coord.setBundleId(bundle.getId());
+        CoordJobQueryExecutor.getInstance().executeUpdate(CoordJobQuery.UPDATE_COORD_JOB_BUNDLEID, coord);
+        bundleAction.setCoordId(coord.getId());
+        BundleActionQueryExecutor.getInstance().executeUpdate(
+                BundleActionQuery.UPDATE_BUNDLE_ACTION_STATUS_PENDING_MODTIME_COORDID, bundleAction);
+
+        Date later = new Date(System.currentTimeMillis() + 20 * 1000); // 20 sec later
+        Date evenLater = new Date(later.getTime() + 10 * 60 * 1000); // 10 min later
+
+        bundle = BundleJobQueryExecutor.getInstance().get(BundleJobQuery.GET_BUNDLE_JOB, bundle.getId());
+        assertEquals(Job.Status.PAUSED, bundle.getStatus());
+        coord = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB, coord.getId());
+        assertEquals(Job.Status.PAUSED, coord.getStatus());
+
+        // coordinator has previous actions
+        coord.setNextMaterializedTime(new Date(startTime.getTime() + (300 * 1000))); // before the new pausetime
+        CoordJobQueryExecutor.getInstance().executeUpdate(CoordJobQuery.UPDATE_COORD_JOB, coord);
+        Date lastMod = coord.getLastModifiedTime();
+        addRecordToCoordActionTable(coord.getId(), 1, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
+
+        // change pausetime to even later
+        new BundleJobChangeXCommand(bundle.getId(), "pausetime=" + DateUtils.formatDateOozieTZ(evenLater)).call();
+        sleep(1000); // time for the queued CoordChangeXCommand to complete
+
+        Runnable runnable = new PauseTransitRunnable();
+        runnable.run(); // simulating run which would happen usually after interval
+        Thread.sleep(1000);
+
+        bundle = BundleJobQueryExecutor.getInstance().get(BundleJobQuery.GET_BUNDLE_JOB, bundle.getId());
+        assertEquals(Job.Status.RUNNING, bundle.getStatus());
+        assertEquals(DateUtils.formatDateOozieTZ(evenLater), DateUtils.formatDateOozieTZ(bundle.getPauseTime()));
+        coord = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB, coord.getId());
+        assertEquals(Job.Status.RUNNING, coord.getStatus());
+        assertEquals(DateUtils.formatDateOozieTZ(evenLater), DateUtils.formatDateOozieTZ(coord.getPauseTime()));
+        assertTrue(coord.getLastModifiedTime().after(lastMod));
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/cb9cbe8b/core/src/test/java/org/apache/oozie/command/coord/TestCoordKillXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/coord/TestCoordKillXCommand.java b/core/src/test/java/org/apache/oozie/command/coord/TestCoordKillXCommand.java
index 218185a..a03ed9a 100644
--- a/core/src/test/java/org/apache/oozie/command/coord/TestCoordKillXCommand.java
+++ b/core/src/test/java/org/apache/oozie/command/coord/TestCoordKillXCommand.java
@@ -95,6 +95,7 @@ public class TestCoordKillXCommand extends XDataTestCase {
         action = jpaService.execute(coordActionGetCmd);
         assertEquals(job.getStatus(), CoordinatorJob.Status.KILLED);
         assertTrue(job.isDoneMaterialization());
+        assertNotNull(job.getLastModifiedTime());
         assertEquals(action.getStatus(), CoordinatorAction.Status.KILLED);
 
         // Change job status to RUNNINGWITHERROR to simulate StatusTransitService changing it to
@@ -119,6 +120,7 @@ public class TestCoordKillXCommand extends XDataTestCase {
         job = jpaService.execute(coordJobGetCmd);
         assertEquals(job.getStatus(), CoordinatorJob.Status.KILLED);
         assertTrue(job.isDoneMaterialization());
+        assertNotNull(job.getLastModifiedTime());
     }
 
     /**
@@ -149,6 +151,7 @@ public class TestCoordKillXCommand extends XDataTestCase {
         job = jpaService.execute(coordJobGetCmd);
         action = jpaService.execute(coordActionGetCmd);
         assertEquals(job.getStatus(), CoordinatorJob.Status.KILLED);
+        assertNotNull(job.getLastModifiedTime());
         assertEquals(action.getStatus(), CoordinatorAction.Status.KILLED);
     }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/cb9cbe8b/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobQueryExecutor.java b/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobQueryExecutor.java
index 1e75bbd..9427db0 100644
--- a/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobQueryExecutor.java
+++ b/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobQueryExecutor.java
@@ -152,6 +152,7 @@ public class TestCoordJobQueryExecutor extends XDataTestCase {
         assertEquals(query.getParameterValue("lastActionNumber"), cjBean.getLastActionNumber());
         assertEquals(query.getParameterValue("lastActionTime"), cjBean.getLastActionTimestamp());
         assertEquals(query.getParameterValue("nextMatdTime"), cjBean.getNextMaterializedTimestamp());
+        assertEquals(query.getParameterValue("lastModifiedTime"), cjBean.getLastModifiedTimestamp());
         assertEquals(query.getParameterValue("id"), cjBean.getId());
         em.close();
     }
@@ -168,6 +169,15 @@ public class TestCoordJobQueryExecutor extends XDataTestCase {
         CoordinatorJobBean job2 = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB, job1.getId());
         assertEquals(job2.getStatus(), CoordinatorJob.Status.SUCCEEDED);
 
+        CoordinatorJobBean job3 = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, false);
+        Date initialLMT = job3.getLastModifiedTime();
+        job3.setLastModifiedTime(new Date()); // similar to what's done by e.g. the change command
+        CoordJobQueryExecutor.getInstance().executeUpdate(CoordJobQuery.UPDATE_COORD_JOB_CHANGE, job3);
+        job3 = CoordJobQueryExecutor.getInstance().get(CoordJobQuery.GET_COORD_JOB, job3.getId());
+        Date afterChangeLMT = job3.getLastModifiedTime();
+        assertNotNull(job3.getLastModifiedTimestamp());
+        assertTrue(afterChangeLMT.after(initialLMT));
+
     }
 
     public void testGet() throws Exception {

http://git-wip-us.apache.org/repos/asf/oozie/blob/cb9cbe8b/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 807e6be..5b3a2da 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1718 Coord Job Query UPDATE_COORD_JOB_CHANGE does not update last modified time (mona)
 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)


[2/2] git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/oozie

Posted by mo...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/oozie


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

Branch: refs/heads/master
Commit: e7f0020731385bbd58683941002003fe0cabfa96
Parents: cb9cbe8 85919ca
Author: Mona Chitnis <ch...@yahoo-inc.com>
Authored: Thu Mar 20 16:33:45 2014 -0700
Committer: Mona Chitnis <ch...@yahoo-inc.com>
Committed: Thu Mar 20 16:33:45 2014 -0700

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