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.CoordJobMatLookupCommand;
024    import org.apache.oozie.store.CoordinatorStore;
025    import org.apache.oozie.store.StoreException;
026    import org.apache.oozie.util.XCallable;
027    import org.apache.oozie.util.XLog;
028    
029    /**
030     * The coordinator Materialization Lookup trigger service schedule lookup trigger command for every interval (default is
031     * 5 minutes ). This interval could be configured through oozie configuration defined is either oozie-default.xml or
032     * oozie-site.xml using the property name oozie.service.CoordJobMatLookupTriggerService.lookup.interval
033     */
034    public class CoordJobMatLookupTriggerService implements Service {
035        public static final String CONF_PREFIX = Service.CONF_PREFIX + "CoordJobMatLookupTriggerService.";
036        /**
037         * Time interval, in seconds, at which the Job materialization service will be scheduled to run.
038         */
039        public static final String CONF_LOOKUP_INTERVAL = CONF_PREFIX + "lookup.interval";
040        /**
041         * This configuration defined the duration for which job should be materialized in future
042         */
043        public static final String CONF_MATERIALIZATION_WINDOW = CONF_PREFIX + "materialization.window";
044        /**
045         * The number of callables to be queued in a batch.
046         */
047        public static final String CONF_CALLABLE_BATCH_SIZE = CONF_PREFIX + "callable.batch.size";
048    
049        private static final String INSTRUMENTATION_GROUP = "coord_job_mat_lookup";
050        private static final String INSTR_MAT_JOBS_COUNTER = "jobs";
051        private static final int CONF_LOOKUP_INTERVAL_DEFAULT = 300;
052        private static final int CONF_MATERIALIZATION_WINDOW_DEFAULT = 3600;
053    
054        /**
055         * This runnable class will run in every "interval" to queue CoordJobMatLookupTriggerCommand.
056         */
057        static class CoordJobMatLookupTriggerRunnable implements Runnable {
058            private int materializationWindow;
059            private long delay = 0;
060            private List<XCallable<Void>> callables;
061            private List<XCallable<Void>> delayedCallables;
062    
063            public CoordJobMatLookupTriggerRunnable(int materializationWindow) {
064                this.materializationWindow = materializationWindow;
065            }
066    
067            @Override
068            public void run() {
069                runCoordJobMatLookup();
070    
071                if (null != callables) {
072                    boolean ret = Services.get().get(CallableQueueService.class).queueSerial(callables);
073                    if (ret == false) {
074                        XLog.getLog(getClass()).warn(
075                                "Unable to queue the callables commands for CoordJobMatLookupTriggerRunnable. "
076                                        + "Most possibly command queue is full. Queue size is :"
077                                        + Services.get().get(CallableQueueService.class).queueSize());
078                    }
079                    callables = null;
080                }
081                if (null != delayedCallables) {
082                    boolean ret = Services.get().get(CallableQueueService.class).queueSerial(delayedCallables, this.delay);
083                    if (ret == false) {
084                        XLog.getLog(getClass()).warn(
085                                "Unable to queue the delayedCallables commands for CoordJobMatLookupTriggerRunnable. "
086                                        + "Most possibly Callable queue is full. Queue size is :"
087                                        + Services.get().get(CallableQueueService.class).queueSize());
088                    }
089                    delayedCallables = null;
090                    this.delay = 0;
091                }
092            }
093    
094            /**
095             * Recover coordinator jobs that should be materialized
096             */
097            private void runCoordJobMatLookup() {
098                XLog.Info.get().clear();
099                XLog log = XLog.getLog(getClass());
100    
101                CoordinatorStore store = null;
102                try {
103                    store = Services.get().get(StoreService.class).getStore(CoordinatorStore.class);
104                    store.beginTrx();
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
109                    // materialized.
110                    List<CoordinatorJobBean> materializeJobs = store.getCoordinatorJobsToBeMaterialized(currDate, 50);
111                    log.debug("CoordJobMatLookupTriggerService - 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                        queueCallable(new CoordJobMatLookupCommand(coordJob.getId(), materializationWindow));
117                    }
118    
119                    store.commitTrx();
120                }
121                catch (StoreException ex) {
122                    if (store != null) {
123                        store.rollbackTrx();
124                    }
125                    log.warn("Exception while accessing the store", ex);
126                }
127                catch (Exception ex) {
128                    log.error("Exception, {0}", ex.getMessage(), ex);
129                    if (store != null && store.isActive()) {
130                        try {
131                            store.rollbackTrx();
132                        }
133                        catch (RuntimeException rex) {
134                            log.warn("openjpa error, {0}", rex.getMessage(), rex);
135                        }
136                    }
137                }
138                finally {
139                    if (store != null) {
140                        if (!store.isActive()) {
141                            try {
142                                store.closeTrx();
143                            }
144                            catch (RuntimeException rex) {
145                                log.warn("Exception while attempting to close store", rex);
146                            }
147                        }
148                        else {
149                            log.warn("transaction is not committed or rolled back before closing entitymanager.");
150                        }
151                    }
152                }
153            }
154    
155            /**
156             * Adds callables to a list. If the number of callables in the list reaches {@link
157             * CoordJobMatLookupTriggerService#CONF_CALLABLE_BATCH_SIZE}, the entire batch is queued and the callables list
158             * is reset.
159             *
160             * @param callable the callable to queue.
161             */
162            private void queueCallable(XCallable<Void> callable) {
163                if (callables == null) {
164                    callables = new ArrayList<XCallable<Void>>();
165                }
166                callables.add(callable);
167                if (callables.size() == Services.get().getConf().getInt(CONF_CALLABLE_BATCH_SIZE, 10)) {
168                    boolean ret = Services.get().get(CallableQueueService.class).queueSerial(callables);
169                    if (ret == false) {
170                        XLog.getLog(getClass()).warn(
171                                "Unable to queue the callables commands for CoordJobMatLookupTriggerRunnable. "
172                                        + "Most possibly command queue is full. Queue size is :"
173                                        + Services.get().get(CallableQueueService.class).queueSize());
174                    }
175                    callables = new ArrayList<XCallable<Void>>();
176                }
177            }
178    
179            /**
180             * Adds callables to a list. If the number of callables in the list reaches {@link
181             * CoordJobMatLookupTriggerService#CONF_CALLABLE_BATCH_SIZE}, the entire batch is queued with the delay set to
182             * the maximum delay of the callables in the list. The callables list and the delay is reset.
183             *
184             * @param callable the callable to queue.
185             * @param delay the delay for the callable.
186             */
187            private void queueCallable(XCallable<Void> callable, long delay) {
188                if (delayedCallables == null) {
189                    delayedCallables = new ArrayList<XCallable<Void>>();
190                }
191                this.delay = Math.max(this.delay, delay);
192                delayedCallables.add(callable);
193                if (delayedCallables.size() == Services.get().getConf().getInt(CONF_CALLABLE_BATCH_SIZE, 10)) {
194                    boolean ret = Services.get().get(CallableQueueService.class).queueSerial(delayedCallables, this.delay);
195                    if (ret == false) {
196                        XLog.getLog(getClass()).warn(
197                                "Unable to queue the delayedCallables commands for CoordJobMatLookupTriggerRunnable. "
198                                        + "Most possibly Callable queue is full. Queue size is :"
199                                        + Services.get().get(CallableQueueService.class).queueSize());
200                    }
201                    delayedCallables = new ArrayList<XCallable<Void>>();
202                    this.delay = 0;
203                }
204            }
205    
206        }
207    
208        @Override
209        public void init(Services services) throws ServiceException {
210            Configuration conf = services.getConf();
211            Runnable lookupTriggerJobsRunnable = new CoordJobMatLookupTriggerRunnable(conf.getInt(
212                    CONF_MATERIALIZATION_WINDOW, CONF_MATERIALIZATION_WINDOW_DEFAULT));// Default is 1 hour
213            services.get(SchedulerService.class).schedule(lookupTriggerJobsRunnable, 10,
214                                                          conf.getInt(CONF_LOOKUP_INTERVAL, CONF_LOOKUP_INTERVAL_DEFAULT),// Default is 5 minutes
215                                                          SchedulerService.Unit.SEC);
216            return;
217        }
218    
219        @Override
220        public void destroy() {
221            // TODO Auto-generated method stub
222    
223        }
224    
225        @Override
226        public Class<? extends Service> getInterface() {
227            return CoordJobMatLookupTriggerService.class;
228        }
229    
230    }