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 2013/06/29 21:51:28 UTC

svn commit: r1497996 [2/2] - in /oozie/branches/branch-4.0: ./ core/src/main/java/org/apache/oozie/ core/src/main/java/org/apache/oozie/command/ core/src/main/java/org/apache/oozie/command/coord/ core/src/main/java/org/apache/oozie/command/wf/ core/src...

Modified: oozie/branches/branch-4.0/core/src/test/java/org/apache/oozie/sla/TestSLAService.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-4.0/core/src/test/java/org/apache/oozie/sla/TestSLAService.java?rev=1497996&r1=1497995&r2=1497996&view=diff
==============================================================================
--- oozie/branches/branch-4.0/core/src/test/java/org/apache/oozie/sla/TestSLAService.java (original)
+++ oozie/branches/branch-4.0/core/src/test/java/org/apache/oozie/sla/TestSLAService.java Sat Jun 29 18:30:15 2013
@@ -18,19 +18,27 @@
 package org.apache.oozie.sla;
 
 import java.util.Date;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.AppType;
+import org.apache.oozie.CoordinatorActionBean;
+import org.apache.oozie.WorkflowJobBean;
 import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.CoordinatorJob;
 import org.apache.oozie.client.WorkflowJob;
 import org.apache.oozie.client.event.JobEvent.EventStatus;
 import org.apache.oozie.client.event.SLAEvent;
+import org.apache.oozie.client.event.SLAEvent.SLAStatus;
+import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobInsertJPAExecutor;
+import org.apache.oozie.executor.jpa.WorkflowJobUpdateJPAExecutor;
+import org.apache.oozie.executor.jpa.sla.SLASummaryGetJPAExecutor;
 import org.apache.oozie.service.EventHandlerService;
+import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.sla.listener.SLAEventListener;
 import org.apache.oozie.sla.service.SLAService;
 import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.workflow.WorkflowInstance;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -48,6 +56,7 @@ public class TestSLAService extends XDat
         conf.set(Services.CONF_SERVICE_EXT_CLASSES, "org.apache.oozie.service.EventHandlerService,"
                 + "org.apache.oozie.sla.service.SLAService");
         conf.setClass(EventHandlerService.CONF_LISTENERS, DummySLAEventListener.class, SLAEventListener.class);
+        conf.setLong(SLAService.CONF_JOB_EVENT_LATENCY, 0);
         services.init();
         cleanUpDBTables();
     }
@@ -118,7 +127,8 @@ public class TestSLAService extends XDat
         output.setLength(0);
 
         // test same job multiple events (start-miss, end-miss) through regular check
-        sla2 = _createSLARegistration("job-4", AppType.WORKFLOW_JOB);
+        WorkflowJobBean job4 = addRecordToWfJobTable(WorkflowJob.Status.KILLED, WorkflowInstance.Status.KILLED);
+        sla2 = _createSLARegistration(job4.getId(), AppType.WORKFLOW_JOB);
         sla2.setExpectedStart(new Date(System.currentTimeMillis() - 2 * 3600 * 1000)); //2 hours back
         sla2.setExpectedEnd(new Date(System.currentTimeMillis() - 1 * 3600 * 1000)); //1 hour back
         slas.addRegistrationEvent(sla2);
@@ -147,9 +157,130 @@ public class TestSLAService extends XDat
         assertTrue(output.toString().contains(sla1.getId() + " Sla END - MET!!!"));
     }
 
-    private SLARegistrationBean _createSLARegistration(String jobId, AppType appType) {
+    @Test
+    public void testEndMissDBConfirm() throws Exception {
+        SLAService slas = Services.get().get(SLAService.class);
+        EventHandlerService ehs = Services.get().get(EventHandlerService.class);
+        JPAService jpaService = Services.get().get(JPAService.class);
+
+        // CASE 1: positive test WF job
+        WorkflowJobBean job1 = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        SLARegistrationBean sla = _createSLARegistration(job1.getId(), AppType.WORKFLOW_JOB);
+        sla.setExpectedEnd(new Date(System.currentTimeMillis() - 1 * 1800 * 1000)); // half hour back
+        slas.addRegistrationEvent(sla);
+
+        // CASE 2: negative test WF job
+        WorkflowJobBean job2 = addRecordToWfJobTable(WorkflowJob.Status.SUCCEEDED, WorkflowInstance.Status.SUCCEEDED);
+        job2.setEndTime(new Date(System.currentTimeMillis() - 1 * 1800 * 1000));
+        job2.setStartTime(new Date(System.currentTimeMillis() - 1 * 2000 * 1000));
+        jpaService.execute(new WorkflowJobUpdateJPAExecutor(job2));
+        sla = _createSLARegistration(job2.getId(), AppType.WORKFLOW_JOB);
+        sla.setExpectedEnd(new Date(System.currentTimeMillis() - 1 * 1500 * 1000)); // in past but > actual end
+        sla.setExpectedDuration(100); //unreasonable to cause MISS
+        slas.addRegistrationEvent(sla);
+
+        // CASE 3: positive test Coord action
+        CoordinatorActionBean action1 = addRecordToCoordActionTable("coord-action-1", 1,
+                CoordinatorAction.Status.WAITING, "coord-action-get.xml", 0);
+        WorkflowJobBean extWf = new WorkflowJobBean();
+        extWf.setId(action1.getExternalId());
+        extWf.setEndTime(new Date(System.currentTimeMillis() - 1 * 1800 * 1000));
+        extWf.setStartTime(new Date(System.currentTimeMillis() - 1 * 2100 * 1000));
+        jpaService.execute(new WorkflowJobInsertJPAExecutor(extWf));
+        sla = _createSLARegistration(action1.getId(), AppType.COORDINATOR_ACTION);
+        sla.setExpectedEnd(new Date(System.currentTimeMillis() - 1 * 2000 * 1000)); // past
+        slas.addRegistrationEvent(sla);
+
+        // CASE 4: positive test coord action
+        CoordinatorActionBean action2 = addRecordToCoordActionTable("coord-action-2", 1,
+                CoordinatorAction.Status.FAILED, "coord-action-get.xml", 0);
+        extWf = new WorkflowJobBean();
+        extWf.setId(action2.getExternalId());
+        // actual end before expected. but action is failed
+        extWf.setEndTime(new Date(System.currentTimeMillis() - 1 * 1800 * 1000));
+        extWf.setStartTime(new Date(System.currentTimeMillis() - 1 * 2000 * 1000));
+        jpaService.execute(new WorkflowJobInsertJPAExecutor(extWf));
+        sla = _createSLARegistration(action2.getId(), AppType.COORDINATOR_ACTION);
+        sla.setExpectedEnd(new Date(System.currentTimeMillis() - 1 * 1500 * 1000));
+        slas.addRegistrationEvent(sla);
+
+        // CASE 5: negative test coord action
+        CoordinatorActionBean action3 = addRecordToCoordActionTable("coord-action-3", 1,
+                CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
+        extWf = new WorkflowJobBean();
+        extWf.setId(action3.getExternalId());
+        extWf.setEndTime(new Date(System.currentTimeMillis() - 1 * 1800 * 1000));
+        extWf.setStartTime(new Date(System.currentTimeMillis() - 1 * 2100 * 1000));
+        jpaService.execute(new WorkflowJobInsertJPAExecutor(extWf));
+        sla = _createSLARegistration(action3.getId(), AppType.COORDINATOR_ACTION);
+        sla.setExpectedEnd(new Date(System.currentTimeMillis() - 1 * 1500 * 1000)); // in past but > actual end
+        slas.addRegistrationEvent(sla);
+
+        slas.runSLAWorker();
+        ehs.new EventWorker().run();
+        int count = 0;
+        for (int ptr = output.indexOf("END - MISS"); ptr < output.length() && ptr > 0; ptr = output.indexOf(
+                "END - MISS", ptr + 1)) {
+            count++;
+        }
+        assertEquals(3, count); // only 3 out of the 5 are correct end_misses
+        assertTrue(output.toString().contains(job1.getId() + " Sla END - MISS!!!"));
+        assertTrue(output.toString().contains(action1.getId() + " Sla END - MISS!!!"));
+        assertTrue(output.toString().contains(action2.getId() + " Sla END - MISS!!!"));
+        assertTrue(output.toString().contains(job2.getId() + " Sla END - MET!!!"));
+        assertTrue(output.toString().contains(job2.getId() + " Sla DURATION - MISS!!!"));
+        assertTrue(output.toString().contains(action3.getId() + " Sla END - MET!!!"));
+
+        // negative on MISS after DB check, updated with actual times
+        SLASummaryBean slaSummary = jpaService.execute(new SLASummaryGetJPAExecutor(job2.getId()));
+        assertEquals(job2.getStartTime(), slaSummary.getActualStart());
+        assertEquals(job2.getEndTime(), slaSummary.getActualEnd());
+        assertEquals(job2.getEndTime().getTime() - job2.getStartTime().getTime(), slaSummary.getActualDuration());
+        assertEquals(job2.getStatusStr(), slaSummary.getJobStatus());
+        assertEquals(SLAEvent.EventStatus.END_MET, slaSummary.getEventStatus());
+        assertEquals(SLAStatus.MET, slaSummary.getSLAStatus());
+        assertEquals(8, slaSummary.getEventProcessed());
+        assertNull(slas.getSLACalculator().get(job2.getId())); //removed from memory
+
+        // positives but also updated with actual times immediately after DB check
+        slaSummary = jpaService.execute(new SLASummaryGetJPAExecutor(action2.getId()));
+        extWf = jpaService.execute(new WorkflowJobGetJPAExecutor(action2.getExternalId()));
+        assertEquals(extWf.getStartTime(), slaSummary.getActualStart());
+        assertEquals(extWf.getEndTime(), slaSummary.getActualEnd());
+        assertEquals(extWf.getEndTime().getTime() - extWf.getStartTime().getTime(), slaSummary.getActualDuration());
+        assertEquals(action2.getStatusStr(), slaSummary.getJobStatus());
+        assertEquals(SLAEvent.EventStatus.END_MISS, slaSummary.getEventStatus());
+        assertEquals(SLAStatus.MISS, slaSummary.getSLAStatus());
+        assertEquals(8, slaSummary.getEventProcessed());
+        assertNull(slas.getSLACalculator().get(action2.getId())); //removed from memory
+
+        slaSummary = jpaService.execute(new SLASummaryGetJPAExecutor(action1.getId()));
+        extWf = jpaService.execute(new WorkflowJobGetJPAExecutor(action1.getExternalId()));
+        assertEquals(extWf.getStartTime(), slaSummary.getActualStart());
+        assertEquals(extWf.getEndTime(), slaSummary.getActualEnd());
+        assertEquals(extWf.getEndTime().getTime() - extWf.getStartTime().getTime(), slaSummary.getActualDuration());
+        assertEquals(action1.getStatusStr(), slaSummary.getJobStatus());
+        assertEquals(SLAEvent.EventStatus.END_MISS, slaSummary.getEventStatus());
+        assertEquals(SLAStatus.MISS, slaSummary.getSLAStatus());
+        assertEquals(8, slaSummary.getEventProcessed());
+        assertNull(slas.getSLACalculator().get(action1.getId())); //removed from memory
+
+        //testing rogue (Does not exist in DB) job miss and sla map update
+        sla = _createSLARegistration("rogue", AppType.WORKFLOW_ACTION);
+        sla.setExpectedEnd(new Date(System.currentTimeMillis() - 1 * 1800 * 1000));
+        slas.addRegistrationEvent(sla);
+        int prevSize = slas.getSLACalculator().size();
+        slas.runSLAWorker();
+        ehs.new EventWorker().run();
+        assertTrue(output.toString().contains("rogue Sla END - MISS!!!"));
+        assertEquals(prevSize - 1, slas.getSLACalculator().size());
+        assertNull(slas.getSLACalculator().get(sla.getId())); //not lying around in history set either
+
+    }
+
+    static SLARegistrationBean _createSLARegistration(String jobId, AppType appType) {
         SLARegistrationBean bean = new SLARegistrationBean();
-        bean.setJobId(jobId);
+        bean.setId(jobId);
         bean.setAppType(appType);
         return bean;
     }

Modified: oozie/branches/branch-4.0/core/src/test/java/org/apache/oozie/sla/TestSLASummaryGetOnRestartJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/branches/branch-4.0/core/src/test/java/org/apache/oozie/sla/TestSLASummaryGetOnRestartJPAExecutor.java?rev=1497996&r1=1497995&r2=1497996&view=diff
==============================================================================
--- oozie/branches/branch-4.0/core/src/test/java/org/apache/oozie/sla/TestSLASummaryGetOnRestartJPAExecutor.java (original)
+++ oozie/branches/branch-4.0/core/src/test/java/org/apache/oozie/sla/TestSLASummaryGetOnRestartJPAExecutor.java Sat Jun 29 18:30:15 2013
@@ -55,7 +55,7 @@ public class TestSLASummaryGetOnRestartJ
         JPAService jpaService = Services.get().get(JPAService.class);
         SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd");
         SLASummaryBean sla1 = new SLASummaryBean();
-        sla1.setJobId("jobId");
+        sla1.setId("jobId");
         sla1.setAppName("appName");
         sla1.setUser("user");
         sla1.setParentId("parent");
@@ -64,7 +64,7 @@ public class TestSLASummaryGetOnRestartJ
         sla1.setLastModifiedTime(new Date(System.currentTimeMillis() - 5*24*60*60*1000));
 
         SLASummaryBean sla2 = new SLASummaryBean();
-        sla2.setJobId("jobId2");
+        sla2.setId("jobId2");
         sla2.setEventProcessed(6);
         // set to long time back
         sla2.setLastModifiedTime(sdf.parse("2009-06-03"));
@@ -78,7 +78,7 @@ public class TestSLASummaryGetOnRestartJ
         SLASummaryGetRecordsOnRestartJPAExecutor slaGetOnRestart = new SLASummaryGetRecordsOnRestartJPAExecutor(7);
         List<SLASummaryBean> beans = jpaService.execute(slaGetOnRestart);
         assertEquals(1, beans.size());
-        assertEquals("jobId", beans.get(0).getJobId());
+        assertEquals("jobId", beans.get(0).getId());
         assertEquals("appName", beans.get(0).getAppName());
         assertEquals("user", beans.get(0).getUser());
         assertEquals("parent", beans.get(0).getParentId());

Added: oozie/branches/branch-4.0/core/src/test/resources/coord-action-sla1.xml
URL: http://svn.apache.org/viewvc/oozie/branches/branch-4.0/core/src/test/resources/coord-action-sla1.xml?rev=1497996&view=auto
==============================================================================
--- oozie/branches/branch-4.0/core/src/test/resources/coord-action-sla1.xml (added)
+++ oozie/branches/branch-4.0/core/src/test/resources/coord-action-sla1.xml Sat Jun 29 18:30:15 2013
@@ -0,0 +1,37 @@
+<!--
+  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.
+-->
+<coordinator-app name="test-coord-sla" frequency="${coord:days(1)}"
+                 action-nominal-time="2009-01-02T08:01Z" end="2010-01-01T08:01Z"
+                 timezone="America/Los_Angeles"
+                 xmlns="uri:oozie:coordinator:0.4"
+                 xmlns:sla="uri:oozie:sla:0.2">
+    <action>
+        <workflow>
+            <app-path>${wfAppPath}</app-path>
+        </workflow>
+        <sla:info>
+            <sla:nominal-time>${nominal_time}</sla:nominal-time>
+            <sla:should-start>${10 * MINUTES}</sla:should-start>
+            <sla:should-end>${30 * MINUTES}</sla:should-end>
+            <sla:max-duration>${30 * MINUTES}</sla:max-duration>
+            <sla:alert-events>start_miss,end_met,end_miss</sla:alert-events>
+            <sla:alert-contact>joe@example.com</sla:alert-contact>
+        </sla:info>
+    </action>
+</coordinator-app>
+

Modified: oozie/branches/branch-4.0/release-log.txt
URL: http://svn.apache.org/viewvc/oozie/branches/branch-4.0/release-log.txt?rev=1497996&r1=1497995&r2=1497996&view=diff
==============================================================================
--- oozie/branches/branch-4.0/release-log.txt (original)
+++ oozie/branches/branch-4.0/release-log.txt Sat Jun 29 18:30:15 2013
@@ -1,5 +1,6 @@
 -- Oozie 4.0.0 release
 
+OOZIE-1379 Generate SLA end_miss event only after confirming against persistent store (mona)
 OOZIE-1439 Job materialization happening even after coordinator is supposed to be killed (virag)
 OOZIE-1438 parentID is null in job event message of subworkflow action (ryota)
 OOZIE-1435 StatusTransitService unnecessarily updates the lastModifiedTime of jobs which causes MaterializationService to bring same jobs in memory (virag)