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.command.wf;
016    
017    import java.io.IOException;
018    import java.net.URI;
019    import java.net.URISyntaxException;
020    import java.util.Collection;
021    import java.util.HashMap;
022    import java.util.HashSet;
023    import java.util.List;
024    import java.util.Map;
025    import java.util.Set;
026    
027    import org.apache.hadoop.conf.Configuration;
028    import org.apache.hadoop.fs.FileSystem;
029    import org.apache.hadoop.fs.Path;
030    import org.apache.oozie.ErrorCode;
031    import org.apache.oozie.WorkflowActionBean;
032    import org.apache.oozie.WorkflowJobBean;
033    import org.apache.oozie.client.OozieClient;
034    import org.apache.oozie.client.WorkflowAction;
035    import org.apache.oozie.client.WorkflowJob;
036    import org.apache.oozie.command.CommandException;
037    import org.apache.oozie.command.PreconditionException;
038    import org.apache.oozie.executor.jpa.JPAExecutorException;
039    import org.apache.oozie.executor.jpa.WorkflowActionDeleteJPAExecutor;
040    import org.apache.oozie.executor.jpa.WorkflowActionsGetForJobJPAExecutor;
041    import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
042    import org.apache.oozie.executor.jpa.WorkflowJobUpdateJPAExecutor;
043    import org.apache.oozie.service.DagXLogInfoService;
044    import org.apache.oozie.service.HadoopAccessorException;
045    import org.apache.oozie.service.HadoopAccessorService;
046    import org.apache.oozie.service.JPAService;
047    import org.apache.oozie.service.Services;
048    import org.apache.oozie.service.WorkflowAppService;
049    import org.apache.oozie.service.WorkflowStoreService;
050    import org.apache.oozie.util.InstrumentUtils;
051    import org.apache.oozie.util.LogUtils;
052    import org.apache.oozie.util.ParamChecker;
053    import org.apache.oozie.util.PropertiesUtils;
054    import org.apache.oozie.util.XConfiguration;
055    import org.apache.oozie.util.XLog;
056    import org.apache.oozie.util.XmlUtils;
057    import org.apache.oozie.workflow.WorkflowApp;
058    import org.apache.oozie.workflow.WorkflowException;
059    import org.apache.oozie.workflow.WorkflowInstance;
060    import org.apache.oozie.workflow.WorkflowLib;
061    import org.apache.oozie.workflow.lite.NodeHandler;
062    
063    /**
064     * This is a RerunXCommand which is used for rerunn.
065     *
066     */
067    public class ReRunXCommand extends WorkflowXCommand<Void> {
068        private final String jobId;
069        private final Configuration conf;
070        private final String authToken;
071        private final Set<String> nodesToSkip = new HashSet<String>();
072        public static final String TO_SKIP = "TO_SKIP";
073        private WorkflowJobBean wfBean;
074        private List<WorkflowActionBean> actions;
075        private JPAService jpaService;
076    
077        private static final Set<String> DISALLOWED_DEFAULT_PROPERTIES = new HashSet<String>();
078        private static final Set<String> DISALLOWED_USER_PROPERTIES = new HashSet<String>();
079    
080        static {
081            String[] badUserProps = { PropertiesUtils.DAYS, PropertiesUtils.HOURS, PropertiesUtils.MINUTES,
082                    PropertiesUtils.KB, PropertiesUtils.MB, PropertiesUtils.GB, PropertiesUtils.TB, PropertiesUtils.PB,
083                    PropertiesUtils.RECORDS, PropertiesUtils.MAP_IN, PropertiesUtils.MAP_OUT, PropertiesUtils.REDUCE_IN,
084                    PropertiesUtils.REDUCE_OUT, PropertiesUtils.GROUPS };
085            PropertiesUtils.createPropertySet(badUserProps, DISALLOWED_USER_PROPERTIES);
086    
087            String[] badDefaultProps = { PropertiesUtils.HADOOP_USER, PropertiesUtils.HADOOP_UGI,
088                    WorkflowAppService.HADOOP_JT_KERBEROS_NAME, WorkflowAppService.HADOOP_NN_KERBEROS_NAME };
089            PropertiesUtils.createPropertySet(badUserProps, DISALLOWED_DEFAULT_PROPERTIES);
090            PropertiesUtils.createPropertySet(badDefaultProps, DISALLOWED_DEFAULT_PROPERTIES);
091        }
092    
093        public ReRunXCommand(String jobId, Configuration conf, String authToken) {
094            super("rerun", "rerun", 1);
095            this.jobId = ParamChecker.notEmpty(jobId, "jobId");
096            this.conf = ParamChecker.notNull(conf, "conf");
097            this.authToken = ParamChecker.notEmpty(authToken, "authToken");
098        }
099    
100        /* (non-Javadoc)
101         * @see org.apache.oozie.command.XCommand#execute()
102         */
103        @Override
104        protected Void execute() throws CommandException {
105            InstrumentUtils.incrJobCounter(getName(), 1, getInstrumentation());
106            LogUtils.setLogInfo(wfBean, logInfo);
107            WorkflowInstance oldWfInstance = this.wfBean.getWorkflowInstance();
108            WorkflowInstance newWfInstance;
109    
110            WorkflowAppService wps = Services.get().get(WorkflowAppService.class);
111            try {
112                XLog.Info.get().setParameter(DagXLogInfoService.TOKEN, conf.get(OozieClient.LOG_TOKEN));
113                WorkflowApp app = wps.parseDef(conf, authToken);
114                XConfiguration protoActionConf = wps.createProtoActionConf(conf, authToken, true);
115                WorkflowLib workflowLib = Services.get().get(WorkflowStoreService.class).getWorkflowLibWithNoDB();
116    
117                URI uri = new URI(conf.get(OozieClient.APP_PATH));
118                FileSystem fs = Services.get().get(HadoopAccessorService.class).createFileSystem(wfBean.getUser(),
119                        wfBean.getGroup(), uri, new Configuration());
120    
121                Path configDefault = null;
122                // app path could be a directory
123                Path path = new Path(uri.getPath());
124                if (!fs.isFile(path)) {
125                    configDefault = new Path(path, SubmitCommand.CONFIG_DEFAULT);
126                } else {
127                    configDefault = new Path(path.getParent(), SubmitCommand.CONFIG_DEFAULT);
128                }
129    
130                if (fs.exists(configDefault)) {
131                    Configuration defaultConf = new XConfiguration(fs.open(configDefault));
132                    PropertiesUtils.checkDisallowedProperties(defaultConf, DISALLOWED_DEFAULT_PROPERTIES);
133                    XConfiguration.injectDefaults(defaultConf, conf);
134                }
135    
136                PropertiesUtils.checkDisallowedProperties(conf, DISALLOWED_USER_PROPERTIES);
137    
138                try {
139                    newWfInstance = workflowLib.createInstance(app, conf, jobId);
140                }
141                catch (WorkflowException e) {
142                    throw new CommandException(e);
143                }
144                wfBean.setAppName(app.getName());
145                wfBean.setProtoActionConf(protoActionConf.toXmlString());
146            }
147            catch (WorkflowException ex) {
148                throw new CommandException(ex);
149            }
150            catch (IOException ex) {
151                throw new CommandException(ErrorCode.E0803, ex);
152            }
153            catch (HadoopAccessorException ex) {
154                throw new CommandException(ex);
155            }
156            catch (URISyntaxException ex) {
157                throw new CommandException(ErrorCode.E0711, ex.getMessage(), ex);
158            }
159    
160            try {
161                for (int i = 0; i < actions.size(); i++) {
162                    if (!nodesToSkip.contains(actions.get(i).getName())) {
163                        jpaService.execute(new WorkflowActionDeleteJPAExecutor(actions.get(i).getId()));
164                        LOG.info("Deleting Action[{0}] for re-run", actions.get(i).getId());
165                    }
166                    else {
167                        copyActionData(newWfInstance, oldWfInstance);
168                    }
169                }
170    
171                wfBean.setAppPath(conf.get(OozieClient.APP_PATH));
172                wfBean.setConf(XmlUtils.prettyPrint(conf).toString());
173                wfBean.setLogToken(conf.get(OozieClient.LOG_TOKEN, ""));
174                wfBean.setUser(conf.get(OozieClient.USER_NAME));
175                wfBean.setGroup(conf.get(OozieClient.GROUP_NAME));
176                wfBean.setExternalId(conf.get(OozieClient.EXTERNAL_ID));
177                wfBean.setEndTime(null);
178                wfBean.setRun(wfBean.getRun() + 1);
179                wfBean.setStatus(WorkflowJob.Status.PREP);
180                wfBean.setWorkflowInstance(newWfInstance);
181                jpaService.execute(new WorkflowJobUpdateJPAExecutor(wfBean));
182            }
183            catch (JPAExecutorException e) {
184                throw new CommandException(e);
185            }
186    
187            return null;
188        }
189    
190        /**
191         * Loading the Wfjob and workflow actions. Parses the config and adds the nodes that are to be skipped to the
192         * skipped node list
193         *
194         * @throws CommandException
195         */
196        @Override
197        protected void eagerLoadState() throws CommandException {
198            super.eagerLoadState();
199            try {
200                jpaService = Services.get().get(JPAService.class);
201                if (jpaService != null) {
202                    this.wfBean = jpaService.execute(new WorkflowJobGetJPAExecutor(this.jobId));
203                    this.actions = jpaService.execute(new WorkflowActionsGetForJobJPAExecutor(this.jobId));
204                }
205                else {
206                    throw new CommandException(ErrorCode.E0610);
207                }
208    
209                if (conf != null) {
210                    if (conf.getBoolean(OozieClient.RERUN_FAIL_NODES, false) == false) { //Rerun with skipNodes
211                        Collection<String> skipNodes = conf.getStringCollection(OozieClient.RERUN_SKIP_NODES);
212                        for (String str : skipNodes) {
213                            // trimming is required
214                            nodesToSkip.add(str.trim());
215                        }
216                        LOG.debug("Skipnode size :" + nodesToSkip.size());
217                    }
218                    else {
219                        for (WorkflowActionBean action : actions) { // Rerun from failed nodes
220                            if (action.getStatus() == WorkflowAction.Status.OK) {
221                                nodesToSkip.add(action.getName());
222                            }
223                        }
224                        LOG.debug("Skipnode size are to rerun from FAIL nodes :" + nodesToSkip.size());
225                    }
226                    StringBuilder tmp = new StringBuilder();
227                    for (String node : nodesToSkip) {
228                        tmp.append(node).append(",");
229                    }
230                    LOG.debug("SkipNode List :" + tmp);
231                }
232            }
233            catch (Exception ex) {
234                throw new CommandException(ErrorCode.E0603, ex);
235            }
236        }
237    
238        /**
239         * Checks the pre-conditions that are required for workflow to recover - Last run of Workflow should be completed -
240         * The nodes that are to be skipped are to be completed successfully in the base run.
241         *
242         * @throws org.apache.oozie.command.CommandException,PreconditionException On failure of pre-conditions
243         */
244        @Override
245        protected void eagerVerifyPrecondition() throws CommandException, PreconditionException {
246            super.eagerVerifyPrecondition();
247            if (!(wfBean.getStatus().equals(WorkflowJob.Status.FAILED)
248                    || wfBean.getStatus().equals(WorkflowJob.Status.KILLED) || wfBean.getStatus().equals(
249                            WorkflowJob.Status.SUCCEEDED))) {
250                throw new CommandException(ErrorCode.E0805, wfBean.getStatus());
251            }
252            Set<String> unmachedNodes = new HashSet<String>(nodesToSkip);
253            for (WorkflowActionBean action : actions) {
254                if (nodesToSkip.contains(action.getName())) {
255                    if (!action.getStatus().equals(WorkflowAction.Status.OK)
256                            && !action.getStatus().equals(WorkflowAction.Status.ERROR)) {
257                        throw new CommandException(ErrorCode.E0806, action.getName());
258                    }
259                    unmachedNodes.remove(action.getName());
260                }
261            }
262            if (unmachedNodes.size() > 0) {
263                StringBuilder sb = new StringBuilder();
264                String separator = "";
265                for (String s : unmachedNodes) {
266                    sb.append(separator).append(s);
267                    separator = ",";
268                }
269                throw new CommandException(ErrorCode.E0807, sb);
270            }
271        }
272    
273        /**
274         * Copys the variables for skipped nodes from the old wfInstance to new one.
275         *
276         * @param newWfInstance : Source WF instance object
277         * @param oldWfInstance : Update WF instance
278         */
279        private void copyActionData(WorkflowInstance newWfInstance, WorkflowInstance oldWfInstance) {
280            Map<String, String> oldVars = new HashMap<String, String>();
281            Map<String, String> newVars = new HashMap<String, String>();
282            oldVars = oldWfInstance.getAllVars();
283            for (String var : oldVars.keySet()) {
284                String actionName = var.split(WorkflowInstance.NODE_VAR_SEPARATOR)[0];
285                if (nodesToSkip.contains(actionName)) {
286                    newVars.put(var, oldVars.get(var));
287                }
288            }
289            for (String node : nodesToSkip) {
290                // Setting the TO_SKIP variable to true. This will be used by
291                // SignalCommand and LiteNodeHandler to skip the action.
292                newVars.put(node + WorkflowInstance.NODE_VAR_SEPARATOR + TO_SKIP, "true");
293                String visitedFlag = NodeHandler.getLoopFlag(node);
294                // Removing the visited flag so that the action won't be considered
295                // a loop.
296                if (newVars.containsKey(visitedFlag)) {
297                    newVars.remove(visitedFlag);
298                }
299            }
300            newWfInstance.setAllVars(newVars);
301        }
302    
303        /* (non-Javadoc)
304         * @see org.apache.oozie.command.XCommand#getEntityKey()
305         */
306        @Override
307        protected String getEntityKey() {
308            return this.jobId;
309        }
310    
311        /* (non-Javadoc)
312         * @see org.apache.oozie.command.XCommand#isLockRequired()
313         */
314        @Override
315        protected boolean isLockRequired() {
316            return true;
317        }
318    
319        /* (non-Javadoc)
320         * @see org.apache.oozie.command.XCommand#loadState()
321         */
322        @Override
323        protected void loadState() throws CommandException {
324        }
325    
326        /* (non-Javadoc)
327         * @see org.apache.oozie.command.XCommand#verifyPrecondition()
328         */
329        @Override
330        protected void verifyPrecondition() throws CommandException, PreconditionException {
331        }
332    }