You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by an...@apache.org on 2012/01/06 22:58:27 UTC

svn commit: r1228451 - in /incubator/oozie/branches/3.1/core/src: main/java/org/apache/oozie/util/CoordActionsInDateRange.java test/java/org/apache/oozie/util/TestCoordActionsInDateRange.java

Author: angeloh
Date: Fri Jan  6 21:58:26 2012
New Revision: 1228451

URL: http://svn.apache.org/viewvc?rev=1228451&view=rev
Log:
OOZIE-553 Ability to view log for coordinator actions that ran in a date range.(Kiran via Mona via Mohammad)

Added:
    incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/util/CoordActionsInDateRange.java
    incubator/oozie/branches/3.1/core/src/test/java/org/apache/oozie/util/TestCoordActionsInDateRange.java

Added: incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/util/CoordActionsInDateRange.java
URL: http://svn.apache.org/viewvc/incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/util/CoordActionsInDateRange.java?rev=1228451&view=auto
==============================================================================
--- incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/util/CoordActionsInDateRange.java (added)
+++ incubator/oozie/branches/3.1/core/src/main/java/org/apache/oozie/util/CoordActionsInDateRange.java Fri Jan  6 21:58:26 2012
@@ -0,0 +1,103 @@
+/**
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.oozie.CoordinatorActionBean;
+import org.apache.oozie.CoordinatorActionInfo;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.XException;
+import org.apache.oozie.command.PreconditionException;
+import org.apache.oozie.executor.jpa.CoordJobGetActionsForDatesJPAExecutor;
+import org.apache.oozie.executor.jpa.JPAExecutorException;
+import org.apache.oozie.service.JPAService;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.util.DateUtils;
+import org.apache.oozie.util.ParamChecker;
+/**
+ * This class provides the utility of listing
+ * coordinator actions that were executed between a certain
+ * date range. This is helpful in turn for retrieving the
+ * required logs in that date range.
+ */
+public class CoordActionsInDateRange {
+
+    /**
+     * Get the list of actions for given date ranges
+     *
+     * @param jobId coordinator job id
+     * @param scope the date range for log. format is comma-separated list of date ranges. Each date range element is specified with two dates separated by '::'
+     * @return the list of coordinator actions for the date range
+     *
+     * Internally involves a database operation by invoking method 'getActionIdsFromDateRange'.
+     */
+    public static List<CoordinatorActionBean> getCoordActionsFromDates(String jobId, String scope) throws XException,Exception {
+        ParamChecker.notEmpty(jobId, "jobId");
+        ParamChecker.notEmpty(scope, "scope");
+        Set<CoordinatorActionBean> actionSet = new HashSet<CoordinatorActionBean>();
+        String[] list = scope.split(",");
+        for (String s : list) {
+            s = s.trim();
+            // This block checks for errors in the format of specifying date range
+            if (s.contains("::")) {
+                String[] dateRange = s.split("::");
+                if (dateRange.length != 2) {
+                    throw new XException(ErrorCode.E0302, "Error in parsing date's range '" + s + "'. Date value expected on both sides of the scope resolution operator '::' to signify start and end of range");
+                }
+                Date start;
+                Date end;
+                start = DateUtils.parseDateUTC(dateRange[0].trim());
+                end = DateUtils.parseDateUTC(dateRange[1].trim());
+                if (start.after(end)) {
+                    throw new XException(ErrorCode.E0302, "Error in parsing date's range '" + s + "'. Start date '" + start + "' is older than end date: '" + end + "'");
+                }
+                List<CoordinatorActionBean> listOfActions = getActionIdsFromDateRange(jobId, start, end);
+                actionSet.addAll(listOfActions);
+            }
+            else {
+                throw new XException(ErrorCode.E0302, "Error in parsing date's range '" + s + "'. Separator '::' is missing for start and end dates of range");
+            }
+        }
+        List<CoordinatorActionBean> coordActions = new ArrayList<CoordinatorActionBean>();
+        for (CoordinatorActionBean coordAction : actionSet) {
+            coordActions.add(coordAction);
+        }
+        return coordActions;
+    }
+
+    /*
+     * Get coordinator action ids between given start and end time
+     *
+     * @param jobId coordinator job id
+     * @param start start time
+     * @param end end time
+     * @return a list of coordinator actions that correspond to the date range
+     */
+    private static List<CoordinatorActionBean> getActionIdsFromDateRange(String jobId, Date start, Date end) throws XException{
+        List<CoordinatorActionBean> list;
+        JPAService jpaService = Services.get().get(JPAService.class);
+        list = jpaService.execute(new CoordJobGetActionsForDatesJPAExecutor(jobId, start, end));
+        return list;
+    }
+}

Added: incubator/oozie/branches/3.1/core/src/test/java/org/apache/oozie/util/TestCoordActionsInDateRange.java
URL: http://svn.apache.org/viewvc/incubator/oozie/branches/3.1/core/src/test/java/org/apache/oozie/util/TestCoordActionsInDateRange.java?rev=1228451&view=auto
==============================================================================
--- incubator/oozie/branches/3.1/core/src/test/java/org/apache/oozie/util/TestCoordActionsInDateRange.java (added)
+++ incubator/oozie/branches/3.1/core/src/test/java/org/apache/oozie/util/TestCoordActionsInDateRange.java Fri Jan  6 21:58:26 2012
@@ -0,0 +1,85 @@
+/**
+ * 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.util;
+
+import java.util.Date;
+
+import org.apache.oozie.CoordinatorActionBean;
+import org.apache.oozie.CoordinatorJobBean;
+import org.apache.oozie.client.CoordinatorAction;
+import org.apache.oozie.client.CoordinatorJob;
+import org.apache.oozie.command.CommandException;
+import org.apache.oozie.local.LocalOozie;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.util.DateUtils;
+
+public class TestCoordActionsInDateRange extends XDataTestCase {
+
+    private Services services;
+
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+        services = new Services();
+        services.init();
+        cleanUpDBTables();
+        LocalOozie.start();
+    }
+
+    @Override
+    protected void tearDown() throws Exception {
+        LocalOozie.stop();
+        services.destroy();
+        super.tearDown();
+    }
+
+    /**
+     * This is unit test case for the 'getCoordActionsFromDates()' method. The method is supposed to retrieve the list of
+     * coordinator actions running between a range of start and end date. The following test case tests its accuracy and fails
+     * otherwise.
+     */
+
+    public void testCoordActionsInDateRange() {
+        try {
+            int actionNum = 1;
+            CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, false, false);
+            CoordinatorActionBean actionId1 = addRecordToCoordActionTable(job.getId(), actionNum,
+                    CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
+            Date nominalTime = actionId1.getNominalTime();
+            long nominalTimeMilliseconds = nominalTime.getTime();
+            long noOfMillisecondsinOneHour = 3600000;
+
+            // Testing for the number of coordinator actions in a date range that spans from half an hour prior to the nominal time to 1 hour after the nominal time
+            String date1 = DateUtils.formatDateUTC(new Date(nominalTimeMilliseconds - noOfMillisecondsinOneHour / 2));
+            String date2 = DateUtils.formatDateUTC(new Date(nominalTimeMilliseconds + noOfMillisecondsinOneHour));
+            int noOfActions = CoordActionsInDateRange.getCoordActionsFromDates(job.getId().toString(), date1 + "::" + date2).size();
+            assertEquals(1, noOfActions);
+
+            // Testing for the number of coordinator actions in a date range that spans from half an hour after the nominal time to 1 hour after the nominal time
+            date1 = DateUtils.formatDateUTC(new Date(nominalTimeMilliseconds + noOfMillisecondsinOneHour / 2));
+            noOfActions = CoordActionsInDateRange.getCoordActionsFromDates(job.getId().toString(), date1 + "::" + date2).size();
+            assertEquals(0, noOfActions);
+        }
+        catch (Exception e) {
+            e.printStackTrace();
+            fail();
+        }
+    }
+}