http://git-wip-us.apache.org/repos/asf/oozie/blob/6a6f2199/docs/src/site/markdown/DG_CoordinatorRerun.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/DG_CoordinatorRerun.md 
b/docs/src/site/markdown/DG_CoordinatorRerun.md
new file mode 100644
index 0000000..f535d16
--- /dev/null
+++ b/docs/src/site/markdown/DG_CoordinatorRerun.md
@@ -0,0 +1,53 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+# Coordinator Rerun
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Pre-Conditions
+
+   * Rerun coordinator action must be in TIMEDOUT/SUCCEEDED/KILLED/FAILED.
+   * Coordinator actions cannot be rerun if the coordinator job is in the PREP 
or IGNORED state.
+   * Rerun a PAUSED coordinator job, the status and pause time and pending 
flag will not be reset.
+   * Rerun a SUSPENDED coordinator job, the status will reset to RUNNING.
+   * All rerun actions must exist already.
+   * Coordinator Rerun will only use the original configs from first run.
+   * Coordinator Rerun will not re-read the coordinator.xml in hdfs.
+
+## Rerun Arguments
+
+
+```
+$oozie job -rerun <coord_Job_id> [-nocleanup] [-refresh] [-failed] [-config 
<arg>]
+[-action 1, 3-4, 7-40] (-action or -date is required to rerun.)
+[-date 2009-01-01T01:00Z::2009-05-31T23:59Z, 2009-11-10T01:00Z, 
2009-12-31T22:00Z]
+(if neither -action nor -date is given, the exception will be thrown.)
+```
+
+   * Either -action or -date should be given.
+   * If -action and -date both are given, an error will be thrown.
+   * Multiple ranges can be used in -action or -date. See the above examples.
+   * If one of action in the given list of -action does not exist or not in 
terminal state, the rerun throws an error.
+   * The dates specified in -date must be UTC.
+   * Single date specified in -date must be able to find an action with 
matched nominal time to be effective.
+   * If -nocleanup is given, coordinator directories will not be removed; 
otherwise the 'output-event' will be deleted, unless nocleanup attribute is 
explicitly set in coordinator.xml
+   * If -refresh is set, new dataset is re-evaluated for latest() and future().
+   * If -refresh is set, all dependencies will be re-checked; otherwise only 
missed dependencies will be checked.
+   * If -failed is set, re-runs the failed workflow actions of the coordinator 
actions.
+   * -config can be used to supply properties to workflow by job configuration 
file '.xml' or '.properties'.
+
+## Rerun coordinator actions
+
+   * Rerun terminated (timeout, succeeded, killed, failed) coordinator actions.
+   * By default, Oozie will delete the 'output-event' directories before 
changing actions' status and materializing actions.
+   * If coordinator job is RUNNING, rerun actions will be materialized and 
compete with current running actions.
+   * Rerun for job, user should use job's start date and end date in -date.
+   * If the user specifies a date range (say Jan 1 to May 1), the actions that 
will be re-run are the existing actions
+     within that range.  If the existing actions are action #5....#40, which 
map to Jan 15 to Feb 15, then only those actions will run.
+   * The rerun action_id and nominal_time of the actions which are eligible to 
rerun will be returned.
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+

http://git-wip-us.apache.org/repos/asf/oozie/blob/6a6f2199/docs/src/site/markdown/DG_CustomActionExecutor.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/DG_CustomActionExecutor.md 
b/docs/src/site/markdown/DG_CustomActionExecutor.md
new file mode 100644
index 0000000..5768b27
--- /dev/null
+++ b/docs/src/site/markdown/DG_CustomActionExecutor.md
@@ -0,0 +1,83 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+# Custom Action Nodes
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Introduction
+Oozie can be extended to support additional action types by writing a custom 
[Action Node](WorkflowFunctionalSpec.html#ActionNodes). Action Nodes can be 
synchronous or asynchronous.
+
+   * Synchronous Node - Sync nodes are executed inline by Oozie, which waits 
for completion of these nodes before proceeding. Hence, these nodes should 
almost never be used and are meant for lightweight tasks like FileSystem move, 
mkdir, delete.
+   * Asynchronous Nodes - Oozie starts asynchronous nodes, and then monitors 
the action being executed for completion. This is done via a callback from the 
action or Oozie polling for the action status.
+
+## Writing a custom Action Node
+Action Executors are configured in the oozie configuration file 
oozie-site.xml. These executors are loaded during Oozie startup. [Deploying a 
Custom Action 
Executor](DG_CustomActionExecutor.html#Deploying_a_custom_Action_Executor).
+
+Action Executors MUST extend the `ActionExecutor` class and override the 
required methods.
+
+Most methods take as argument the Execution Context and the actual Action 
object with various configuration properties resolved.
+### ActionExecutor.Context
+The Execution context gives Action Nodes access to configuration properties, 
methods to set the state of the action, methods to set variables which are to 
be made available later in the execution path.
+
+**The following methods from the ActionExecutor interface should be 
implemented.**
+### Constructor
+A no argument constructor should be implemented, which calls 
super(ACTION_TYPE). ACTION_TYPE is the name of the action which will be used in 
the workflow xml, and is used by Oozie to instantiate the correct type of 
Executor.
+
+### initActionType()
+This method is called once during initialization of the Action Executor during 
Oozie startup. Any common initialization code for the Action Node should go 
here.
+
+As an example, setting up of error handling for the Custom Action should be 
done here.
+
+This method must call super.initActionType() as it's first statement.
+
+### start(ActionExecutor.Context context, Action action)
+The action start up happens here.
+
+   * Async Actions - The action should be started and 
context.setStartData(externalId, trackerUri, consoleUrl) must be set. A check 
can be made for whether the action has completed, in which case 
context.setExecutionData(externalStatus, actionData) must be called.
+   * Sync Actions - The action should be started and should complete 
execution. context.setExecutionData(externalStatus, actionData) must be called.
+### check(ActionExecutor.Context context, Action action)
+`check(...)` is used by Oozie to poll for the status of the action. This 
method should interact with the action started previously, and update the 
status. If the action has completed, context.setExecutionData(externalStatus, 
actionData) must be called. Otherwise, the status can be updated using 
context.setExternalStatus(externalStatus).
+
+For sync actions, this method will not be called, and should throw an 
UnsupportedOperationException().
+### kill(ActionExecutor.Context context, Action action)
+`kill(...)` is called when there is an attempt to kill the running job or 
action. No workflow transition is made after this.
+
+The implementation for a custom action should interact with and kill the 
running action, and take care of any cleanup which may be required. 
context.setEndData(status, signalValue) should be called with both values set 
to Action.Status.KILLED.
+### end(ActionExecutor.Context context, Action action)
+`end(...)` is used for any cleanup or processing which may need to be done 
after completion of the action. After any processing, 
context.setEndData(status, signalValue) should be called to complete execution 
of the action and trigger the next workflow transition. signalValue can be 
Action.Status.OK or Action.Status.ERROR.
+### Registering Errors
+Oozie actions can generate different types of Errors.
+
+   * TRANSIENT - will be retried
+   * NON TRANSIENT - the job will be suspended and can be resumed later by 
human intervention, after fixing whatever problem caused this error.
+   * ERROR - causes the error transition to be taken.
+   * FAILED - the action and the job are set to FAILED state. No transitions 
are taken.
+registerError(exceptionClassName, errorType, errorMessage) can be used to 
register possible exceptions while executing the action, along with their type 
and error message. This will normally be done during initialization of the 
Action Executor.
+
+## Deploying a custom Action Executor
+Action Nodes can be registered in the oozie configuration file oozie-site.xml, 
by changing the property 'oozie.service.ActionService.executor.ext.classes'. 
For multiple Executors, the class name should be separated by commas.
+
+```  <property>
+    <name>oozie.service.ActionService.executor.ext.classes</name>
+    <value>
+      org.apache.oozie.wf.action.decision.CustomActionExecutor,
+         Custom_Action_Executr_2.class
+    </value>
+  </property>
+```
+Any configuration properties to be made available to this class should also be 
added to oozie-site.xml. The convention to be followed for naming these 
properties is 'oozie.action.[ActionName].property.name'
+
+The XML schema (XSD) for the new Actions should be added to oozie-site.xml, 
under the property 'oozie.service.WorkflowSchemaService.ext.schemas'. A comma 
separated list for multiple Action schemas.
+
+The XML schema (XSD) for the new action should be also added to Fluent Job 
API. Please refer to
+[Fluent Job API :: How To 
Extend](DG_FluentJobAPI.html#AE.C_Appendix_C_How_To_Extend) for details.
+
+The executor class should be placed along with the oozie webapp in the correct 
path. Once Oozie is restarted, the custom action node can be used in workflows.
+
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+

http://git-wip-us.apache.org/repos/asf/oozie/blob/6a6f2199/docs/src/site/markdown/DG_DistCpActionExtension.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/DG_DistCpActionExtension.md 
b/docs/src/site/markdown/DG_DistCpActionExtension.md
new file mode 100644
index 0000000..13c2a0a
--- /dev/null
+++ b/docs/src/site/markdown/DG_DistCpActionExtension.md
@@ -0,0 +1,220 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+-----
+
+# Oozie DistCp Action Extension
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## DistCp Action
+
+The `DistCp` action uses Hadoop distributed copy to copy files from one 
cluster to another or within the same cluster.
+
+**IMPORTANT:** The DistCp action may not work properly with all configurations 
(secure, insecure) in all versions
+of Hadoop. For example, distcp between two secure clusters is tested and works 
well. Same is true with two insecure
+clusters. In cases where a secure and insecure clusters are involved, distcp 
will not work.
+
+Both Hadoop clusters have to be configured with proxyuser for the Oozie 
process as explained
+[here](DG_QuickStart.html#HadoopProxyUser) on the Quick Start page.
+
+**Syntax:**
+
+
+```
+<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
+    ...
+    <action name="distcp-example">
+        <distcp xmlns="uri:oozie:distcp-action:1.0">
+            <resource-manager>${resourceManager}</resource-manager>
+            <name-node>${nameNode1}</name-node>
+            <arg>${nameNode1}/path/to/input.txt</arg>
+            <arg>${nameNode2}/path/to/output.txt</arg>
+            </distcp>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
+    ...
+</workflow-app>
+```
+
+The first `arg` indicates the input and the second `arg` indicates the output. 
 In the above example, the input is on `namenode1`
+and the output is on `namenode2`.
+
+**IMPORTANT:** If using the DistCp action between 2 secure clusters, the 
following property must be added to the `configuration` of
+the action:
+
+```
+<property>
+    <name>oozie.launcher.mapreduce.job.hdfs-servers</name>
+    <value>${nameNode1},${nameNode2}</value>
+</property>
+```
+
+The `DistCp` action is also commonly used to copy files within the same 
cluster. Cases where copying files within
+a directory to another directory or directories to target directory is 
supported. Example below will illustrate a
+copy within a cluster, notice the source and target `nameNode` is the same and 
use of `*` syntax is supported to
+represent only child files or directories within a source directory. For the 
sake of the example, `jobTracker` and `resourceManager`
+are synonymous.
+
+**Syntax:**
+
+
+```
+<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
+    ...
+    <action name="copy-example">
+        <distcp xmlns="uri:oozie:distcp-action:1.0">
+            <resource-manager>${resourceManager}</resource-manager>
+            <name-node>${nameNode}</name-node>
+            <arg>${nameNode}/path/to/source/*</arg>
+            <arg>${nameNode}/path/to/target/</arg>
+        </distcp>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
+    ...
+</workflow-app>
+```
+
+## Appendix, DistCp XML-Schema
+
+### AE.A Appendix A, DistCp XML-Schema
+
+#### DistCp Action Schema Version 1.0
+
+```
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema";
+           xmlns:distcp="uri:oozie:distcp-action:1.0" 
elementFormDefault="qualified"
+           targetNamespace="uri:oozie:distcp-action:1.0">
+.
+    <xs:include schemaLocation="oozie-common-1.0.xsd"/>
+.
+    <xs:element name="distcp" type="distcp:ACTION"/>
+.
+    <xs:complexType name="ACTION">
+        <xs:sequence>
+            <xs:choice>
+                <xs:element name="job-tracker" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+                <xs:element name="resource-manager" type="xs:string" 
minOccurs="0" maxOccurs="1"/>
+            </xs:choice>
+            <xs:element name="name-node" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="prepare" type="distcp:PREPARE" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="launcher" type="distcp:LAUNCHER" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="configuration" type="distcp:CONFIGURATION" 
minOccurs="0" maxOccurs="1"/>
+            <xs:element name="java-opts" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="arg" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+.
+</xs:schema>
+```
+
+#### DistCp Action Schema Version 0.2
+
+```
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema";
+           xmlns:distcp="uri:oozie:distcp-action:0.2" 
elementFormDefault="qualified"
+           targetNamespace="uri:oozie:distcp-action:0.2">
+.
+    <xs:element name="distcp" type="distcp:ACTION"/>
+.
+    <xs:complexType name="ACTION">
+        <xs:sequence>
+                <xs:element name="job-tracker" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+                <xs:element name="name-node" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+                <xs:element name="prepare" type="distcp:PREPARE" minOccurs="0" 
maxOccurs="1"/>
+                <xs:element name="configuration" type="distcp:CONFIGURATION" 
minOccurs="0" maxOccurs="1"/>
+                <xs:element name="java-opts" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+                <xs:element name="arg" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="CONFIGURATION">
+        <xs:sequence>
+            <xs:element name="property" minOccurs="1" maxOccurs="unbounded">
+                <xs:complexType>
+                    <xs:sequence>
+                        <xs:element name="name" minOccurs="1" maxOccurs="1" 
type="xs:string"/>
+                        <xs:element name="value" minOccurs="1" maxOccurs="1" 
type="xs:string"/>
+                        <xs:element name="description" minOccurs="0" 
maxOccurs="1" type="xs:string"/>
+                    </xs:sequence>
+                </xs:complexType>
+            </xs:element>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="PREPARE">
+        <xs:sequence>
+            <xs:element name="delete" type="distcp:DELETE" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="mkdir" type="distcp:MKDIR" minOccurs="0" 
maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="DELETE">
+        <xs:attribute name="path" type="xs:string" use="required"/>
+    </xs:complexType>
+.
+    <xs:complexType name="MKDIR">
+        <xs:attribute name="path" type="xs:string" use="required"/>
+    </xs:complexType>
+.
+</xs:schema>
+```
+
+#### DistCp Action Schema Version 0.1
+
+```
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema";
+           xmlns:distcp="uri:oozie:distcp-action:0.1" 
elementFormDefault="qualified"
+           targetNamespace="uri:oozie:distcp-action:0.1">
+.
+    <xs:element name="distcp" type="distcp:ACTION"/>
+.
+    <xs:complexType name="ACTION">
+        <xs:sequence>
+                <xs:element name="job-tracker" type="xs:string" minOccurs="1" 
maxOccurs="1"/>
+                <xs:element name="name-node" type="xs:string" minOccurs="1" 
maxOccurs="1"/>
+                <xs:element name="prepare" type="distcp:PREPARE" minOccurs="0" 
maxOccurs="1"/>
+                <xs:element name="configuration" type="distcp:CONFIGURATION" 
minOccurs="0" maxOccurs="1"/>
+                <xs:element name="java-opts" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+                <xs:element name="arg" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="CONFIGURATION">
+        <xs:sequence>
+            <xs:element name="property" minOccurs="1" maxOccurs="unbounded">
+                <xs:complexType>
+                    <xs:sequence>
+                        <xs:element name="name" minOccurs="1" maxOccurs="1" 
type="xs:string"/>
+                        <xs:element name="value" minOccurs="1" maxOccurs="1" 
type="xs:string"/>
+                        <xs:element name="description" minOccurs="0" 
maxOccurs="1" type="xs:string"/>
+                    </xs:sequence>
+                </xs:complexType>
+            </xs:element>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="PREPARE">
+        <xs:sequence>
+            <xs:element name="delete" type="distcp:DELETE" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="mkdir" type="distcp:MKDIR" minOccurs="0" 
maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="DELETE">
+        <xs:attribute name="path" type="xs:string" use="required"/>
+    </xs:complexType>
+.
+    <xs:complexType name="MKDIR">
+        <xs:attribute name="path" type="xs:string" use="required"/>
+    </xs:complexType>
+.
+</xs:schema>
+```
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+

http://git-wip-us.apache.org/repos/asf/oozie/blob/6a6f2199/docs/src/site/markdown/DG_EmailActionExtension.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/DG_EmailActionExtension.md 
b/docs/src/site/markdown/DG_EmailActionExtension.md
new file mode 100644
index 0000000..1afcbb4
--- /dev/null
+++ b/docs/src/site/markdown/DG_EmailActionExtension.md
@@ -0,0 +1,136 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+-----
+
+# Oozie Email Action Extension
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+<a name="EmailAction"></a>
+## 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), `bcc` addresses (optional), a `subject` 
and a `body`.
+Multiple recipients 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:**
+
+
+```
+<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:0.1">
+    ...
+    <action name="[NODE-NAME]">
+        <email xmlns="uri:oozie:email-action:0.2">
+            <to>[COMMA-SEPARATED-TO-ADDRESSES]</to>
+            <cc>[COMMA-SEPARATED-CC-ADDRESSES]</cc> <!-- cc is optional -->
+            <bcc>[COMMA-SEPARATED-BCC-ADDRESSES]</bcc> <!-- bcc is optional -->
+            <subject>[SUBJECT]</subject>
+            <body>[BODY]</body>
+            <content_type>[CONTENT-TYPE]</content_type> <!-- content_type is 
optional -->
+            <attachment>[COMMA-SEPARATED-HDFS-FILE-PATHS]</attachment> <!-- 
attachment is optional -->
+        </email>
+        <ok to="[NODE-NAME]"/>
+        <error to="[NODE-NAME]"/>
+    </action>
+    ...
+</workflow-app>
+```
+
+The `to` and `cc` and `bcc` commands are used to specify recipients who should 
get the mail. Multiple email recipients
+can be provided using comma-separated values. Providing a `to` command is 
necessary, while the `cc` or `bcc` may
+optionally be used along.
+
+The `subject` and `body` commands are used to specify subject and body of the 
mail.
+From uri:oozie:email-action:0.2 one can also specify mail content type as 
<content_type>text/html</content_type>.
+"text/plain" is default.
+
+The `attachment` is used to attach a file(s) on HDFS to the mail. Multiple 
attachment can be provided using comma-separated values.
+Non fully qualified path is considered as a file on default HDFS. A local file 
cannot be attached.
+
+**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.port` - 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.starttls.enable` - Boolean property that toggles if use 
TLS communication 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).
+   * `oozie.email.attachment.enabled` - Boolean property that toggles if 
configured attachments are to be placed into the emails.
+   (false by default).
+   * `oozie.email.smtp.socket.timeout.ms` - The timeout to apply over all SMTP 
server socket operations (10000ms by default).
+
+**Example:**
+
+
+```
+<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:0.1">
+    ...
+    <action name="an-email">
+        <email xmlns="uri:oozie:email-action:0.1">
+            <to>b...@initech.com,the.other....@initech.com</to>
+            <cc>w...@initech.com</cc>
+            <bcc>yet.another....@initech.com</bcc>
+            <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>
+```
+
+In the above example, an email is sent to 'bob', 'the.other.bob', 'will' (cc), 
yet.another.bob (bcc)
+with the subject and body both containing the workflow ID after substitution.
+
+## AE.A Appendix A, Email XML-Schema
+
+
+```
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema";
+           xmlns:email="uri:oozie:email-action:0.2" 
elementFormDefault="qualified"
+           targetNamespace="uri:oozie:email-action:0.2">
+.
+    <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="bcc" 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:element name="content_type" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="attachment" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+        </xs:sequence>
+    </xs:complexType>
+</xs:schema>
+```
+
+**GMail example to oozie-site.xml**
+
+
+```
+oozie.email.smtp.host=smtp.gmail.com
+oozie.email.smtp.port=587
+oozie.email.from.address=<some email address>
+oozie.email.smtp.auth=true
+oozie.email.smtp.starttls.enable=true
+oozie.email.smtp.username=<Gmail Id>
+oozie.email.smtp.password=<Gmail Pass>
+```
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+

http://git-wip-us.apache.org/repos/asf/oozie/blob/6a6f2199/docs/src/site/markdown/DG_Examples.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/DG_Examples.md 
b/docs/src/site/markdown/DG_Examples.md
new file mode 100644
index 0000000..ff33506
--- /dev/null
+++ b/docs/src/site/markdown/DG_Examples.md
@@ -0,0 +1,204 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+# Oozie Examples
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Command Line Examples
+
+### Setting Up the Examples
+
+Oozie examples are bundled within the Oozie distribution in the 
`oozie-examples.tar.gz` file.
+
+Expanding this file will create an `examples/` directory in the local file 
system.
+
+The `examples/` directory must be copied to the user HOME directory in HDFS:
+
+
+```
+$ hadoop fs -put examples examples
+```
+
+**NOTE:** If an examples directory already exists in HDFS, it must be deleted 
before copying it again. Otherwise files may not be
+copied.
+
+### Running the Examples
+
+For the Streaming and Pig example, the [Oozie Share 
Library](DG_QuickStart.html#OozieShareLib) must be installed in HDFS.
+
+Add Oozie `bin/` to the environment PATH.
+
+The examples assume the ResourceManager is `localhost:8032` and the NameNode 
is `hdfs://localhost:8020`. If the actual
+values are different, the job properties files in the examples directory must 
be edited to the correct values.
+
+The example applications are under the examples/app directory, one directory 
per example. The directory contains the
+application XML file (workflow, or workflow and coordinator), the 
`job.properties` file to submit the job and any JAR
+files the example may need.
+
+The inputs for all examples are in the `examples/input-data/` directory.
+
+The examples create output under the `examples/output-data/${EXAMPLE_NAME}` 
directory.
+
+**Note**: The `job.properties` file needs to be a local file during 
submissions, and not a HDFS path.
+
+**How to run an example application:**
+
+
+```
+$ oozie job -oozie http://localhost:11000/oozie -config 
examples/apps/map-reduce/job.properties -run
+.
+job: 14-20090525161321-oozie-tucu
+```
+
+Check the workflow job status:
+
+
+```
+$ oozie job -oozie http://localhost:11000/oozie -info 
14-20090525161321-oozie-tucu
+.
+.----------------------------------------------------------------------------------------------------------------------------------------------------------------
+Workflow Name :  map-reduce-wf
+App Path      :  hdfs://localhost:8020/user/tucu/examples/apps/map-reduce
+Status        :  SUCCEEDED
+Run           :  0
+User          :  tucu
+Group         :  users
+Created       :  2009-05-26 05:01 +0000
+Started       :  2009-05-26 05:01 +0000
+Ended         :  2009-05-26 05:01 +0000
+Actions
+.----------------------------------------------------------------------------------------------------------------------------------------------------------------
+Action Name             Type        Status     Transition  External Id         
   External Status  Error Code    Start Time              End Time
+.----------------------------------------------------------------------------------------------------------------------------------------------------------------
+mr-node                 map-reduce  OK         end         
job_200904281535_0254  SUCCEEDED        -             2009-05-26 05:01 +0000  
2009-05-26 05:01 +0000
+.----------------------------------------------------------------------------------------------------------------------------------------------------------------
+```
+
+To check the workflow job status via the Oozie web console, with a browser go 
to `http://localhost:11000/oozie`.
+
+To avoid having to provide the `-oozie` option with the Oozie URL with every 
`oozie` command, set `OOZIE_URL` env
+variable to the Oozie URL in the shell environment. For example:
+
+
+```
+$ export OOZIE_URL="http://localhost:11000/oozie";
+$
+$ oozie job -info 14-20090525161321-oozie-tucu
+```
+
+## Java API Example
+
+Oozie provides a [Java Client 
API](./apidocs/org/org/apache/oozie/client/package-summary.html) that simplifies
+integrating Oozie with Java applications. This Java Client API is a 
convenience API to interact with Oozie Web-Services
+API.
+
+The following code snippet shows how to submit an Oozie job using the Java 
Client API.
+
+
+```
+import org.apache.oozie.client.OozieClient;
+import org.apache.oozie.client.WorkflowJob;
+.
+import java.util.Properties;
+.
+    ...
+.
+    // get a OozieClient for local Oozie
+    OozieClient wc = new OozieClient("http://bar:11000/oozie";);
+.
+    // create a workflow job configuration and set the workflow application 
path
+    Properties conf = wc.createConfiguration();
+    conf.setProperty(OozieClient.APP_PATH, 
"hdfs://foo:8020/usr/tucu/my-wf-app");
+.
+    // setting workflow parameters
+    conf.setProperty("resourceManager", "foo:8032");
+    conf.setProperty("inputDir", "/usr/tucu/inputdir");
+    conf.setProperty("outputDir", "/usr/tucu/outputdir");
+    ...
+.
+    // submit and start the workflow job
+    String jobId = wc.run(conf);
+    System.out.println("Workflow job submitted");
+.
+    // wait until the workflow job finishes printing the status every 10 
seconds
+    while (wc.getJobInfo(jobId).getStatus() == Workflow.Status.RUNNING) {
+        System.out.println("Workflow job running ...");
+        Thread.sleep(10 * 1000);
+    }
+.
+    // print the final status of the workflow job
+    System.out.println("Workflow job completed ...");
+    System.out.println(wf.getJobInfo(jobId));
+    ...
+```
+
+## Local Oozie Example
+
+Oozie provides an embedded Oozie implementation,  
[LocalOozie](./apidocs/org/apache/oozie/local/LocalOozie.html) ,
+which is useful for development, debugging and testing of workflow 
applications within the convenience of an IDE.
+
+The code snippet below shows the usage of the `LocalOozie` class. All the 
interaction with Oozie is done using Oozie
+ `OozieClient` Java API, as shown in the previous section.
+
+The examples bundled with Oozie include the complete and running class, 
`LocalOozieExample` from where this snippet was
+taken.
+
+
+```
+import org.apache.oozie.local.LocalOozie;
+import org.apache.oozie.client.OozieClient;
+import org.apache.oozie.client.WorkflowJob;
+.
+import java.util.Properties;
+.
+    ...
+    // start local Oozie
+    LocalOozie.start();
+.
+    // get a OozieClient for local Oozie
+    OozieClient wc = LocalOozie.getClient();
+.
+    // create a workflow job configuration and set the workflow application 
path
+    Properties conf = wc.createConfiguration();
+    conf.setProperty(OozieClient.APP_PATH, 
"hdfs://foo:8020/usr/tucu/my-wf-app");
+.
+    // setting workflow parameters
+    conf.setProperty("resourceManager", "foo:8032");
+    conf.setProperty("inputDir", "/usr/tucu/inputdir");
+    conf.setProperty("outputDir", "/usr/tucu/outputdir");
+    ...
+.
+    // submit and start the workflow job
+    String jobId = wc.run(conf);
+    System.out.println("Workflow job submitted");
+.
+    // wait until the workflow job finishes printing the status every 10 
seconds
+    while (wc.getJobInfo(jobId).getStatus() == WorkflowJob.Status.RUNNING) {
+        System.out.println("Workflow job running ...");
+        Thread.sleep(10 * 1000);
+    }
+.
+    // print the final status of the workflow job
+    System.out.println("Workflow job completed ...");
+    System.out.println(wc.getJobInfo(jobId));
+.
+    // stop local Oozie
+    LocalOozie.stop();
+    ...
+```
+
+Also asynchronous actions like FS action can be used / tested using 
`LocalOozie` / `OozieClient` API. Please see the module
+`oozie-mini` for details like `fs-decision.xml` workflow example.
+
+
+## Fluent Job API Examples
+
+There are some elaborate examples how to use the [Fluent Job 
API](DG_FluentJobAPI.html), under `examples/fluentjob/`. There are two
+simple examples covered under [Fluent Job API :: A Simple 
Example](DG_FluentJobAPI.html#A_Simple_Example) and
+[Fluent Job API :: A More Verbose 
Example](DG_FluentJobAPI.html#A_More_Verbose_Example).
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+

http://git-wip-us.apache.org/repos/asf/oozie/blob/6a6f2199/docs/src/site/markdown/DG_FluentJobAPI.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/DG_FluentJobAPI.md 
b/docs/src/site/markdown/DG_FluentJobAPI.md
new file mode 100644
index 0000000..bd36517
--- /dev/null
+++ b/docs/src/site/markdown/DG_FluentJobAPI.md
@@ -0,0 +1,384 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+# Fluent Job API
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Introduction
+
+Oozie is a mature workflow scheduler system. XML is the standard way of 
defining workflow, coordinator, or bundle jobs.  For users
+who prefer an alternative, the Fluent Job API provides a Java interface 
instead.
+
+### Motivation
+
+Prior to Oozie 5.1.0, the following ways were available to submit a workflow, 
coordinator, or bundle job: through Oozie CLI or via
+HTTP submit a generic workflow, coordinator, or bundle job, or submit a Pig, 
Hive, Sqoop, or MapReduce workflow job.
+
+As the generic way goes, the user has to have uploaded a workflow, 
coordinator, or bundle XML and all necessary dependencies like
+scripts, JAR or ZIP files, to HDFS beforehand, as well as have a 
`job.properties` file at command line and / or provide any
+missing parameters as part of the command.
+
+As the specific Pig, Hive, or Sqoop ways go, the user can provide all 
necessary parameters as part of the command issued. A
+ `workflow.xml` file will be generated with all the necessary details and 
stored to HDFS so that Oozie can grab it. Note that
+dependencies have to be uploaded to HDFS beforehand as well.
+
+There are some usability problems by using the XML job definition. XML is not 
an ideal way to express dependencies and a directed
+acyclic graph (DAG). We have to define a control flow, that is, which action 
follows the actual one. It's also necessary to build
+the whole control flow up front as XML is a declarative language that doesn't 
allow for dynamic evaluation. We have to define also
+boilerplate actions like start and end - those are present in every Oozie 
workflow, still need to explicitly define these.
+
+Apart from boilerplate actions, all the transitions between actions have also 
to be defined and taken care of. Furthermore, multiple
+similar actions cannot inherit common properties from each other. Again, the 
reason being workflows are defined in XML.
+
+Fork and join actions have to be defined in pairs, that is, there shouldn't be 
defined a join those incoming actions do not share
+the same ancestor fork. Such situations would result still in a DAG, but Oozie 
doesn't currently allow that. Note that with Fluent
+Job API new dependencies are introduced automatically when the DAG represented 
by API code couldn't have been expressed as
+fork / join pairs automatically.
+
+Either way, there were no programmatic ways to define workflow jobs. That 
doesn't mean users could not generate XML themselves -
+actually this is something HUE's Oozie UI also tries to target.
+
+### Goals
+
+Fluent Job API aims to solve following from the user's perspective. It 
provides a Java API instead of declarative XML to define
+workflows. It defines dependencies across actions as opposed to defining a 
control flow. This is how data engineers and data
+scientists think. It eliminates all boilerplate actions and transitions. Only 
the necessary bits should be defined.
+
+Multiple similar actions can inherit from each other. In fact, since Fluent 
Job API is programmatic, it's possible to generate
+actions or even workflows using conditional, iterative, or recursive 
structures.
+
+Fluent Job API is backwards compatible with workflows defined as XML. That is, 
it should also be possible to have a Fluent Job API
+workflow rendered as XML, as well as coexist XML based and Fluent Job API 
based workflows in the same Oozie installation at the same
+time all workflow action types. When XSDs change, as few manual steps are 
necessary as possible both on API internal and public
+side.
+
+### Non-goals
+
+The following points are not targeted for the initial release of Fluent Job 
API with Oozie 5.1.0. It doesn't provide API in any
+language other than Java. It doesn't provide a REPL. It doesn't allow for 
dynamic action instantiation depending on e.g. conditional
+logic. That is, using the API users still have to implement the whole workflow 
generation logic in advance.
+
+It has no support for programmatic coordinators and bundles, or even EL 
expressions created by API builders. Note that EL
+expressions for workflows can now be expressed the way these are used in XML 
workflow definitions, as strings in the right places.
+
+At the moment only the transformation from Fluent Job API to workflow 
definition is present. The other direction, from workflow
+definition to Fluent Job API JAR artifact, though sensible, is not supported.
+
+It's based only on latest XSDs. Older XSD versions, as well as conversion 
between XSD versions are not supported. Also no support
+for user-supplied custom actions / XSDs.
+
+Most of the non-goals may be targeted as enhancements of the Fluent Job API 
for future Oozie releases.
+
+### Approach
+
+When using the Fluent Job API, the following points are different from the XML 
jobs definition. Instead of control flow (successor)
+definition, the user can define dependencies (parents of an action).
+
+All boilerplate (start, end, ...) has been eliminated, only nodes having 
useful actions have to be defined.
+
+Control flow and necessary boilerplate are generated automatically by keeping 
user defined dependencies, and possibly introducing
+new dependencies to keep Oozie workflow format of nested fork / join pairs. 
Note that not every dependency DAG can be expressed in
+the Oozie workflow format. When this is not possible, user is notified at 
build time.
+
+## How To Use
+
+### A Simple Example
+
+The simplest thing to create using the Oozie Fluent Job API is a workflow 
consisting of only one action. Let's see how it goes, step
+by step.
+
+First, put the project `org.apache.oozie:oozie-fluent-job-api` to the build 
path. In case of a Maven managed build, create a new
+Maven project and declare a Maven dependency to 
`org.apache.oozie:oozie-fluent-job-api`.
+
+Then, create a class that `implements WorkflowFactory` and implement the 
method `WorkflowFactory#create()`. inside that method,
+create a `ShellAction` using `ShellActionBuilder`, fill in some attributes 
then create a `Workflow` using `WorkflowBuilder` using
+the `ShellAction` just built. Return the `Workflow`.
+
+Compile a Fluent Job API jar that has the `Main-Class` attribute set to the 
`WorkflowFactory` subclass just created,
+e.g. `shell-workflow.jar`.
+
+Moving on, [check via command 
line](DG_CommandLineTool.html#Checking_a_workflow_definition_generated_by_a_Fluent_Job_API_jar_file)
 that
+the compiled API JAR file is valid.
+
+As a finishing touch,
+[run via command 
line](DG_CommandLineTool.html#Running_a_workflow_definition_generated_by_a_Fluent_Job_API_jar_file)
 the Fluent Job API
+workflow.
+
+**For reference, a simplistic API JAR example consisting of a `Workflow` 
having only one `ShellAction`:**
+
+```
+public class MyFirstWorkflowFactory implements WorkflowFactory {
+.
+    @Override
+    public Workflow create() {
+        final ShellAction shellAction = ShellActionBuilder.create()
+                .withName("shell-action")
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withConfigProperty("mapred.job.queue.name", "${queueName}")
+                .withExecutable("echo")
+                .withArgument("my_output=Hello Oozie")
+                .withCaptureOutput(true)
+                .build();
+.
+        final Workflow shellWorkflow = new WorkflowBuilder()
+                .withName("shell-workflow")
+                .withDagContainingNode(shellAction).build();
+.
+        return shellWorkflow;
+    }
+}
+```
+
+**After check, the generated workflow XML looks like this:**
+
+```
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<workflow:workflow-app xmlns:workflow="uri:oozie:workflow:1.0"  
xmlns:shell="uri:oozie:shell-action:1.0" name="shell-workflow">
+.
+    <workflow:start to="parent"/>
+.
+    <workflow:kill name="kill">
+        <workflow:message>Action failed, error 
message[${wf:errorMessage(wf:lastErrorNode())}]</workflow:message>
+    </workflow:kill>
+.
+    <workflow:action name="shell-action">
+        <shell:shell>
+            <shell:resource-manager>${resourceManager}</shell:resource-manager>
+            <shell:name-node>${nameNode}</shell:name-node>
+            <shell:configuration>
+                <shell:property>
+                    <shell:name>mapred.job.queue.name</shell:name>
+                    <shell:value>${queueName}</shell:value>
+                </shell:property>
+            </shell:configuration>
+            <shell:exec>echo</shell:exec>
+            <shell:argument>my_output=Hello Oozie</shell:argument>
+            <shell:capture-output/>
+        </shell:shell>
+        <workflow:ok to="end"/>
+        <workflow:error to="kill"/>
+    </workflow:action>
+.
+    <workflow:end name="end"/>
+.
+</workflow:workflow-app>
+```
+
+
+### A More Verbose Example
+
+**Error handling**
+
+If you would like to provide some error handling in case of action failure, 
you should add an `ErrorHandler` to the `Node`
+representing the action. The error handler action will be added as the 
`"error-transition"` of the original action in the generated
+Oozie workflow XML. Both the `"ok-transition"` and the `"error-transition"` of 
the error handler action itself will lead to an
+autogenerated kill node.
+
+**Here you find an example consisting of a `Workflow` having three 
`ShellAction`s, an error handler `EmailAction`, and one `decision`
+to sort out which way to go:**
+
+```
+public class MySecondWorkflowFactory implements WorkflowFactory {
+.
+    @Override
+    public Workflow create() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withConfigProperty("mapred.job.queue.name", "${queueName}")
+                .withExecutable("echo")
+                .withArgument("my_output=Hello Oozie")
+                .withCaptureOutput(true)
+                
.withErrorHandler(ErrorHandler.buildAsErrorHandler(EmailActionBuilder.create()
+                        .withName("email-on-error")
+                        .withRecipient("someb...@apache.org")
+                        .withSubject("Workflow error")
+                        .withBody("Shell action failed, error 
message[${wf:errorMessage(wf:lastErrorNode())}]")))
+                .build();
+.
+        ShellActionBuilder.createFromExistingAction(parent)
+                .withName("happy-path")
+                .withParentWithCondition(parent, 
"${wf:actionData('parent')['my_output'] eq 'Hello Oozie'}")
+                .withoutArgument("my_output=Hello Oozie")
+                .withArgument("Happy path")
+                .withCaptureOutput(null)
+                .build();
+.
+        ShellActionBuilder.createFromExistingAction(parent)
+                .withName("sad-path")
+                .withParentDefaultConditional(parent)
+                .withArgument("Sad path")
+                .build();
+.
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("shell-example")
+                .withDagContainingNode(parent).build();
+.
+        return workflow;
+    }
+}
+```
+
+**After check, the generated workflow XML looks like this:**
+
+```
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<workflow:workflow-app ... name="shell-example">
+.
+    <workflow:start to="parent"/>
+.
+    <workflow:kill name="kill">
+        <workflow:message>Action failed, error 
message[${wf:errorMessage(wf:lastErrorNode())}]</workflow:message>
+    </workflow:kill>
+.
+    <workflow:action name="email-on-error">
+        <email:email>
+            <email:to>someb...@apache.org</email:to>
+            <email:subject>Workflow error</email:subject>
+            <email:body>Shell action failed, error 
message[${wf:errorMessage(wf:lastErrorNode())}]</email:body>
+        </email:email>
+        <workflow:ok to="kill"/>
+        <workflow:error to="kill"/>
+    </workflow:action>
+.
+    <workflow:action name="parent">
+        <shell:shell>
+            <shell:resource-manager>${resourceManager}</shell:resource-manager>
+            <shell:name-node>${nameNode}</shell:name-node>
+            <shell:configuration>
+                <shell:property>
+                    <shell:name>mapred.job.queue.name</shell:name>
+                    <shell:value>${queueName}</shell:value>
+                </shell:property>
+            </shell:configuration>
+            <shell:exec>echo</shell:exec>
+            <shell:argument>my_output=Hello Oozie</shell:argument>
+            <shell:capture-output/>
+        </shell:shell>
+        <workflow:ok to="decision1"/>
+        <workflow:error to="email-on-error"/>
+    </workflow:action>
+.
+    <workflow:decision name="decision1">
+        <workflow:switch>
+            <workflow:case 
to="happy-path">${wf:actionData('parent')['my_output'] eq 'Hello 
Oozie'}</workflow:case>
+            <workflow:default to="sad-path"/>
+        </workflow:switch>
+    </workflow:decision>
+.
+    <workflow:action name="happy-path">
+        <shell:shell>
+            <shell:resource-manager>${resourceManager}</shell:resource-manager>
+            <shell:name-node>${nameNode}</shell:name-node>
+            <shell:configuration>
+                <shell:property>
+                    <shell:name>mapred.job.queue.name</shell:name>
+                    <shell:value>${queueName}</shell:value>
+                </shell:property>
+            </shell:configuration>
+            <shell:exec>echo</shell:exec>
+            <shell:argument>Happy path</shell:argument>
+        </shell:shell>
+        <workflow:ok to="end"/>
+        <workflow:error to="email-on-error"/>
+    </workflow:action>
+.
+    <workflow:action name="sad-path">
+        <shell:shell>
+            <shell:resource-manager>${resourceManager}</shell:resource-manager>
+            <shell:name-node>${nameNode}</shell:name-node>
+            <shell:configuration>
+                <shell:property>
+                    <shell:name>mapred.job.queue.name</shell:name>
+                    <shell:value>${queueName}</shell:value>
+                </shell:property>
+            </shell:configuration>
+            <shell:exec>echo</shell:exec>
+            <shell:argument>my_output=Hello Oozie</shell:argument>
+            <shell:argument>Sad path</shell:argument>
+            <shell:capture-output/>
+        </shell:shell>
+        <workflow:ok to="end"/>
+        <workflow:error to="email-on-error"/>
+    </workflow:action>
+.
+    <workflow:end name="end"/>
+.
+</workflow:workflow-app>
+```
+
+### Runtime Limitations
+
+Even if Fluent Job API tries to abstract away the task of assembly job 
descriptor XML files, there are some runtime
+limitations apart from the [non-goals 
section](DG_FluentJobAPI.html#Non-goals). All such limitations are based on the 
current
+implementations and subject to further improvements and fixes.
+
+There is only one `kill` possibility in every `workflow`. That is, there can 
be defined only one `action` to be executed just before
+any other `action` turns to be `kill`ed. Furthermore, `kill` goes to `end` 
directly. That means, there cannot be defined an
+intricate network of `kill` nodes, cascading sometimes to other `action` 
nodes, avoiding going to `end` in the first place.
+
+There are places where `decision` node generation fails, throwing an 
`Exception`. The problem is that during the transformation,
+Fluent Job API reaches a state where there is a `fork` that transitions to two 
`decision` nodes, which in turn split into two paths
+each. One of the paths from the first `decision` joins a path from the other 
`decision`, but the remaining conditional paths never
+meet. Therefore, not all paths originating from the `fork` converge to the 
same `join`.
+
+## Appendixes
+
+### AE.A Appendix A, API JAR format
+
+It's kept simple - all the necessary Java class files that are needed are 
packed into a JAR file, that has a `META-INF/MANIFEST.MF`
+with a single entry having the `Main-Class` attribute set to the fully 
qualified name of the entry class, the one that
+`implements WorkflowFactory`:
+
+```
+Main-Class: org.apache.oozie.jobs.api.factory.MyFirstWorkflowFactory
+```
+
+**An example of the command line assembly of such an API JAR:**
+
+```
+jar cfe simple-workflow.jar 
org.apache.oozie.fluentjob.api.factory.MyFirstWorkflowFactory \
+-C /Users/forsage/Workspace/oozie/fluent-job/fluent-job-api/target/classes \
+org/apache/oozie/jobs/api/factory/MyFirstWorkflowFactory.class
+```
+
+### AE.B Appendix B, Some Useful Builder classes
+
+For a complete list of `Builder` classes, please have a look at 
`oozie-fluent-job-api` artifact's following packages:
+
+   * `org.apache.oozie.fluentjob.api.action` - `ActionBuilder` classes
+   * `org.apache.oozie.fluentjob.api.factory` - the single entry point, 
`WorkflowFactory` is here
+   * `org.apache.oozie.fluentjob.api.workflow` - workflow related `Builder` 
classes
+
+On examples how to use these please see `oozie-examples` artifact's 
`org.apache.oozie.example.fluentjob` package.
+
+### AE.C Appendix C, How To Extend
+
+Sometimes there are new XSD versions of an existing custom or core workflow 
action, sometimes it's a new custom workflow action that
+gets introduced. In any case, Fluent Job API needs to keep up with the changes.
+
+Here are the steps needed:
+
+   * in `fluent-job-api/pom.xml` extend or modify `jaxb2-maven-plugin` section 
`sources` by a new `source`
+   * in `fluent-job-api/src/main/xjb/bindings.xml` extend by a new or modify 
an existing `jaxb:bindings`
+   * in `fluent-job-api`, `org.apache.oozie.fluentjob.api.mapping` package, 
introduce a new or modify an existing `DozerConverter`
+   * in `dozer_config.xml`, introduce a new or modify an existing `converter` 
inside `custom-converters`
+   * in `fluent-job-api`, `org.apache.oozie.fluentjob.api.action`, introduce a 
new `Action` and a new `Builder`
+   * write new / modify existing relevant unit and integration tests
+
+### AE.D Appendix D, API compatibility guarantees
+
+Fluent Job API is available beginning version 5.1.0. It's marked 
`@InterfaceAudience.Private` (intended for use in Oozie itself) and
+`@InterfaceStability.Unstable` (no stability guarantees are provided across 
any level of release granularity) to indicate that for
+the next few minor releases it's bound to change a lot.
+
+Beginning from around 5.4.0 planning the next phase, 
`@InterfaceStability.Evolving` (compatibility breaking only between minors),
+and a few minor releases later, `@InterfaceAudience.Public` (safe to use 
outside of Oozie).
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+

http://git-wip-us.apache.org/repos/asf/oozie/blob/6a6f2199/docs/src/site/markdown/DG_HCatalogIntegration.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/DG_HCatalogIntegration.md 
b/docs/src/site/markdown/DG_HCatalogIntegration.md
new file mode 100644
index 0000000..5c592e8
--- /dev/null
+++ b/docs/src/site/markdown/DG_HCatalogIntegration.md
@@ -0,0 +1,147 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+# HCatalog Integration (Since Oozie 4.x)
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## HCatalog Overview
+HCatalog is a table and storage management layer for Hadoop that enables users 
with different data processing
+tools - Pig, MapReduce, and Hive - to more easily read and write data on the 
grid. HCatalog's table abstraction presents
+users with a relational view of data in the Hadoop distributed file system 
(HDFS).
+
+Read [HCatalog 
Documentation](http://incubator.apache.org/hcatalog/docs/r0.5.0/index.html) to 
know more about HCatalog.
+Working with HCatalog using pig is detailed in
+[HCatLoader and 
HCatStorer](http://incubator.apache.org/hcatalog/docs/r0.5.0/loadstore.html).
+Working with HCatalog using MapReduce directly is detailed in
+[HCatInputFormat and 
HCatOutputFormat](http://incubator.apache.org/hcatalog/docs/r0.5.0/inputoutput.html).
+
+### HCatalog notifications
+   HCatalog provides notifications through a JMS provider like ActiveMQ when a 
new partition is added to a table in the
+database. This allows applications to consume those events and schedule the 
work that depends on them. In case of Oozie,
+the notifications are used to determine the availability of HCatalog 
partitions defined as data dependencies in the
+Coordinator and trigger workflows.
+
+Read [HCatalog 
Notification](http://incubator.apache.org/hcatalog/docs/r0.5.0/notification.html)
 to know more about
+notifications in HCatalog.
+
+## Oozie HCatalog Integration
+   Oozie's Coordinators so far have been supporting HDFS directories as a 
input data dependency. When a HDFS URI
+template is specified as a dataset and input events are defined in Coordinator 
for the dataset, Oozie performs data
+availability checks by polling the HDFS directory URIs resolved based on the 
nominal time. When all the data
+dependencies are met, the Coordinator's workflow is triggered which then 
consumes the available HDFS data.
+
+With addition of HCatalog support, Coordinators also support specifying a set 
of HCatalog tables or table partitions as a dataset.
+The workflow is triggered when the HCatalog table partitions are available and 
the workflow actions can then read the
+partition data. A mix of HDFS and HCatalog dependencies can be specified as 
input data dependencies.
+Similar to HDFS directories, HCatalog table partitions can also be specified 
as output dataset events.
+
+With HDFS data dependencies, Oozie has to poll HDFS every time to determine 
the availability of a directory.
+If the HCatalog server is configured to publish partition availability 
notifications to a JMS provider, Oozie can be
+configured to subscribe to it and trigger jobs immediately. This pub-sub model 
reduces pressure on Namenode and also
+cuts down on delays caused by polling intervals.
+
+In the absence of a message bus in the deployment, Oozie will always
+poll the HCatalog server directly for partition availability with the same 
frequency as the HDFS polling. Even when
+subscribed to notifications, Oozie falls back to polling HCatalog server for 
partitions that were available before the
+coordinator action was materialized and to deal with missed notifications due 
to system downtimes. The frequency of the
+fallback polling is usually lower than the constant polling. Defaults are 10 
minutes and 1 minute respectively.
+
+
+### Oozie Server Configuration
+   Refer to [HCatalog Configuration](AG_Install.html#HCatalog_Configuration) 
section of [Oozie Install](AG_Install.html)
+documentation for the Oozie server side configuration required to support 
HCatalog table partitions as a data dependency.
+
+### HCatalog URI Format
+
+Oozie supports specifying HCatalog partitions as a data dependency through a 
URI notation. The HCatalog partition URI is
+used to identify a set of table partitions: 
`hcat://bar:8020/logsDB/logsTable/dt=20090415;region=US`
+
+The format to specify a HCatalog table URI is:
+
+hcat://[metastore server]:[port]/[database name]/[table name]
+
+The format to specify a HCatalog table partition URI is:
+
+hcat://[metastore server]:[port]/[database name]/[table 
name]/[partkey1]=[value];[partkey2]=[value];...
+
+For example,
+
+```
+  <dataset name="logs" frequency="${coord:days(1)}"
+           initial-instance="2009-02-15T08:15Z" timezone="America/Los_Angeles">
+    <uri-template>
+      
hcat://myhcatmetastore:9080/database1/table1/datestamp=${YEAR}${MONTH}${DAY}${HOUR};region=USA
+    </uri-template>
+  </dataset>
+```
+
+Post Oozie-4.3.0 release, Oozie also supports the multiple HCatalog servers in 
the URI. Each of the server needs to be
+separated by single comma (,).
+
+The format to specify a HCatalog table partition URI with multiple HCatalog 
server is:
+
+hcat://[metastore_server]:[port],[metastore_server]:[port]/[database_name]/[table_name]/[partkey1]=[value];[partkey2]=[value];...
+
+For example,
+
+```
+  <dataset name="logs" frequency="${coord:days(1)}"
+           initial-instance="2009-02-15T08:15Z" timezone="America/Los_Angeles">
+    <uri-template>
+      
hcat://myhcatmetastore:9080,myhcatmetastore:9080/database1/table1/datestamp=${YEAR}${MONTH}${DAY}${HOUR};region=USA
+    </uri-template>
+  </dataset>
+```
+
+The regex for parsing the multiple HCatalog URI is exposed via oozie-site.xml, 
So Users can modify if there is any
+requirement. Key for the regex is: `oozie.hcat.uri.regex.pattern`
+
+For example, following has multiple HCatalog URI with multiple HCatalog 
servers. To understand this, Oozie will split them into
+two HCatalog URIs. For splitting the URIs, above mentioned regex is used.
+
+`hcat://hostname1:1000,hcat://hostname2:2000/mydb/clicks/datastamp=12;region=us,scheme://hostname3:3000,scheme://hostname4:4000,scheme://hostname5:5000/db/table/p1=12;p2=us`
+
+After split: (This is internal Oozie mechanism)
+
+`hcat://hostname1:1000,hcat://hostname2:2000/mydb/clicks/datastamp=12;region=us`
+
+`scheme://hostname3:3000,scheme://hostname4:4000,scheme://hostname5:5000/db/table/p1=12;p2=us`
+
+<a name="HCatalogLibraries"></a>
+### HCatalog Libraries
+
+A workflow action interacting with HCatalog requires the following jars in the 
classpath:
+hcatalog-core.jar, hcatalog-pig-adapter.jar, webhcat-java-client.jar, 
hive-common.jar, hive-exec.jar,
+hive-metastore.jar, hive-serde.jar and libfb303.jar.
+hive-site.xml which has the configuration to talk to the HCatalog server also 
needs to be in the classpath. The correct
+version of HCatalog and hive jars should be placed in classpath based on the 
version of HCatalog installed on the cluster.
+
+The jars can be added to the classpath of the action using one of the below 
ways.
+
+   * You can place the jars and hive-site.xml in the system shared library. 
The shared library for a pig, hive or java action can be overridden to include 
hcatalog shared libraries along with the action's shared library. Refer to 
[Shared 
Libraries](WorkflowFunctionalSpec.html#a17_HDFS_Share_Libraries_for_Workflow_Applications_since_Oozie_2.3)
 for more information. The oozie-sharelib-[version].tar.gz in the oozie 
distribution bundles the required HCatalog jars in a hcatalog sharelib. If 
using a different version of HCatalog than the one bundled in the sharelib, 
copy the required HCatalog jars from such version into the sharelib.
+   * You can place the jars and hive-site.xml in the workflow application lib/ 
path.
+   * You can specify the location of the jar files in `archive` tag and the 
hive-site.xml in `file` tag in the corresponding pig, hive or java action.
+
+### Coordinator
+
+Refer to [Coordinator Functional 
Specification](CoordinatorFunctionalSpec.html) for more information about
+
+   * how to specify HCatalog partitions as a data dependency using input 
dataset events
+   * how to specify HCatalog partitions as output dataset events
+   * the various EL functions available to work with HCatalog dataset events 
and how to use them to access HCatalog partitions in pig, hive or java actions 
in a workflow.
+
+### Workflow
+Refer to [Workflow Functional Specification](WorkflowFunctionalSpec.html) for 
more information about
+
+   * how to drop HCatalog table/partitions in the prepare block of a action
+   * the HCatalog EL functions available to use in workflows
+
+Refer to [Action Authentication](DG_ActionAuthentication.html) for more 
information about
+
+   * how to access a secure HCatalog from any action (e.g. hive, pig, etc) in 
a workflow
+
+### Known Issues
+   * When rerunning a coordinator action without specifying -nocleanup option 
if the 'output-event' are hdfs directories, then they are deleted. But if the 
'output-event' is a hcatalog partition, currently the partition is not dropped.
+

http://git-wip-us.apache.org/repos/asf/oozie/blob/6a6f2199/docs/src/site/markdown/DG_Hive2ActionExtension.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/DG_Hive2ActionExtension.md 
b/docs/src/site/markdown/DG_Hive2ActionExtension.md
new file mode 100644
index 0000000..d81ed02
--- /dev/null
+++ b/docs/src/site/markdown/DG_Hive2ActionExtension.md
@@ -0,0 +1,315 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+-----
+
+# Oozie Hive 2 Action Extension
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Hive 2 Action
+
+The `hive2` action runs Beeline to connect to Hive Server 2.
+
+The workflow job will wait until the Hive Server 2 job completes before
+continuing to the next action.
+
+To run the Hive Server 2 job, you have to configure the `hive2` action with 
the `resource-manager`, `name-node`, `jdbc-url`,
+ `password` elements, and either Hive's `script` or `query` element, as well 
as the necessary parameters and configuration.
+
+A `hive2` action can be configured to create or delete HDFS directories
+before starting the Hive Server 2 job.
+
+Oozie EL expressions can be used in the inline configuration. Property
+values specified in the `configuration` element override values specified
+in the `job-xml` file.
+
+As with Hadoop `map-reduce` jobs, it is possible to add files and
+archives in order to make them available to Beeline. Refer to the
+[Adding Files and Archives for the 
Job](WorkflowFunctionalSpec.html#FilesArchives)
+section for more information about this feature.
+
+Oozie Hive 2 action supports Hive scripts with parameter variables, their
+syntax is `${VARIABLES}`.
+
+**Syntax:**
+
+
+```
+<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
+    ...
+    <action name="[NODE-NAME]">
+        <hive2 xmlns="uri:oozie:hive2-action:1.0">
+            <resource-manager>[RESOURCE-MANAGER]</resource-manager>
+            <name-node>[NAME-NODE]</name-node>
+            <prepare>
+               <delete path="[PATH]"/>
+               ...
+               <mkdir path="[PATH]"/>
+               ...
+            </prepare>
+            <job-xml>[HIVE SETTINGS FILE]</job-xml>
+            <configuration>
+                <property>
+                    <name>[PROPERTY-NAME]</name>
+                    <value>[PROPERTY-VALUE]</value>
+                </property>
+                ...
+            </configuration>
+            <jdbc-url>[jdbc:hive2://HOST:10000/default]</jdbc-url>
+            <password>[PASS]</password>
+            <script>[HIVE-SCRIPT]</script>
+            <param>[PARAM-VALUE]</param>
+                ...
+            <param>[PARAM-VALUE]</param>
+            <argument>[ARG-VALUE]</argument>
+                ...
+            <argument>[ARG-VALUE]</argument>
+            <file>[FILE-PATH]</file>
+            ...
+            <archive>[FILE-PATH]</archive>
+            ...
+        </hive2>
+        <ok to="[NODE-NAME]"/>
+        <error to="[NODE-NAME]"/>
+    </action>
+    ...
+</workflow-app>
+```
+
+The `prepare` element, if present, indicates a list of paths to delete
+or create before starting the job. Specified paths must start with 
`hdfs://HOST:PORT`.
+
+The `job-xml` element, if present, specifies a file containing configuration
+for Beeline. Multiple `job-xml` elements are allowed in order to specify 
multiple `job.xml` files.
+
+The `configuration` element, if present, contains configuration
+properties that are passed to the Beeline job.
+
+The `jdbc-url` element must contain the JDBC URL for the Hive Server 2.  
Beeline will use this to know where to connect to.
+
+The `password` element must contain the password of the current user.  
However, the `password` is only used if Hive Server 2 is
+backed by something requiring a password (e.g. LDAP); non-secured Hive Server 
2 or Kerberized Hive Server 2 don't require a password
+so in those cases the `password` is ignored and can be omitted from the action 
XML.  It is up to the user to ensure that a password
+is specified when required.
+
+The `script` element must contain the path of the Hive script to
+execute. The Hive script can be templatized with variables of the form
+`${VARIABLE}`. The values of these variables can then be specified
+using the `params` element.
+
+The `query` element available from uri:oozie:hive2-action:0.2, can be used 
instead of the `script` element. It allows for embedding
+queries within the `worklfow.xml` directly.  Similar to the `script` element, 
it also allows for the templatization of variables
+in the form `${VARIABLE}`.
+
+The `params` element, if present, contains parameters to be passed to
+the Hive script.
+
+The `argument` element, if present, contains arguments to be passed as-is to 
Beeline.
+
+All the above elements can be parameterized (templatized) using EL
+expressions.
+
+**Example:**
+
+
+```
+<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
+    ...
+    <action name="my-hive2-action">
+        <hive2 xmlns="uri:oozie:hive2-action:1.0">
+            <resource-manager>foo:8032</resource-manager>
+            <name-node>bar:8020</name-node>
+            <prepare>
+                <delete path="${jobOutput}"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapred.compress.map.output</name>
+                    <value>true</value>
+                </property>
+            </configuration>
+            <jdbc-url>jdbc:hive2://localhost:10000/default</jdbc-url>
+            <password>foo</password>
+            <script>myscript.q</script>
+            <param>InputDir=/home/rkanter/input-data</param>
+            <param>OutputDir=${jobOutput}</param>
+        </hive2>
+        <ok to="my-other-action"/>
+        <error to="error-cleanup"/>
+    </action>
+    ...
+</workflow-app>
+```
+
+
+### Security
+
+As mentioned above, `password` is only used in cases where Hive Server 2 is 
backed by something requiring a password (e.g. LDAP).
+Non-secured Hive Server 2 and Kerberized Hive Server 2 don't require a 
password so in these cases it can be omitted.
+
+## Appendix, Hive 2 XML-Schema
+
+### AE.A Appendix A, Hive 2 XML-Schema
+
+#### Hive 2 Action Schema Version 1.0
+
+```
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema";
+           xmlns:hive2="uri:oozie:hive2-action:1.0" 
elementFormDefault="qualified"
+           targetNamespace="uri:oozie:hive2-action:1.0">
+.
+    <xs:include schemaLocation="oozie-common-1.0.xsd"/>
+.
+    <xs:element name="hive2" type="hive2:ACTION"/>
+.
+    <xs:complexType name="ACTION">
+        <xs:sequence>
+            <xs:choice>
+                <xs:element name="job-tracker" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+                <xs:element name="resource-manager" type="xs:string" 
minOccurs="0" maxOccurs="1"/>
+            </xs:choice>
+            <xs:element name="name-node" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="prepare" type="hive2:PREPARE" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="launcher" type="hive2:LAUNCHER" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="job-xml" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="configuration" type="hive2:CONFIGURATION" 
minOccurs="0" maxOccurs="1"/>
+            <xs:element name="jdbc-url" type="xs:string" minOccurs="1" 
maxOccurs="1"/>
+            <xs:element name="password" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:choice minOccurs="1" maxOccurs="1">
+                <xs:element name="script" type="xs:string" minOccurs="1" 
maxOccurs="1"/>
+                <xs:element name="query" type="xs:string" minOccurs="1" 
maxOccurs="1"/>
+            </xs:choice>
+            <xs:element name="param" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="argument" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="file" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="archive" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+.
+</xs:schema>
+```
+
+#### Hive 2 Action Schema Version 0.2
+
+```
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema";
+           xmlns:hive2="uri:oozie:hive2-action:0.2" 
elementFormDefault="qualified"
+           targetNamespace="uri:oozie:hive2-action:0.2">
+.
+    <xs:element name="hive2" type="hive2:ACTION"/>
+.
+    <xs:complexType name="ACTION">
+        <xs:sequence>
+            <xs:element name="job-tracker" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="name-node" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="prepare" type="hive2:PREPARE" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="job-xml" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="configuration" type="hive2:CONFIGURATION" 
minOccurs="0" maxOccurs="1"/>
+            <xs:element name="jdbc-url" type="xs:string" minOccurs="1" 
maxOccurs="1"/>
+            <xs:element name="password" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:choice minOccurs="1" maxOccurs="1">
+                <xs:element name="script" type="xs:string" minOccurs="1" 
maxOccurs="1"/>
+                <xs:element name="query"  type="xs:string" minOccurs="1" 
maxOccurs="1"/>
+            </xs:choice>
+            <xs:element name="param" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="argument" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="file" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="archive" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="CONFIGURATION">
+        <xs:sequence>
+            <xs:element name="property" minOccurs="1" maxOccurs="unbounded">
+                <xs:complexType>
+                    <xs:sequence>
+                        <xs:element name="name" minOccurs="1" maxOccurs="1" 
type="xs:string"/>
+                        <xs:element name="value" minOccurs="1" maxOccurs="1" 
type="xs:string"/>
+                        <xs:element name="description" minOccurs="0" 
maxOccurs="1" type="xs:string"/>
+                    </xs:sequence>
+                </xs:complexType>
+            </xs:element>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="PREPARE">
+        <xs:sequence>
+            <xs:element name="delete" type="hive2:DELETE" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="mkdir" type="hive2:MKDIR" minOccurs="0" 
maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="DELETE">
+        <xs:attribute name="path" type="xs:string" use="required"/>
+    </xs:complexType>
+.
+    <xs:complexType name="MKDIR">
+        <xs:attribute name="path" type="xs:string" use="required"/>
+    </xs:complexType>
+.
+</xs:schema>
+```
+
+#### Hive 2 Action Schema Version 0.1
+
+```
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema";
+           xmlns:hive2="uri:oozie:hive2-action:0.1" 
elementFormDefault="qualified"
+           targetNamespace="uri:oozie:hive2-action:0.1">
+.
+    <xs:element name="hive2" type="hive2:ACTION"/>
+.
+    <xs:complexType name="ACTION">
+        <xs:sequence>
+            <xs:element name="job-tracker" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="name-node" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="prepare" type="hive2:PREPARE" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="job-xml" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="configuration" type="hive2:CONFIGURATION" 
minOccurs="0" maxOccurs="1"/>
+            <xs:element name="jdbc-url" type="xs:string" minOccurs="1" 
maxOccurs="1"/>
+            <xs:element name="password" type="xs:string" minOccurs="0" 
maxOccurs="1"/>
+            <xs:element name="script" type="xs:string" minOccurs="1" 
maxOccurs="1"/>
+            <xs:element name="param" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="argument" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="file" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="archive" type="xs:string" minOccurs="0" 
maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="CONFIGURATION">
+        <xs:sequence>
+            <xs:element name="property" minOccurs="1" maxOccurs="unbounded">
+                <xs:complexType>
+                    <xs:sequence>
+                        <xs:element name="name" minOccurs="1" maxOccurs="1" 
type="xs:string"/>
+                        <xs:element name="value" minOccurs="1" maxOccurs="1" 
type="xs:string"/>
+                        <xs:element name="description" minOccurs="0" 
maxOccurs="1" type="xs:string"/>
+                    </xs:sequence>
+                </xs:complexType>
+            </xs:element>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="PREPARE">
+        <xs:sequence>
+            <xs:element name="delete" type="hive2:DELETE" minOccurs="0" 
maxOccurs="unbounded"/>
+            <xs:element name="mkdir" type="hive2:MKDIR" minOccurs="0" 
maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+.
+    <xs:complexType name="DELETE">
+        <xs:attribute name="path" type="xs:string" use="required"/>
+    </xs:complexType>
+.
+    <xs:complexType name="MKDIR">
+        <xs:attribute name="path" type="xs:string" use="required"/>
+    </xs:complexType>
+.
+</xs:schema>
+```
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+

Reply via email to