Skip to content

HTTPS clone URL

Subversion checkout URL

You can clone with
or
.
Download ZIP
Browse files

Closes OOZIE-81 Add an email action to Oozie

  • Loading branch information...
commit f44d4438c4d07f446ea5fe6c352208aa8f479aea 1 parent 1fcf0e4
@QwertyManiac QwertyManiac authored Angelo Kaichen Huang committed
View
2  client/src/main/java/org/apache/oozie/cli/OozieCLI.java
@@ -1187,6 +1187,8 @@ private void validateCommand(CommandLine commandLine) throws OozieCLIException {
List<StreamSource> sources = new ArrayList<StreamSource>();
sources.add(new StreamSource(Thread.currentThread().getContextClassLoader().getResourceAsStream(
"oozie-workflow-0.1.xsd")));
+ sources.add(new StreamSource(Thread.currentThread().getContextClassLoader().getResourceAsStream(
+ "email-action-0.1.xsd")));
SchemaFactory factory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
Schema schema = factory.newSchema(sources.toArray(new StreamSource[sources.size()]));
Validator validator = schema.newValidator();
View
30 client/src/main/resources/email-action-0.1.xsd
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Copyright (c) 2011 Yahoo! Inc. All rights reserved.
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License. See accompanying LICENSE file.
+-->
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
+ xmlns:email="uri:oozie:email-action:0.1" elementFormDefault="qualified"
+ targetNamespace="uri:oozie:email-action:0.1">
+
+ <xs:element name="email" type="email:ACTION"/>
+
+ <xs:complexType name="ACTION">
+ <xs:sequence>
+ <xs:element name="to" type="xs:string" minOccurs="1" maxOccurs="1"/>
+ <xs:element name="cc" type="xs:string" minOccurs="0" maxOccurs="1"/>
+ <xs:element name="subject" type="xs:string" minOccurs="1" maxOccurs="1"/>
+ <xs:element name="body" type="xs:string" minOccurs="1" maxOccurs="1"/>
+ </xs:sequence>
+ </xs:complexType>
+</xs:schema>
View
14 core/pom.xml
@@ -174,7 +174,19 @@
<artifactId>derby</artifactId>
<scope>compile</scope>
</dependency>
-
+
+ <dependency>
+ <groupId>javax.mail</groupId>
+ <artifactId>mail</artifactId>
+ <scope>compile</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>com.icegreen</groupId>
+ <artifactId>greenmail</artifactId>
+ <scope>test</scope>
+ </dependency>
+
</dependencies>
<build>
View
14 core/src/main/conf/oozie-site.xml
@@ -21,6 +21,20 @@
Oozie configuration properties and their default values.
-->
+<!-- Uncomment to enable additional actions (email).
+ <property>
+ <name>oozie.service.ActionService.executor.ext.classes</name>
+ <value>
+ org.apache.oozie.action.email.EmailActionExecutor
+ </value>
+ </property>
+
+ <property>
+ <name>oozie.service.SchemaService.wf.ext.schemas</name>
+ <value>email-action-0.1.xsd</value>
+ </property>
+-->
+
<property>
<name>oozie.system.id</name>
<value>oozie-${user.name}</value>
View
219 core/src/main/java/org/apache/oozie/action/email/EmailActionExecutor.java
@@ -0,0 +1,219 @@
+/**
+ * Copyright (c) 2011 Yahoo! Inc. All rights reserved.
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License. See accompanying LICENSE file.
+ */
+package org.apache.oozie.action.email;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import javax.mail.Authenticator;
+import javax.mail.Message;
+import javax.mail.Message.RecipientType;
+import javax.mail.MessagingException;
+import javax.mail.NoSuchProviderException;
+import javax.mail.PasswordAuthentication;
+import javax.mail.Session;
+import javax.mail.Transport;
+import javax.mail.internet.AddressException;
+import javax.mail.internet.InternetAddress;
+import javax.mail.internet.MimeMessage;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.oozie.action.ActionExecutor;
+import org.apache.oozie.action.ActionExecutorException;
+import org.apache.oozie.action.ActionExecutorException.ErrorType;
+import org.apache.oozie.client.WorkflowAction;
+import org.apache.oozie.util.XmlUtils;
+import org.jdom.Element;
+import org.jdom.Namespace;
+
+/**
+ * Email action executor. It takes to, cc addresses along with a subject and body and sends
+ * out an email.
+ */
+public class EmailActionExecutor extends ActionExecutor {
+
+ private final static String TO = "to";
+ private final static String CC = "cc";
+ private final static String SUB = "subject";
+ private final static String BOD = "body";
+ private final static String COMMA = ",";
+
+ public EmailActionExecutor() {
+ super("email");
+ }
+
+ @Override
+ public void initActionType() {
+ super.initActionType();
+ }
+
+ @Override
+ public void start(Context context, WorkflowAction action) throws ActionExecutorException {
+ try {
+ context.setStartData("-", "-", "-");
+ Element actionXml = XmlUtils.parseXml(action.getConf());
+ validateAndMail(context, actionXml);
+ context.setExecutionData("OK", null);
+ }
+ catch (Exception ex) {
+ throw convertException(ex);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ protected void validateAndMail(Context context, Element element) throws ActionExecutorException {
+ // The XSD does the min/max occurrence validation for us.
+ Namespace ns = Namespace.getNamespace("uri:oozie:email-action:0.1");
+ String tos[] = new String[0];
+ String ccs[] = new String[0];
+ String subject = "";
+ String body = "";
+ Element child = null;
+
+ // <to> - One ought to exist.
+ String text = element.getChildTextTrim(TO, ns);
+ if (text.isEmpty()) {
+ throw new ActionExecutorException(ErrorType.ERROR, "EM001", "No receipents were specified in the to-address field.");
+ }
+ tos = text.split(COMMA);
+
+ // <cc> - Optional, but only one ought to exist.
+ try {
+ ccs = element.getChildTextTrim(CC, ns).split(COMMA);
+ } catch (Exception e) {
+ // It is alright for cc to be given empty or not be present.
+ ccs = new String[0];
+ }
+
+ // <subject> - One ought to exist.
+ subject = element.getChildTextTrim(SUB, ns);
+
+ // <body> - One ought to exist.
+ body = element.getChildTextTrim(BOD, ns);
+
+ // All good - lets try to mail!
+ email(context, tos, ccs, subject, body);
+ }
+
+ protected void email(Context context, String[] to, String[] cc, String subject, String body) throws ActionExecutorException {
+ // Get mailing server details.
+ String smtpHost = context.getProtoActionConf().get("oozie.email.smtp.host");
+ String smtpPort = context.getProtoActionConf().get("oozie.email.smtp.port", "25");
+ Boolean smtpAuth = context.getProtoActionConf().getBoolean("oozie.email.smtp.auth", false);
+ String smtpUser = context.getProtoActionConf().get("oozie.email.smtp.username", "");
+ String smtpPassword = context.getProtoActionConf().get("oozie.email.smtp.password", "");
+ String fromAddr = context.getProtoActionConf().get("oozie.email.from.address");
+
+ Properties properties = new Properties();
+ properties.setProperty("mail.smtp.host", smtpHost);
+ properties.setProperty("mail.smtp.port", smtpPort);
+ properties.setProperty("mail.smtp.auth", smtpAuth.toString());
+
+ Session session;
+ if (!smtpAuth) {
+ session = Session.getDefaultInstance(properties);
+ } else {
+ session = Session.getDefaultInstance(properties, new JavaMailAuthenticator(smtpUser, smtpPassword));
+ }
+
+ Message message = new MimeMessage(session);
+ InternetAddress from;
+ List<InternetAddress> toAddrs = new ArrayList<InternetAddress>(to.length);
+ List<InternetAddress> ccAddrs = new ArrayList<InternetAddress>(cc.length);
+
+ try {
+ from = new InternetAddress(fromAddr);
+ message.setFrom(from);
+ } catch (AddressException e) {
+ throw new ActionExecutorException(ErrorType.ERROR, "EM002", "Bad from address specified in ${oozie.email.from.address}.", e);
+ } catch (MessagingException e) {
+ throw new ActionExecutorException(ErrorType.ERROR, "EM003", "Error setting a from address in the message.", e);
+ }
+
+ try {
+ // Add all <to>
+ for (String toStr : to) {
+ toAddrs.add(new InternetAddress(toStr.trim()));
+ }
+ message.addRecipients(RecipientType.TO, toAddrs.toArray(new InternetAddress[0]));
+
+ // Add all <cc>
+ for (String ccStr : cc) {
+ ccAddrs.add(new InternetAddress(ccStr.trim()));
+ }
+ message.addRecipients(RecipientType.CC, ccAddrs.toArray(new InternetAddress[0]));
+
+ // Set subject, and plain-text body.
+ message.setSubject(subject);
+ message.setContent(body, "text/plain");
+ } catch (AddressException e) {
+ throw new ActionExecutorException(ErrorType.ERROR, "EM004", "Bad address format in <to> or <cc>.", e);
+ } catch (MessagingException e) {
+ throw new ActionExecutorException(ErrorType.ERROR, "EM005", "An error occured while adding recipients.", e);
+ }
+
+ try {
+ // Send over SMTP Transport
+ // (Session+Message has adequate details.)
+ Transport.send(message);
+ } catch (NoSuchProviderException e) {
+ throw new ActionExecutorException(ErrorType.ERROR, "EM006", "Could not find an SMTP transport provider to email.", e);
+ } catch (MessagingException e) {
+ throw new ActionExecutorException(ErrorType.ERROR, "EM007", "Encountered an error while sending the email message over SMTP.", e);
+ }
+ }
+
+ @Override
+ public void end(Context context, WorkflowAction action) throws ActionExecutorException {
+ String externalStatus = action.getExternalStatus();
+ WorkflowAction.Status status = externalStatus.equals("OK") ? WorkflowAction.Status.OK :
+ WorkflowAction.Status.ERROR;
+ context.setEndData(status, getActionSignal(status));
+ }
+
+ @Override
+ public void check(Context context, WorkflowAction action)
+ throws ActionExecutorException {
+
+ }
+
+ @Override
+ public void kill(Context context, WorkflowAction action)
+ throws ActionExecutorException {
+
+ }
+
+ @Override
+ public boolean isCompleted(String externalStatus) {
+ return true;
+ }
+
+ private static class JavaMailAuthenticator extends Authenticator {
+
+ String user;
+ String password;
+
+ public JavaMailAuthenticator(String user, String password) {
+ this.user = user;
+ this.password = password;
+ }
+
+ @Override
+ protected PasswordAuthentication getPasswordAuthentication() {
+ return new PasswordAuthentication(user, password);
+ }
+ }
+}
View
153 core/src/test/java/org/apache/oozie/action/email/TestEmailActionExecutor.java
@@ -0,0 +1,153 @@
+/**
+ * Copyright (c) 2011 Yahoo! Inc. All rights reserved.
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License. See accompanying LICENSE file.
+ */
+package org.apache.oozie.action.email;
+
+import org.apache.oozie.WorkflowActionBean;
+import org.apache.oozie.WorkflowJobBean;
+import org.apache.oozie.action.hadoop.ActionExecutorTestCase;
+import org.apache.oozie.service.WorkflowAppService;
+import org.apache.oozie.util.XConfiguration;
+import org.apache.oozie.util.XmlUtils;
+import org.jdom.Element;
+import org.jdom.JDOMException;
+
+import com.icegreen.greenmail.util.GreenMail;
+import com.icegreen.greenmail.util.GreenMailUtil;
+
+public class TestEmailActionExecutor extends ActionExecutorTestCase {
+
+ // GreenMail helps unit test with functional in-memory mail servers.
+ GreenMail server;
+
+ @Override
+ protected void setUp() throws Exception {
+ super.setUp();
+ server = new GreenMail();
+ server.start();
+ }
+
+ @Override
+ protected void setSystemProps() {
+ super.setSystemProps();
+ setSystemProperty("oozie.service.ActionService.executor.classes", EmailActionExecutor.class.getName());
+ }
+
+ private Context createNormalContext(String actionXml) throws Exception {
+ EmailActionExecutor ae = new EmailActionExecutor();
+
+ XConfiguration protoConf = new XConfiguration();
+ protoConf.set(WorkflowAppService.HADOOP_USER, getTestUser());
+ protoConf.set(WorkflowAppService.HADOOP_UGI, getTestUser() + "," + getTestGroup());
+ protoConf.setInt("oozie.email.smtp.port", server.getSmtp().getPort());
+ protoConf.set("oozie.email.smtp.host", "localhost");
+ protoConf.set("oozie.email.from.address", "test@oozie.com");
+ injectKerberosInfo(protoConf);
+
+ WorkflowJobBean wf = createBaseWorkflow(protoConf, "email-action");
+ WorkflowActionBean action = (WorkflowActionBean) wf.getActions().get(0);
+ action.setType(ae.getType());
+ action.setConf(actionXml);
+
+ return new Context(wf, action);
+ }
+
+ private Context createAuthContext(String actionXml) throws Exception {
+ Context ctx = createNormalContext(actionXml);
+ ctx.getProtoActionConf().setBoolean("oozie.email.smtp.auth", true);
+ ctx.getProtoActionConf().set("oozie.email.smtp.username", "test@oozie.com");
+ ctx.getProtoActionConf().set("oozie.email.smtp.password", "oozie");
+ return ctx;
+ }
+
+ private Element prepareEmailElement(Boolean ccs) throws JDOMException {
+ StringBuilder elem = new StringBuilder();
+ elem.append("<email xmlns=\"uri:oozie:email-action:0.1\">");
+ elem.append("<to> abc@oozie.com, def@oozie.com </to>");
+ if (ccs) {
+ elem.append("<cc>ghi@oozie.com,jkl@oozie.com</cc>");
+ }
+ elem.append("<subject>sub</subject>");
+ elem.append("<body>bod</body>");
+ elem.append("</email>");
+ return XmlUtils.parseXml(elem.toString());
+ }
+
+ private Element prepareBadElement(String elem) throws JDOMException {
+ Element good = prepareEmailElement(true);
+ good.getChild("email").addContent(new Element(elem));
+ return good;
+ }
+
+ public void testSetupMethods() {
+ EmailActionExecutor email = new EmailActionExecutor();
+ assertEquals("email", email.getType());
+ }
+
+ public void testDoNormalEmail() throws Exception {
+ EmailActionExecutor email = new EmailActionExecutor();
+ email.validateAndMail(createNormalContext("email-action"), prepareEmailElement(false));
+ assertEquals("bod", GreenMailUtil.getBody(server.getReceivedMessages()[0]));
+ }
+
+ public void testDoAuthEmail() throws Exception {
+ EmailActionExecutor email = new EmailActionExecutor();
+ email.validateAndMail(createAuthContext("email-action"), prepareEmailElement(true));
+ assertEquals("bod", GreenMailUtil.getBody(server.getReceivedMessages()[0]));
+ }
+
+ public void testValidation() throws Exception {
+ EmailActionExecutor email = new EmailActionExecutor();
+
+ Context ctx = createNormalContext("email-action");
+
+ // Multiple <to>s
+ try {
+ email.validateAndMail(ctx, prepareBadElement("to"));
+ fail();
+ } catch (Exception e) {
+ // Validation succeeded.
+ }
+
+ // Multiple <cc>s
+ try {
+ email.validateAndMail(ctx, prepareBadElement("cc"));
+ fail();
+ } catch (Exception e) {
+ // Validation succeeded.
+ }
+
+ // Multiple <subject>s
+ try {
+ email.validateAndMail(ctx, prepareBadElement("subject"));
+ fail();
+ } catch (Exception e) {
+ // Validation succeeded.
+ }
+
+ // Multiple <body>s
+ try {
+ email.validateAndMail(ctx, prepareBadElement("body"));
+ fail();
+ } catch (Exception e) {
+ // Validation succeeded.
+ }
+ }
+
+ @Override
+ protected void tearDown() throws Exception {
+ super.tearDown();
+ server.stop();
+ }
+}
View
103 docs/src/site/twiki/DG_EmailActionExtension.twiki
@@ -0,0 +1,103 @@
+<noautolink>
+
+[[index][::Go back to Oozie Documentation Index::]]
+
+-----
+
+---+!! Oozie Email Action Extension
+
+%TOC%
+
+#EmailAction
+---++++ 3.2.4 Email action
+
+The =email= action allows sending emails in Oozie from a workflow application. An email action must provide =to=
+addresses, =cc= addresses (optional), a =subject= and a =body=. Multiple reciepents of an email can be provided
+as comma separated addresses.
+
+The email action is executed synchronously, and the workflow job will wait until the specified
+emails are sent before continuing to the next action.
+
+All values specified in the =email= action can be parameterized (templatized) using EL expressions.
+
+*Syntax:*
+
+<verbatim>
+<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:0.1">
+ ...
+ <action name="[NODE-NAME]">
+ <email xmlns="uri:oozie:email-action:0.1">
+ <to>[COMMA-SEPARATED-TO-ADDRESSES]</to>
+ <cc>[COMMA-SEPARATED-CC-ADDRESSES]</cc> <!-- cc is optional -->
+ <subject>[SUBJECT]</subject>
+ <body>[BODY]</body>
+ </email>
+ <ok to="[NODE-NAME]"/>
+ <error to="[NODE-NAME]"/>
+ </action>
+ ...
+</workflow-app>
+</verbatim>
+
+The =to= and =cc= commands are used to specify reciepents who should get the mail. Multiple email reciepents can be provided
+using comma-separated values. Providing a =to= command is necessary, while the =cc= may optionally be used along.
+
+The =subject= and =body= commands are used to specify the plain-text subject and body of the mail.
+
+*Configuration*
+
+The =email= action requires some SMTP server configuration to be present (in oozie-site.xml). The following are the values
+it looks for:
+
+=oozie.email.smtp.host= - The host where the email action may find the SMTP server (localhost by default).
+=oozie.email.smtp.host= - The port to connect to for the SMTP server (25 by default).
+=oozie.email.from.address= - The from address to be used for mailing all emails (oozie@localhost by default).
+=oozie.email.smtp.auth= - Boolean property that toggles if authentication is to be done or not. (false by default).
+=oozie.email.smtp.username= - If authentication is enabled, the username to login as (empty by default).
+=oozie.email.smtp.password= - If authentication is enabled, the username's password (empty by default).
+
+*Example:*
+
+<verbatim>
+<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:0.1">
+ ...
+ <action name="an-email">
+ <email xmlns="uri:oozie:email-action:0.1">
+ <to>bob@initech.com,the.other.bob@initech.com</to>
+ <cc>will@initech.com</cc>
+ <subject>Email notifications for ${wf:id()}</subject>
+ <body>The wf ${wf:id()} successfully completed.</body>
+ </email>
+ <ok to="myotherjob"/>
+ <error to="errorcleanup"/>
+ </action>
+ ...
+</workflow-app>
+</verbatim>
+
+In the above example, an email is sent to 'bob', 'the.other.bob' and 'will' (cc)
+with the subject and body both containing the workflow ID after substitution.
+
+---+++ AE.A Appendix A, Email XML-Schema
+
+<verbatim>
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
+ xmlns:hive="uri:oozie:email-action:0.1" elementFormDefault="qualified"
+ targetNamespace="uri:oozie:email-action:0.1">
+.
+ <xs:element name="email" type="email:ACTION"/>
+.
+ <xs:complexType name="ACTION">
+ <xs:sequence>
+ <xs:element name="to" type="xs:string" minOccurs="1" maxOccurs="1"/>
+ <xs:element name="cc" type="xs:string" minOccurs="0" maxOccurs="1"/>
+ <xs:element name="subject" type="xs:string" minOccurs="1" maxOccurs="1"/>
+ <xs:element name="body" type="xs:string" minOccurs="1" maxOccurs="1"/>
+ </xs:sequence>
+ </xs:complexType>
+</xs:schema>
+</verbatim>
+
+[[index][::Go back to Oozie Documentation Index::]]
+
+</noautolink>
View
1  docs/src/site/twiki/index.twiki
@@ -45,6 +45,7 @@ Enough reading already? Follow the steps in [[DG_QuickStart][Oozie Quick Start]]
* [[DG_CommandLineTool][Command Line Tool]]
* [[DG_WorkflowReRun][Workflow Re-runs Explained]]
* [[DG_UsingHadoopKerberos][Using a Hadoop cluster with Kerberos Authentication]]
+ * [[DG_EmailActionExtension][Email Action]]
* [[DG_CustomActionExecutor][Writing a Custom Action Executor]]
* [[./client/apidocs/index.html][Oozie Client Javadocs]]
View
12 pom.xml
@@ -427,6 +427,18 @@
<version>10.6.1.0</version>
<scope>compile</scope>
</dependency>
+
+ <dependency>
+ <groupId>javax.mail</groupId>
+ <artifactId>mail</artifactId>
+ <version>1.4</version>
+ </dependency>
+
+ <dependency>
+ <groupId>com.icegreen</groupId>
+ <artifactId>greenmail</artifactId>
+ <version>1.3.1b</version>
+ </dependency>
</dependencies>
</dependencyManagement>
View
1  release-log.txt
@@ -1,5 +1,6 @@
-- Oozie 3.1.0 release
+OOZIE-81 Add an email action to Oozie
OOZIE-113 merge changes for OOZIE-101 to ActionEndXCommand
OOZIE-112 workflow kill node message is not resolved and set it to action error message
OOZIE-111 adjust configuration for DBCP data source
Please sign in to comment.
Something went wrong with that request. Please try again.