001    /**
002     * Copyright (c) 2010 Yahoo! Inc. All rights reserved.
003     * Licensed under the Apache License, Version 2.0 (the "License");
004     * you may not use this file except in compliance with the License.
005     * You may obtain a copy of the License at
006     *
007     *   http://www.apache.org/licenses/LICENSE-2.0
008     *
009     *  Unless required by applicable law or agreed to in writing, software
010     *  distributed under the License is distributed on an "AS IS" BASIS,
011     *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
012     *  See the License for the specific language governing permissions and
013     *  limitations under the License. See accompanying LICENSE file.
014     */
015    package org.apache.oozie.service;
016    
017    import java.util.ArrayList;
018    import java.util.Date;
019    import java.util.List;
020    
021    import org.apache.hadoop.conf.Configuration;
022    import org.apache.oozie.CoordinatorActionBean;
023    import org.apache.oozie.CoordinatorJobBean;
024    import org.apache.oozie.WorkflowActionBean;
025    import org.apache.oozie.client.CoordinatorJob;
026    import org.apache.oozie.command.coord.CoordActionInputCheckCommand;
027    import org.apache.oozie.command.coord.CoordActionReadyCommand;
028    import org.apache.oozie.command.coord.CoordActionStartCommand;
029    import org.apache.oozie.command.coord.CoordRecoveryCommand;
030    import org.apache.oozie.command.wf.ActionEndCommand;
031    import org.apache.oozie.command.wf.ActionStartCommand;
032    import org.apache.oozie.command.wf.SignalCommand;
033    import org.apache.oozie.store.CoordinatorStore;
034    import org.apache.oozie.store.Store;
035    import org.apache.oozie.store.StoreException;
036    import org.apache.oozie.store.WorkflowStore;
037    import org.apache.oozie.util.XCallable;
038    import org.apache.oozie.util.XLog;
039    
040    /**
041     * The Recovery Service checks for pending actions and premater coordinator jobs older than a configured age and then
042     * queues them for execution.
043     */
044    public class RecoveryService implements Service {
045    
046        public static final String CONF_PREFIX = Service.CONF_PREFIX + "RecoveryService.";
047        public static final String CONF_PREFIX_WF_ACTIONS = Service.CONF_PREFIX + "wf.actions.";
048        public static final String CONF_PREFIX_COORD = Service.CONF_PREFIX + "coord.";
049        /**
050         * Time interval, in seconds, at which the recovery service will be scheduled to run.
051         */
052        public static final String CONF_SERVICE_INTERVAL = CONF_PREFIX + "interval";
053        /**
054         * The number of callables to be queued in a batch.
055         */
056        public static final String CONF_CALLABLE_BATCH_SIZE = CONF_PREFIX + "callable.batch.size";
057        /**
058         * Age of actions to queue, in seconds.
059         */
060        public static final String CONF_WF_ACTIONS_OLDER_THAN = CONF_PREFIX_WF_ACTIONS + "older.than";
061        /**
062         * Age of coordinator jobs to recover, in seconds.
063         */
064        public static final String CONF_COORD_OLDER_THAN = CONF_PREFIX_COORD + "older.than";
065    
066        private static final String INSTRUMENTATION_GROUP = "recovery";
067        private static final String INSTR_RECOVERED_ACTIONS_COUNTER = "actions";
068        private static final String INSTR_RECOVERED_COORD_JOBS_COUNTER = "coord_jobs";
069        private static final String INSTR_RECOVERED_COORD_ACTIONS_COUNTER = "coord_actions";
070    
071        /**
072         * RecoveryRunnable is the Runnable which is scheduled to run with the configured interval, and takes care of the
073         * queuing of commands.
074         */
075        static class RecoveryRunnable<S extends Store> implements Runnable {
076            private long olderThan;
077            private long coordOlderThan;
078            private long delay = 0;
079            private List<XCallable<Void>> callables;
080            private List<XCallable<Void>> delayedCallables;
081            private StringBuilder msg = null;
082    
083            public RecoveryRunnable(long olderThan, long coordOlderThan) {
084                this.olderThan = olderThan;
085                this.coordOlderThan = coordOlderThan;
086            }
087    
088            public void run() {
089                XLog.Info.get().clear();
090                XLog log = XLog.getLog(getClass());
091                msg = new StringBuilder();
092                runWFRecovery();
093                runCoordJobRecovery();
094                runCoordActionRecovery();
095                runCoordActionRecoveryForReady();
096                log.debug("QUEUING [{0}] for potential recovery", msg.toString());
097                boolean ret = false;
098                if (null != callables) {
099                    ret = Services.get().get(CallableQueueService.class).queueSerial(callables);
100                    if (ret == false) {
101                        log.warn("Unable to queue the callables commands for RecoveryService. "
102                                + "Most possibly command queue is full. Queue size is :"
103                                + Services.get().get(CallableQueueService.class).queueSize());
104                    }
105                    callables = null;
106                }
107                if (null != delayedCallables) {
108                    ret = Services.get().get(CallableQueueService.class).queueSerial(delayedCallables, this.delay);
109                    if (ret == false) {
110                        log.warn("Unable to queue the delayedCallables commands for RecoveryService. "
111                                + "Most possibly Callable queue is full. Queue size is :"
112                                + Services.get().get(CallableQueueService.class).queueSize());
113                    }
114                    delayedCallables = null;
115                    this.delay = 0;
116                }
117            }
118    
119            /**
120             * Recover coordinator jobs that are running and have lastModifiedTimestamp older than the specified interval
121             */
122            private void runCoordJobRecovery() {
123                XLog.Info.get().clear();
124                XLog log = XLog.getLog(getClass());
125    
126                CoordinatorStore store = null;
127                try {
128                    store = Services.get().get(StoreService.class).getStore(CoordinatorStore.class);
129                    store.beginTrx();
130    
131                    // get list of all jobs that have lastModifiedTimestamp older
132                    // than the specified interval
133                    List<CoordinatorJobBean> jobs = store.getCoordinatorJobsOlderThanStatus(coordOlderThan,
134                                                                                            CoordinatorJob.Status.PREMATER.toString(), 50, false);
135                    //log.debug("QUEUING[{0}] PREMATER coord jobs for potential recovery", jobs.size());
136                    msg.append(", COORD_JOBS : " + jobs.size());
137                    for (CoordinatorJobBean coordJob : jobs) {
138                        Services.get().get(InstrumentationService.class).get().incr(INSTRUMENTATION_GROUP,
139                                                                                    INSTR_RECOVERED_COORD_JOBS_COUNTER, 1);
140                        queueCallable(new CoordRecoveryCommand(coordJob.getId()));
141                    }
142    
143                    store.commitTrx();
144                }
145                catch (StoreException ex) {
146                    if (store != null) {
147                        store.rollbackTrx();
148                    }
149                    log.warn("Exception while accessing the store", ex);
150                }
151                catch (Exception ex) {
152                    log.error("Exception, {0}", ex.getMessage(), ex);
153                    if (store != null && store.isActive()) {
154                        try {
155                            store.rollbackTrx();
156                        }
157                        catch (RuntimeException rex) {
158                            log.warn("openjpa error, {0}", rex.getMessage(), rex);
159                        }
160                    }
161                }
162                finally {
163                    if (store != null) {
164                        if (!store.isActive()) {
165                            try {
166                                store.closeTrx();
167                            }
168                            catch (RuntimeException rex) {
169                                log.warn("Exception while attempting to close store", rex);
170                            }
171                        }
172                        else {
173                            log.warn("transaction is not committed or rolled back before closing entitymanager.");
174                        }
175                    }
176                }
177            }
178    
179            /**
180             * Recover coordinator actions that are staying in WAITING or SUBMITTED too long
181             */
182            private void runCoordActionRecovery() {
183                XLog.Info.get().clear();
184                XLog log = XLog.getLog(getClass());
185    
186                CoordinatorStore store = null;
187                try {
188                    store = Services.get().get(StoreService.class).getStore(CoordinatorStore.class);
189                    store.beginTrx();
190    
191                    List<CoordinatorActionBean> cactions = store.getRecoveryActionsOlderThan(coordOlderThan, false);
192                    //log.debug("QUEUING[{0}] WAITING and SUBMITTED coord actions for potential recovery", cactions.size());
193                    msg.append(", COORD_ACTIONS : " + cactions.size());
194                    for (CoordinatorActionBean caction : cactions) {
195                        Services.get().get(InstrumentationService.class).get().incr(INSTRUMENTATION_GROUP,
196                                                                                    INSTR_RECOVERED_COORD_ACTIONS_COUNTER, 1);
197                        if (caction.getStatus() == CoordinatorActionBean.Status.WAITING) {
198                            queueCallable(new CoordActionInputCheckCommand(caction.getId()));
199                            log.info("Recover a WAITTING coord action :" + caction.getId());
200                        }
201                        else {
202                            if (caction.getStatus() == CoordinatorActionBean.Status.SUBMITTED) {
203                                CoordinatorJobBean coordJob = store.getCoordinatorJob(caction.getJobId(), false);
204                                queueCallable(new CoordActionStartCommand(caction.getId(), coordJob.getUser(), coordJob
205                                        .getAuthToken()));
206                                log.info("Recover a SUBMITTED coord action :" + caction.getId());
207                            }
208                        }
209                    }
210                    store.commitTrx();
211                }
212                catch (StoreException ex) {
213                    if (store != null) {
214                        store.rollbackTrx();
215                    }
216                    log.warn("Exception while accessing the store", ex);
217                }
218                catch (Exception ex) {
219                    log.error("Exception, {0}", ex.getMessage(), ex);
220                    if (store != null && store.isActive()) {
221                        try {
222                            store.rollbackTrx();
223                        }
224                        catch (RuntimeException rex) {
225                            log.warn("openjpa error, {0}", rex.getMessage(), rex);
226                        }
227                    }
228                }
229                finally {
230                    if (store != null) {
231                        if (!store.isActive()) {
232                            try {
233                                store.closeTrx();
234                            }
235                            catch (RuntimeException rex) {
236                                log.warn("Exception while attempting to close store", rex);
237                            }
238                        }
239                        else {
240                            log.warn("transaction is not committed or rolled back before closing entitymanager.");
241                        }
242                    }
243                }
244            }
245    
246            /**
247             * Recover coordinator actions that are staying in READY too long
248             */
249            private void runCoordActionRecoveryForReady() {
250                XLog.Info.get().clear();
251                XLog log = XLog.getLog(getClass());
252    
253                CoordinatorStore store = null;
254                try {
255                    store = Services.get().get(StoreService.class).getStore(CoordinatorStore.class);
256                    store.beginTrx();
257                    List<String> jobids = store.getRecoveryActionsGroupByJobId(coordOlderThan);
258                    //log.debug("QUEUING[{0}] READY coord jobs for potential recovery", jobids.size());
259                    msg.append(", COORD_READY_JOBS : " + jobids.size());
260                    for (String jobid : jobids) {
261                        queueCallable(new CoordActionReadyCommand(jobid));
262                        log.info("Recover READY coord actions for jobid :" + jobid);
263                    }
264                    store.commitTrx();
265                }
266                catch (StoreException ex) {
267                    if (store != null) {
268                        store.rollbackTrx();
269                    }
270                    log.warn("Exception while accessing the store", ex);
271                }
272                catch (Exception ex) {
273                    log.error("Exception, {0}", ex.getMessage(), ex);
274                    if (store != null && store.isActive()) {
275                        try {
276                            store.rollbackTrx();
277                        }
278                        catch (RuntimeException rex) {
279                            log.warn("openjpa error, {0}", rex.getMessage(), rex);
280                        }
281                    }
282                }
283                finally {
284                    if (store != null) {
285                        if (!store.isActive()) {
286                            try {
287                                store.closeTrx();
288                            }
289                            catch (RuntimeException rex) {
290                                log.warn("Exception while attempting to close store", rex);
291                            }
292                        }
293                        else {
294                            log.warn("transaction is not committed or rolled back before closing entitymanager.");
295                        }
296                    }
297                }
298            }
299    
300            /**
301             * Recover wf actions
302             */
303            private void runWFRecovery() {
304                XLog.Info.get().clear();
305                XLog log = XLog.getLog(getClass());
306                // queue command for action recovery
307                WorkflowStore store = null;
308                try {
309                    store = Services.get().get(StoreService.class).getStore(WorkflowStore.class);
310                    store.beginTrx();
311                    List<WorkflowActionBean> actions = null;
312                    try {
313                        actions = store.getPendingActions(olderThan);
314                    }
315                    catch (StoreException ex) {
316                        log.warn("Exception while reading pending actions from storage", ex);
317                    }
318                    //log.debug("QUEUING[{0}] pending wf actions for potential recovery", actions.size());
319                    msg.append(" WF_ACTIONS " + actions.size());
320    
321                    for (WorkflowActionBean action : actions) {
322                        Services.get().get(InstrumentationService.class).get().incr(INSTRUMENTATION_GROUP,
323                                                                                    INSTR_RECOVERED_ACTIONS_COUNTER, 1);
324                        if (action.getStatus() == WorkflowActionBean.Status.PREP
325                                || action.getStatus() == WorkflowActionBean.Status.START_MANUAL) {
326                            queueCallable(new ActionStartCommand(action.getId(), action.getType()));
327                        }
328                        else {
329                            if (action.getStatus() == WorkflowActionBean.Status.START_RETRY) {
330                                Date nextRunTime = action.getPendingAge();
331                                queueCallable(new ActionStartCommand(action.getId(), action.getType()), nextRunTime.getTime()
332                                        - System.currentTimeMillis());
333                            }
334                            else {
335                                if (action.getStatus() == WorkflowActionBean.Status.DONE
336                                        || action.getStatus() == WorkflowActionBean.Status.END_MANUAL) {
337                                    queueCallable(new ActionEndCommand(action.getId(), action.getType()));
338                                }
339                                else {
340                                    if (action.getStatus() == WorkflowActionBean.Status.END_RETRY) {
341                                        Date nextRunTime = action.getPendingAge();
342                                        queueCallable(new ActionEndCommand(action.getId(), action.getType()), nextRunTime.getTime()
343                                                - System.currentTimeMillis());
344                                    }
345                                    else {
346                                        if (action.getStatus() == WorkflowActionBean.Status.OK
347                                                || action.getStatus() == WorkflowActionBean.Status.ERROR) {
348                                            queueCallable(new SignalCommand(action.getJobId(), action.getId()));
349                                        }
350                                    }
351                                }
352                            }
353                        }
354                    }
355                    store.commitTrx();
356                }
357                catch (StoreException ex) {
358                    if (store != null) {
359                        store.rollbackTrx();
360                    }
361                    log.warn("Exception while getting store to get pending actions", ex);
362                }
363                catch (Exception ex) {
364                    log.error("Exception, {0}", ex.getMessage(), ex);
365                    if (store != null && store.isActive()) {
366                        try {
367                            store.rollbackTrx();
368                        }
369                        catch (RuntimeException rex) {
370                            log.warn("openjpa error, {0}", rex.getMessage(), rex);
371                        }
372                    }
373                }
374                finally {
375                    if (store != null) {
376                        if (!store.isActive()) {
377                            try {
378                                store.closeTrx();
379                            }
380                            catch (RuntimeException rex) {
381                                log.warn("Exception while attempting to close store", rex);
382                            }
383                        }
384                        else {
385                            log.warn("transaction is not committed or rolled back before closing entitymanager.");
386                        }
387                    }
388                }
389            }
390    
391            /**
392             * Adds callables to a list. If the number of callables in the list reaches {@link
393             * RecoveryService#CONF_CALLABLE_BATCH_SIZE}, the entire batch is queued and the callables list is reset.
394             *
395             * @param callable the callable to queue.
396             */
397            private void queueCallable(XCallable<Void> callable) {
398                if (callables == null) {
399                    callables = new ArrayList<XCallable<Void>>();
400                }
401                callables.add(callable);
402                if (callables.size() == Services.get().getConf().getInt(CONF_CALLABLE_BATCH_SIZE, 10)) {
403                    boolean ret = Services.get().get(CallableQueueService.class).queueSerial(callables);
404                    if (ret == false) {
405                        XLog.getLog(getClass()).warn(
406                                "Unable to queue the callables commands for RecoveryService. "
407                                        + "Most possibly command queue is full. Queue size is :"
408                                        + Services.get().get(CallableQueueService.class).queueSize());
409                    }
410                    callables = new ArrayList<XCallable<Void>>();
411                }
412            }
413    
414            /**
415             * Adds callables to a list. If the number of callables in the list reaches {@link
416             * RecoveryService#CONF_CALLABLE_BATCH_SIZE}, the entire batch is queued with the delay set to the maximum delay
417             * of the callables in the list. The callables list and the delay is reset.
418             *
419             * @param callable the callable to queue.
420             * @param delay the delay for the callable.
421             */
422            private void queueCallable(XCallable<Void> callable, long delay) {
423                if (delayedCallables == null) {
424                    delayedCallables = new ArrayList<XCallable<Void>>();
425                }
426                this.delay = Math.max(this.delay, delay);
427                delayedCallables.add(callable);
428                if (delayedCallables.size() == Services.get().getConf().getInt(CONF_CALLABLE_BATCH_SIZE, 10)) {
429                    boolean ret = Services.get().get(CallableQueueService.class).queueSerial(delayedCallables, this.delay);
430                    if (ret == false) {
431                        XLog.getLog(getClass()).warn("Unable to queue the delayedCallables commands for RecoveryService. "
432                                + "Most possibly Callable queue is full. Queue size is :"
433                                + Services.get().get(CallableQueueService.class).queueSize());
434                    }
435                    delayedCallables = new ArrayList<XCallable<Void>>();
436                    this.delay = 0;
437                }
438            }
439        }
440    
441        /**
442         * Initializes the RecoveryService.
443         *
444         * @param services services instance.
445         */
446        @Override
447        public void init(Services services) {
448            Configuration conf = services.getConf();
449            Runnable recoveryRunnable = new RecoveryRunnable(conf.getInt(CONF_WF_ACTIONS_OLDER_THAN, 120), conf.getInt(
450                    CONF_COORD_OLDER_THAN, 600));
451            services.get(SchedulerService.class).schedule(recoveryRunnable, 10, conf.getInt(CONF_SERVICE_INTERVAL, 600),
452                                                          SchedulerService.Unit.SEC);
453        }
454    
455        /**
456         * Destroy the Recovery Service.
457         */
458        @Override
459        public void destroy() {
460        }
461    
462        /**
463         * Return the public interface for the Recovery Service.
464         *
465         * @return {@link RecoveryService}.
466         */
467        @Override
468        public Class<? extends Service> getInterface() {
469            return RecoveryService.class;
470        }
471    }