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.CoordinatorJobBean;
023    import org.apache.oozie.command.coord.CoordMaterializeTransitionXCommand;
024    import org.apache.oozie.executor.jpa.CoordActionsActiveCountJPAExecutor;
025    import org.apache.oozie.executor.jpa.CoordJobGetRunningActionsCountJPAExecutor;
026    import org.apache.oozie.executor.jpa.CoordJobUpdateJPAExecutor;
027    import org.apache.oozie.executor.jpa.CoordJobsToBeMaterializedJPAExecutor;
028    import org.apache.oozie.executor.jpa.JPAExecutorException;
029    import org.apache.oozie.util.XCallable;
030    import org.apache.oozie.util.XLog;
031    
032    /**
033     * The coordinator Materialization Lookup trigger service schedule lookup trigger command for every interval (default is
034     * 5 minutes ). This interval could be configured through oozie configuration defined is either oozie-default.xml or
035     * oozie-site.xml using the property name oozie.service.CoordMaterializeTriggerService.lookup.interval
036     */
037    public class CoordMaterializeTriggerService implements Service {
038        public static final String CONF_PREFIX = Service.CONF_PREFIX + "CoordMaterializeTriggerService.";
039        /**
040         * Time interval, in seconds, at which the Job materialization service will be scheduled to run.
041         */
042        public static final String CONF_LOOKUP_INTERVAL = CONF_PREFIX + "lookup.interval";
043        /**
044         * This configuration defined the duration for which job should be materialized in future
045         */
046        public static final String CONF_MATERIALIZATION_WINDOW = CONF_PREFIX + "materialization.window";
047        /**
048         * The number of callables to be queued in a batch.
049         */
050        public static final String CONF_CALLABLE_BATCH_SIZE = CONF_PREFIX + "callable.batch.size";
051    
052        private static final String INSTRUMENTATION_GROUP = "coord_job_mat";
053        private static final String INSTR_MAT_JOBS_COUNTER = "jobs";
054        private static final int CONF_LOOKUP_INTERVAL_DEFAULT = 300;
055        private static final int CONF_MATERIALIZATION_WINDOW_DEFAULT = 3600;
056    
057        /**
058         * This runnable class will run in every "interval" to queue CoordMaterializeTransitionXCommand.
059         */
060        static class CoordMaterializeTriggerRunnable implements Runnable {
061            private int materializationWindow;
062            private long delay = 0;
063            private List<XCallable<Void>> callables;
064            private List<XCallable<Void>> delayedCallables;
065    
066            public CoordMaterializeTriggerRunnable(int materializationWindow) {
067                this.materializationWindow = materializationWindow;
068            }
069    
070            @Override
071            public void run() {
072                runCoordJobMatLookup();
073    
074                if (null != callables) {
075                    boolean ret = Services.get().get(CallableQueueService.class).queueSerial(callables);
076                    if (ret == false) {
077                        XLog.getLog(getClass()).warn(
078                                "Unable to queue the callables commands for CoordMaterializeTriggerRunnable. "
079                                        + "Most possibly command queue is full. Queue size is :"
080                                        + Services.get().get(CallableQueueService.class).queueSize());
081                    }
082                    callables = null;
083                }
084                if (null != delayedCallables) {
085                    boolean ret = Services.get().get(CallableQueueService.class).queueSerial(delayedCallables, this.delay);
086                    if (ret == false) {
087                        XLog.getLog(getClass()).warn(
088                                "Unable to queue the delayedCallables commands for CoordMaterializeTriggerRunnable. "
089                                        + "Most possibly Callable queue is full. Queue size is :"
090                                        + Services.get().get(CallableQueueService.class).queueSize());
091                    }
092                    delayedCallables = null;
093                    this.delay = 0;
094                }
095            }
096    
097            /**
098             * Recover coordinator jobs that should be materialized
099             */
100            private void runCoordJobMatLookup() {
101                XLog.Info.get().clear();
102                XLog LOG = XLog.getLog(getClass());
103                JPAService jpaService = Services.get().get(JPAService.class);
104                try {
105    
106                    // get current date
107                    Date currDate = new Date(new Date().getTime() + CONF_LOOKUP_INTERVAL_DEFAULT * 1000);
108                    // get list of all jobs that have actions that should be materialized.
109                    CoordJobsToBeMaterializedJPAExecutor cmatcmd = new CoordJobsToBeMaterializedJPAExecutor(currDate, 50);
110                    List<CoordinatorJobBean> materializeJobs = jpaService.execute(cmatcmd);
111                    LOG.debug("CoordMaterializeTriggerService - Curr Date= " + currDate + ", Num jobs to materialize = "
112                            + materializeJobs.size());
113                    for (CoordinatorJobBean coordJob : materializeJobs) {
114                        Services.get().get(InstrumentationService.class).get().incr(INSTRUMENTATION_GROUP,
115                                INSTR_MAT_JOBS_COUNTER, 1);
116                        int numWaitingActions = jpaService
117                                .execute(new CoordActionsActiveCountJPAExecutor(coordJob.getId()));
118                        LOG.debug("Job :" + coordJob.getId() + "  numWaitingActions : " + numWaitingActions + " MatThrottle : "
119                                + coordJob.getMatThrottling());
120                        if (numWaitingActions >= coordJob.getMatThrottling()) {
121                            LOG.debug("Materialization skipped for JobID [" + coordJob.getId() + " already waiting "
122                                    + numWaitingActions + " actions. MatThrottle is : " + coordJob.getMatThrottling());
123                            continue;
124                        }
125                        queueCallable(new CoordMaterializeTransitionXCommand(coordJob.getId(), materializationWindow));
126                        //update lastModifiedTime so next time others might have higher chance to get pick up
127                        coordJob.setLastModifiedTime(new Date());
128                        jpaService.execute(new CoordJobUpdateJPAExecutor(coordJob));
129    
130                    }
131    
132                }
133                catch (JPAExecutorException jex) {
134                    LOG.warn("JPAExecutorException while attempting to materialize coordinator jobs", jex);
135                }
136            }
137    
138            /**
139             * Adds callables to a list. If the number of callables in the list reaches {@link
140             * CoordMaterializeTriggerService#CONF_CALLABLE_BATCH_SIZE}, the entire batch is queued and the callables list
141             * is reset.
142             *
143             * @param callable the callable to queue.
144             */
145            private void queueCallable(XCallable<Void> callable) {
146                if (callables == null) {
147                    callables = new ArrayList<XCallable<Void>>();
148                }
149                callables.add(callable);
150                if (callables.size() == Services.get().getConf().getInt(CONF_CALLABLE_BATCH_SIZE, 10)) {
151                    boolean ret = Services.get().get(CallableQueueService.class).queueSerial(callables);
152                    if (ret == false) {
153                        XLog.getLog(getClass()).warn(
154                                "Unable to queue the callables commands for CoordMaterializeTriggerRunnable. "
155                                        + "Most possibly command queue is full. Queue size is :"
156                                        + Services.get().get(CallableQueueService.class).queueSize());
157                    }
158                    callables = new ArrayList<XCallable<Void>>();
159                }
160            }
161    
162        }
163    
164        @Override
165        public void init(Services services) throws ServiceException {
166            Configuration conf = services.getConf();
167            Runnable lookupTriggerJobsRunnable = new CoordMaterializeTriggerRunnable(conf.getInt(
168                    CONF_MATERIALIZATION_WINDOW, CONF_MATERIALIZATION_WINDOW_DEFAULT));// Default is 1 hour
169            services.get(SchedulerService.class).schedule(lookupTriggerJobsRunnable, 10,
170                                                          conf.getInt(CONF_LOOKUP_INTERVAL, CONF_LOOKUP_INTERVAL_DEFAULT),// Default is 5 minutes
171                                                          SchedulerService.Unit.SEC);
172        }
173    
174        @Override
175        public void destroy() {
176        }
177    
178        @Override
179        public Class<? extends Service> getInterface() {
180            return CoordMaterializeTriggerService.class;
181        }
182    
183    }