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.action.oozie; 016 017 import org.apache.oozie.client.OozieClientException; 018 import org.apache.oozie.action.ActionExecutor; 019 import org.apache.oozie.action.ActionExecutorException; 020 import org.apache.oozie.DagEngine; 021 import org.apache.oozie.LocalOozieClient; 022 import org.apache.oozie.WorkflowJobBean; 023 import org.apache.oozie.service.DagEngineService; 024 import org.apache.oozie.service.WorkflowAppService; 025 import org.apache.oozie.client.WorkflowAction; 026 import org.apache.oozie.client.OozieClient; 027 import org.apache.oozie.client.WorkflowJob; 028 import org.apache.oozie.command.CommandException; 029 import org.apache.oozie.util.PropertiesUtils; 030 import org.apache.oozie.util.XmlUtils; 031 import org.apache.oozie.util.XConfiguration; 032 import org.apache.oozie.util.XLog; 033 import org.apache.oozie.service.Services; 034 import org.apache.hadoop.conf.Configuration; 035 import org.jdom.Element; 036 import org.jdom.Namespace; 037 038 import java.io.StringReader; 039 import java.io.IOException; 040 import java.util.Set; 041 import java.util.HashSet; 042 043 public class SubWorkflowActionExecutor extends ActionExecutor { 044 public static final String ACTION_TYPE = "sub-workflow"; 045 public static final String LOCAL = "local"; 046 047 private static final Set<String> DISALLOWED_DEFAULT_PROPERTIES = new HashSet<String>(); 048 049 static { 050 String[] badUserProps = {PropertiesUtils.DAYS, PropertiesUtils.HOURS, PropertiesUtils.MINUTES, 051 PropertiesUtils.KB, PropertiesUtils.MB, PropertiesUtils.GB, PropertiesUtils.TB, PropertiesUtils.PB, 052 PropertiesUtils.RECORDS, PropertiesUtils.MAP_IN, PropertiesUtils.MAP_OUT, PropertiesUtils.REDUCE_IN, 053 PropertiesUtils.REDUCE_OUT, PropertiesUtils.GROUPS}; 054 055 String[] badDefaultProps = {PropertiesUtils.HADOOP_USER, PropertiesUtils.HADOOP_UGI, 056 WorkflowAppService.HADOOP_JT_KERBEROS_NAME, WorkflowAppService.HADOOP_NN_KERBEROS_NAME}; 057 PropertiesUtils.createPropertySet(badUserProps, DISALLOWED_DEFAULT_PROPERTIES); 058 PropertiesUtils.createPropertySet(badDefaultProps, DISALLOWED_DEFAULT_PROPERTIES); 059 } 060 061 protected SubWorkflowActionExecutor() { 062 super(ACTION_TYPE); 063 } 064 065 public void initActionType() { 066 super.initActionType(); 067 } 068 069 protected OozieClient getWorkflowClient(Context context, String oozieUri) { 070 OozieClient oozieClient; 071 if (oozieUri.equals(LOCAL)) { 072 WorkflowJobBean workflow = (WorkflowJobBean) context.getWorkflow(); 073 String user = workflow.getUser(); 074 String group = workflow.getGroup(); 075 String authToken = workflow.getAuthToken(); 076 DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(user, authToken); 077 oozieClient = new LocalOozieClient(dagEngine); 078 } 079 else { 080 // TODO we need to add authToken to the WC for the remote case 081 oozieClient = new OozieClient(oozieUri); 082 } 083 return oozieClient; 084 } 085 086 protected void injectInline(Element eConf, Configuration subWorkflowConf) throws IOException, 087 ActionExecutorException { 088 if (eConf != null) { 089 String strConf = XmlUtils.prettyPrint(eConf).toString(); 090 Configuration conf = new XConfiguration(new StringReader(strConf)); 091 try { 092 PropertiesUtils.checkDisallowedProperties(conf, DISALLOWED_DEFAULT_PROPERTIES); 093 } 094 catch (CommandException ex) { 095 throw convertException(ex); 096 } 097 XConfiguration.copy(conf, subWorkflowConf); 098 } 099 } 100 101 @SuppressWarnings("unchecked") 102 protected void injectCallback(Context context, Configuration conf) { 103 String callback = context.getCallbackUrl("$status"); 104 if (conf.get(OozieClient.WORKFLOW_NOTIFICATION_URL) != null) { 105 XLog.getLog(getClass()) 106 .warn("Sub-Workflow configuration has a custom job end notification URI, overriding"); 107 } 108 conf.set(OozieClient.WORKFLOW_NOTIFICATION_URL, callback); 109 } 110 111 protected void injectRecovery(String externalId, Configuration conf) { 112 conf.set(OozieClient.EXTERNAL_ID, externalId); 113 } 114 115 protected String checkIfRunning(OozieClient oozieClient, String extId) throws OozieClientException { 116 String jobId = oozieClient.getJobId(extId); 117 if (jobId.equals("")) { 118 return null; 119 } 120 return jobId; 121 } 122 123 public void start(Context context, WorkflowAction action) throws ActionExecutorException { 124 try { 125 Element eConf = XmlUtils.parseXml(action.getConf()); 126 Namespace ns = eConf.getNamespace(); 127 Element e = eConf.getChild("oozie", ns); 128 String oozieUri = (e == null) ? LOCAL : e.getTextTrim(); 129 OozieClient oozieClient = getWorkflowClient(context, oozieUri); 130 String subWorkflowId = null; 131 String extId = context.getRecoveryId(); 132 String runningJobId = null; 133 if (extId != null) { 134 runningJobId = checkIfRunning(oozieClient, extId); 135 } 136 if (runningJobId == null) { 137 String appPath = eConf.getChild("app-path", ns).getTextTrim(); 138 139 XConfiguration subWorkflowConf = new XConfiguration(); 140 if (eConf.getChild(("propagate-configuration"), ns) != null) { 141 Configuration parentConf = new XConfiguration(new StringReader(context.getWorkflow().getConf())); 142 XConfiguration.copy(parentConf, subWorkflowConf); 143 } 144 145 // the proto has the necessary credentials 146 Configuration protoActionConf = context.getProtoActionConf(); 147 XConfiguration.copy(protoActionConf, subWorkflowConf); 148 subWorkflowConf.set(OozieClient.APP_PATH, appPath); 149 injectInline(eConf.getChild("configuration", ns), subWorkflowConf); 150 injectCallback(context, subWorkflowConf); 151 injectRecovery(extId, subWorkflowConf); 152 153 subWorkflowId = oozieClient.run(subWorkflowConf.toProperties()); 154 } 155 else { 156 subWorkflowId = runningJobId; 157 } 158 WorkflowJob workflow = oozieClient.getJobInfo(subWorkflowId); 159 String consoleUrl = workflow.getConsoleUrl(); 160 context.setStartData(subWorkflowId, oozieUri, consoleUrl); 161 if (runningJobId != null) { 162 check(context, action); 163 } 164 } 165 catch (Exception ex) { 166 throw convertException(ex); 167 } 168 } 169 170 public void end(Context context, WorkflowAction action) throws ActionExecutorException { 171 try { 172 String externalStatus = action.getExternalStatus(); 173 WorkflowAction.Status status = externalStatus.equals("SUCCEEDED") ? WorkflowAction.Status.OK 174 : WorkflowAction.Status.ERROR; 175 context.setEndData(status, getActionSignal(status)); 176 } 177 catch (Exception ex) { 178 throw convertException(ex); 179 } 180 } 181 182 public void check(Context context, WorkflowAction action) throws ActionExecutorException { 183 try { 184 String subWorkflowId = action.getExternalId(); 185 String oozieUri = action.getTrackerUri(); 186 OozieClient oozieClient = getWorkflowClient(context, oozieUri); 187 WorkflowJob subWorkflow = oozieClient.getJobInfo(subWorkflowId); 188 WorkflowJob.Status status = subWorkflow.getStatus(); 189 switch (status) { 190 case FAILED: 191 case KILLED: 192 case SUCCEEDED: 193 context.setExecutionData(status.toString(), null); 194 break; 195 default: 196 context.setExternalStatus(status.toString()); 197 break; 198 } 199 } 200 catch (Exception ex) { 201 throw convertException(ex); 202 } 203 } 204 205 public void kill(Context context, WorkflowAction action) throws ActionExecutorException { 206 try { 207 String subWorkflowId = action.getExternalId(); 208 String oozieUri = action.getTrackerUri(); 209 OozieClient oozieClient = getWorkflowClient(context, oozieUri); 210 oozieClient.kill(subWorkflowId); 211 context.setEndData(WorkflowAction.Status.KILLED, getActionSignal(WorkflowAction.Status.KILLED)); 212 } 213 catch (Exception ex) { 214 throw convertException(ex); 215 } 216 } 217 218 private static Set<String> FINAL_STATUS = new HashSet<String>(); 219 220 static { 221 FINAL_STATUS.add("SUCCEEDED"); 222 FINAL_STATUS.add("KILLED"); 223 FINAL_STATUS.add("FAILED"); 224 } 225 226 public boolean isCompleted(String externalStatus) { 227 return FINAL_STATUS.contains(externalStatus); 228 } 229 }