OOZIE-2734 [docs] Switch from TWiki to Markdown (asalamon74 via andras.piros, pbacsko, gezapeti)
diff --git a/README.txt b/README.txt
index 8cc0f59..f9c6bdf 100644
--- a/README.txt
+++ b/README.txt
@@ -42,7 +42,7 @@
http://oozie.apache.org/
Oozie Quick Start:
-http://oozie.apache.org/docs/4.3.0/DG_QuickStart.html
+http://oozie.apache.org/docs/5.0.0/DG_QuickStart.html
Supported Hadoop Versions:
@@ -50,7 +50,7 @@
This version of Oozie was primarily tested against Hadoop 2.4.x and 2.6.x.
---------------------------------------
+
If you have any questions/issues, please send an email to:
diff --git a/docs/pom.xml b/docs/pom.xml
index ae2e94c..35b868f 100644
--- a/docs/pom.xml
+++ b/docs/pom.xml
@@ -78,7 +78,7 @@
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-project-info-reports-plugin</artifactId>
<!-- Using version because plugin management does not work for reporting plugins -->
- <version>2.2</version>
+ <version>2.9</version>
<reportSets>
<reportSet>
<reports>
@@ -89,7 +89,12 @@
</plugins>
</reporting>
<build>
- <plugins>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-project-info-reports-plugin</artifactId>
+ <version>2.9</version>
+ </plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-site-plugin</artifactId>
@@ -99,16 +104,19 @@
</configuration>
<dependencies>
<dependency>
- <!-- Customized Doxia Maven Plugin for twiki documentation -->
- <groupId>org.apache.maven.doxia</groupId>
- <artifactId>doxia-module-twiki</artifactId>
- <version>1.0-alpha-9.2y</version>
+ <groupId>org.apache.velocity</groupId>
+ <artifactId>velocity</artifactId>
+ <version>1.5</version>
</dependency>
<dependency>
- <!-- Customized Doxia Maven Plugin for twiki documentation -->
+ <groupId>org.apache.maven.doxia</groupId>
+ <artifactId>doxia-module-markdown</artifactId>
+ <version>1.8</version>
+ </dependency>
+ <dependency>
<groupId>org.apache.maven.doxia</groupId>
<artifactId>doxia-core</artifactId>
- <version>1.0-alpha-9.2y</version>
+ <version>1.7</version>
</dependency>
</dependencies>
<executions>
diff --git a/docs/src/site/markdown/AG_OozieLogging.md b/docs/src/site/markdown/AG_OozieLogging.md
new file mode 100644
index 0000000..873e9e3
--- /dev/null
+++ b/docs/src/site/markdown/AG_OozieLogging.md
@@ -0,0 +1,89 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+# Oozie Logging
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Default Oozie Logging
+
+Oozie's logging properties can be configured in its log4j properties file (default is `oozie-log4j.properties`). Most log messages
+are configured by default to be written to the `oozie` appender.
+
+The default configuration for the `oozie` appender is shown below.
+
+### Default Configuration
+
+
+```
+log4j.appender.oozie=org.apache.log4j.rolling.RollingFileAppender
+log4j.appender.oozie.RollingPolicy=org.apache.oozie.util.OozieRollingPolicy
+log4j.appender.oozie.File=${oozie.log.dir}/oozie.log
+log4j.appender.oozie.Append=true
+log4j.appender.oozie.layout=org.apache.log4j.PatternLayout
+log4j.appender.oozie.layout.ConversionPattern=%d{ISO8601} %5p %c{1}:%L - %m%n
+log4j.appender.oozie.RollingPolicy.FileNamePattern=${log4j.appender.oozie.File}-%d{yyyy-MM-dd-HH}
+log4j.appender.oozie.RollingPolicy.MaxHistory=720
+```
+
+In this configuration, the active log file will be named `oozie.log` and all old log files will be named `oozie.log-yyyy-MM-dd-HH`
+(where `yyyy-MM-dd-HH` is the time that that log file was created; e.g. 2012-07-21-05). All log files are in the same directory
+(whatever `oozie.log.dir` is assigned to). A maximum of 720 older log files will be retained. The active log file is rolled every
+hour, so 720 old logs means that they are kept for 30 days before being deleted.
+
+To keep all old logs instead of deleting them, `log4j.appender.oozie.RollingPolicy.MaxHistory` can be set to `-1`.
+Additionally, `log4j.appender.oozie.RollingPolicy` can be set to `org.apache.log4j.rolling.TimeBasedRollingPolicy`, which has the
+same exact behavior as `org.apache.oozie.util.OozieRollingPolicy` except that it does not delete old logs.
+
+### Restrictions
+
+In order for Oozie logging to work 100% correctly, the following restrictions must be observed (described below and in
+the `oozie-log4j.properties` file):
+
+* The appender that Oozie uses must be named "oozie" (i.e. `log4j.appender.oozie`)
+
+* `log4j.appender.oozie.RollingPolicy.FileNamePattern` must end with "-%d{yyyy-MM-dd-HH}.gz" or "-%d{yyyy-MM-dd-HH}".
+If it ends with ".gz" the old logs will be compressed when rolled
+
+* `log4j.appender.oozie.RollingPolicy.FileNamePattern` must start with the value of `log4j.appender.oozie.File`
+
+## Previous Default Oozie Logging
+
+Oozie previously used the logging configuration shown below as the default for the `oozie` appender. The other appender that Oozie
+writes to still use a configuration similar to this.
+
+### Previous Default Configuration
+
+
+```
+log4j.appender.oozie=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.oozie.File=${oozie.log.dir}/oozie.log
+log4j.appender.oozie.Append=true
+log4j.appender.oozie.layout=org.apache.log4j.PatternLayout
+log4j.appender.oozie.layout.ConversionPattern=%d{ISO8601} %5p %c{1}:%L - %m%n
+log4j.appender.oozie.DatePattern='.'yyyy-MM-dd-HH
+```
+
+In this configuration, the active log file will be named `oozie.log` and all old log files will be named `oozie.log.yyyy-MM-dd-HH`
+(where `yyyy-MM-dd-HH` is the time that the log file was created; e.g. 2012-07-21-05). All log files are in the same directory
+(whatever `oozie.log.dir` is assigned to). All older log files are retained. The active log file is rolled every hour.
+
+### Restrictions
+
+In order for Oozie logging to work 100% correctly, the following restrictions must be observed (described below and in the
+`oozie-log4j.properties` file):
+
+* The appender that Oozie uses must be named "oozie" (i.e. `log4j.appender.oozie`)
+
+* `log4j.appender.oozie.DatePattern` must end with either "dd" or "HH". If it ends with "HH", the log will be rolled every hour;
+if it ends with "dd", the log will be rolled every day.
+
+## Other Oozie Logging
+
+While Oozie can technically use any valid log4j Appender or configurations that violate the above restrictions, certain
+features related to logs may be disabled and/or not work correctly, and is thus not advised.
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+
diff --git a/docs/src/site/markdown/DG_QuickStart.md b/docs/src/site/markdown/DG_QuickStart.md
new file mode 100644
index 0000000..64f67ce
--- /dev/null
+++ b/docs/src/site/markdown/DG_QuickStart.md
@@ -0,0 +1,240 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+# Oozie Quick Start
+
+These instructions install and run Oozie using an embedded Jetty server and an embedded Derby database.
+
+For detailed install and configuration instructions refer to [Oozie Install](AG_Install.html).
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Building Oozie
+
+### System Requirements:
+ * Unix box (tested on Mac OS X and Linux)
+ * Java JDK 1.8+
+ * Maven 3.0.1+
+ * Hadoop 2.6.0+
+ * Pig 0.10.1+
+
+JDK commands (java, javac) must be in the command path.
+
+The Maven command (mvn) must be in the command path.
+
+### Building Oozie
+
+Download a source distribution of Oozie from the "Releases" drop down menu on the [Oozie site](http://oozie.apache.org).
+
+Expand the source distribution `tar.gz` and change directories into it.
+
+The simplest way to build Oozie is to run the `mkdistro.sh` script:
+
+```
+$ bin/mkdistro.sh [-DskipTests]
+```
+
+
+Running `mkdistro.sh` will create the binary distribution of Oozie. By default, oozie war will not contain hadoop and
+hcatalog libraries, however they are required for oozie to work. There are 2 options to add these libraries:
+
+1. At install time, copy the hadoop and hcatalog libraries to libext and run oozie-setup.sh to setup Oozie. This is
+ suitable when same oozie package needs to be used in multiple set-ups with different hadoop/hcatalog versions.
+
+2. Build with -Puber which will bundle the required libraries in the oozie war. Further, the following options are
+ available to customise the versions of the dependencies:
+ ```
+ -Dhadoop.version=<version> - default 2.6.0
+ -Ptez - Bundle tez jars in hive and pig sharelibs. Useful if you want to use tez
+ +as the execution engine for those applications.
+ -Dpig.version=<version> - default 0.16.0
+ -Dpig.classifier=<classifier> - default h2
+ -Dsqoop.version=<version> - default 1.4.3
+ -Dsqoop.classifier=<classifier> - default hadoop100
+ -Djetty.version=<version> - default 9.3.20.v20170531
+ -Dopenjpa.version=<version> - default 2.2.2
+ -Dxerces.version=<version> - default 2.10.0
+ -Dcurator.version=<version> - default 2.5.0
+ -Dhive.version=<version - default 1.2.0
+ -Dhbase.version=<version> - default 1.2.3
+ -Dtez.version=<version> - default 0.8.4
+
+ *IMPORTANT:* Profile hadoop-3 must be activated if building against Hadoop 3
+ ```
+
+ More details on building Oozie can be found on the [Building Oozie](ENG_Building.html) page.
+
+## Server Installation
+
+### System Requirements
+
+ * Unix (tested in Linux and Mac OS X)
+ * Java 1.8+
+ * Hadoop
+ * [Apache Hadoop](http://hadoop.apache.org) (tested with 1.2.1 & 2.6.0+)
+ * ExtJS library (optional, to enable Oozie webconsole)
+ * [ExtJS 2.2](http://archive.cloudera.com/gplextras/misc/ext-2.2.zip)
+
+The Java 1.8+ `bin` directory should be in the command path.
+
+### Server Installation
+
+**IMPORTANT:** Oozie ignores any set value for `OOZIE_HOME`, Oozie computes its home automatically.
+
+ * Build an Oozie binary distribution
+ * Download a Hadoop binary distribution
+ * Download ExtJS library (it must be version 2.2)
+
+**NOTE:** The ExtJS library is not bundled with Oozie because it uses a different license.
+
+**NOTE:** Oozie UI browser compatibility Chrome (all), Firefox (3.5), Internet Explorer (8.0), Opera (10.5).
+
+**NOTE:** It is recommended to use a Oozie Unix user for the Oozie server.
+
+Expand the Oozie distribution `tar.gz`.
+
+Expand the Hadoop distribution `tar.gz` (as the Oozie Unix user).
+
+<a name="HadoopProxyUser"></a>
+
+**NOTE:** Configure the Hadoop cluster with proxyuser for the Oozie process.
+
+The following two properties are required in Hadoop core-site.xml:
+
+
+```
+ <!-- OOZIE -->
+ <property>
+ <name>hadoop.proxyuser.[OOZIE_SERVER_USER].hosts</name>
+ <value>[OOZIE_SERVER_HOSTNAME]</value>
+ </property>
+ <property>
+ <name>hadoop.proxyuser.[OOZIE_SERVER_USER].groups</name>
+ <value>[USER_GROUPS_THAT_ALLOW_IMPERSONATION]</value>
+ </property>
+```
+
+Replace the capital letter sections with specific values and then restart Hadoop.
+
+The ExtJS library is optional (only required for the Oozie web-console to work)
+
+**IMPORTANT:** all Oozie server scripts (`oozie-setup.sh`, `oozied.sh`, `oozie-start.sh`, `oozie-run.sh`
+and `oozie-stop.sh`) run only under the Unix user that owns the Oozie installation directory,
+if necessary use `sudo -u OOZIE_USER` when invoking the scripts.
+
+As of Oozie 3.3.2, use of `oozie-start.sh`, `oozie-run.sh`, and `oozie-stop.sh` has
+been deprecated and will print a warning. The `oozied.sh` script should be used
+instead; passing it `start`, `run`, or `stop` as an argument will perform the
+behaviors of `oozie-start.sh`, `oozie-run.sh`, and `oozie-stop.sh` respectively.
+
+Create a **libext/** directory in the directory where Oozie was expanded.
+
+If using the ExtJS library copy the ZIP file to the **libext/** directory. If hadoop and hcatalog libraries are not
+already included in the war, add the corresponding libraries to **libext/** directory.
+
+A "sharelib create -fs fs_default_name [-locallib sharelib]" command is available when running oozie-setup.sh
+for uploading new sharelib into hdfs where the first argument is the default fs name
+and the second argument is the Oozie sharelib to install, it can be a tarball or the expanded version of it.
+If the second argument is omitted, the Oozie sharelib tarball from the Oozie installation directory will be used.
+Upgrade command is deprecated, one should use create command to create new version of sharelib.
+Sharelib files are copied to new `lib_<timestamped>` directory. At start, server picks the sharelib from latest time-stamp directory.
+While starting server also purge sharelib directory which is older than sharelib retention days
+(defined as oozie.service.ShareLibService.temp.sharelib.retention.days and 7 days is default).
+
+db create|upgrade|postupgrade -run [-sqlfile \<FILE\>] command is for create, upgrade or postupgrade oozie db with an
+optional sql file
+
+Run the `oozie-setup.sh` script to configure Oozie with all the components added to the **libext/** directory.
+
+
+```
+$ bin/oozie-setup.sh sharelib create -fs <FS_URI> [-locallib <PATH>]
+ sharelib upgrade -fs <FS_URI> [-locallib <PATH>]
+ db create|upgrade|postupgrade -run [-sqlfile <FILE>]
+```
+
+**IMPORTANT**: If the Oozie server needs to establish secure connection with an external server with a self-signed certificate,
+make sure you specify the location of a truststore that contains required certificates. It can be done by configuring
+`oozie.https.truststore.file` in `oozie-site.xml`, or by setting the `javax.net.ssl.trustStore` system property.
+If it is set in both places, the value passed as system property will be used.
+
+Create the Oozie DB using the 'ooziedb.sh' command line tool:
+
+
+```
+$ bin/ooziedb.sh create -sqlfile oozie.sql -run
+
+Validate DB Connection.
+DONE
+Check DB schema does not exist
+DONE
+Check OOZIE_SYS table does not exist
+DONE
+Create SQL schema
+DONE
+DONE
+Create OOZIE_SYS table
+DONE
+
+Oozie DB has been created for Oozie version '3.2.0'
+
+$
+```
+
+Start Oozie as a daemon process run:
+
+
+```
+$ bin/oozied.sh start
+```
+
+To start Oozie as a foreground process run:
+
+
+```
+$ bin/oozied.sh run
+```
+
+Check the Oozie log file `logs/oozie.log` to ensure Oozie started properly.
+
+Using the Oozie command line tool check the status of Oozie:
+
+
+```
+$ bin/oozie admin -oozie http://localhost:11000/oozie -status
+```
+
+Using a browser go to the [Oozie web console](http://localhost:11000/oozie.html), Oozie status should be **NORMAL**.
+
+Refer to the [Running the Examples](DG_Examples.html) document for details on running the examples.
+
+## Client Installation
+
+### System Requirements
+
+ * Unix (tested in Linux and Mac OS X)
+ * Java 1.8+
+
+The Java 1.8+ `bin` directory should be in the command path.
+
+### Client Installation
+
+Copy and expand the `oozie-client` TAR.GZ file bundled with the distribution. Add the `bin/` directory to the `PATH`.
+
+Refer to the [Command Line Interface Utilities](DG_CommandLineTool.html) document for a full reference of the `oozie`
+command line tool.
+
+NOTE: The Oozie server installation includes the Oozie client. The Oozie client should be installed in remote machines
+only.
+
+<a name="OozieShareLib"></a>
+## Oozie Share Lib Installation
+
+Oozie share lib has been installed by oozie-setup.sh create command explained in the earlier section.
+
+See the [Workflow Functional Specification](WorkflowFunctionalSpec.html#ShareLib) and [Installation](AG_Install.html#Oozie_Share_Lib) for more information about the Oozie ShareLib.
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+
diff --git a/docs/src/site/markdown/ENG_Building.md b/docs/src/site/markdown/ENG_Building.md
new file mode 100644
index 0000000..92f988a
--- /dev/null
+++ b/docs/src/site/markdown/ENG_Building.md
@@ -0,0 +1,281 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+# Building Oozie
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## System Requirements
+
+ * Unix box (tested on Mac OS X and Linux)
+ * Java JDK 1.8+
+ * [Maven 3.0.1+](http://maven.apache.org/)
+ * [Hadoop 2.6.0+](http://hadoop.apache.org/core/releases.html)
+ * [Pig 0.10.1+](http://hadoop.apache.org/pig/releases.html)
+
+JDK commands (java, javac) must be in the command path.
+
+The Maven command (mvn) must be in the command path.
+
+## Oozie Documentation Generation
+
+To generate the documentation, Oozie uses a Doxia plugin for Maven with markdown support.
+
+
+
+
+
+
+```
+$ mvn install
+```
+
+<a name="SshSetup"></a>
+## Passphrase-less SSH Setup
+
+**NOTE: SSH actions are deprecated in Oozie 2.**
+
+To run SSH Testcases and for easier Hadoop start/stop configure SSH to localhost to be passphrase-less.
+
+Create your SSH keys without a passphrase and add the public key to the authorized file:
+
+
+```
+$ ssh-keygen -t dsa
+$ cat ~/.ssh/id_dsa.pub >> ~/.ssh/authorized_keys2
+```
+
+Test that you can ssh without password:
+
+
+```
+$ ssh localhost
+```
+
+## Building with different Java Versions
+
+Oozie requires a minimum Java version of 1.8. Any newer version can be used but by default bytecode will be generated
+which is compatible with 1.8. This can be changed by specifying the build property **targetJavaVersion**.
+
+## Building and Testing Oozie
+
+The JARs for the specified Hadoop and Pig versions must be available in one of the Maven repositories defined in Oozie
+main 'pom.xml' file. Or they must be installed in the local Maven cache.
+
+### Examples Running Oozie Testcases with Different Configurations
+
+**Using embedded Hadoop minicluster with 'simple' authentication:**
+
+
+```
+$ mvn clean test
+```
+
+**Using a Hadoop cluster with 'simple' authentication:**
+
+
+```
+$ mvn clean test -Doozie.test.hadoop.minicluster=false
+```
+
+**Using embedded Hadoop minicluster with 'simple' authentication and Derby database:**
+
+
+```
+$ mvn clean test -Doozie.test.hadoop.minicluster=false -Doozie.test.db=derby
+```
+
+**Using a Hadoop cluster with 'kerberos' authentication:**
+
+
+```
+$ mvn clean test -Doozie.test.hadoop.minicluster=false -Doozie.test.hadoop.security=kerberos
+```
+
+NOTE: The embedded minicluster cannot be used when testing with 'kerberos' authentication.
+
+**Using a custom Oozie configuration for testcases:**
+
+
+```
+$ mvn clean test -Doozie.test.config.file=/home/tucu/custom-oozie-sitel.xml
+```
+
+**Running the testcases with different databases:**
+
+
+```
+$ mvn clean test -Doozie.test.db=[hsqldb*|derby|mysql|postgres|oracle]
+```
+
+Using `mysql` and `oracle` enables profiles that will include their JARs files in the build. If using
+ `oracle`, the Oracle JDBC JAR file must be manually installed in the local Maven cache (the JAR is
+not available in public Maven repos).
+
+### Build Options Reference
+
+All these options can be set using **-D**.
+
+Except for the options marked with `(*)`, the options can be specified in the `test.properties` in the root
+of the Oozie project. The options marked with `(*)` are used in Maven POMs, thus they don't take effect if
+specified in the `test.properties` file (which is loaded by the `XTestCase` class at class initialization time).
+
+**hadoop.version** `(*)`: indicates the Hadoop version you wish to build Oozie against specifically. It will
+substitute this value in the Oozie POM properties and pull the corresponding Hadoop artifacts from Maven.
+The default version is 2.6.0 and that is the minimum supported Hadoop version.
+
+**generateSite** (*): generates Oozie documentation, default is undefined (no documentation is generated)
+
+**skipTests** (*): skips the execution of all testcases, no value required, default is undefined
+
+**test**= (*): runs a single test case, to run a test give the test class name without package and extension, no default
+
+**oozie.test.db**= (*): indicates the database to use for running the testcases, supported values are 'hsqldb', 'derby',
+ 'mysql', 'postgres' and 'oracle'; default value is 'hsqldb'. For each database there is
+ `core/src/test/resources/DATABASE-oozie-site.xml` file preconfigured.
+
+**oozie.test.properties** (*): indicates the file to load the test properties from, by default is `test.properties`.
+Having this option allows having different test properties sets, for example: minicluster, simple & kerberos.
+
+**oozie.test.waitfor.ratio**= : multiplication factor for testcases using waitfor, the ratio is used to adjust the
+effective time out. For slow machines the ratio should be increased. The default value is `1`.
+
+**oozie.test.config.file**= : indicates a custom Oozie configuration file for running the testcases. The specified file
+must be an absolute path. For example, it can be useful to specify different database than HSQL for running the
+testcases.
+
+**oozie.test.hadoop.minicluster**= : indicates if Hadoop minicluster should be started for testcases, default value 'true'
+
+**oozie.test.job.tracker**= : indicates the URI of the JobTracker when using a Hadoop cluster for testing, default value
+'localhost:8021'
+
+**oozie.test.name.node**= : indicates the URI of the NameNode when using a Hadoop cluster for testing, default value
+'`hdfs://localhost:8020`'
+
+**oozie.test.hadoop.security**= : indicates the type of Hadoop authentication for testing, valid values are 'simple' or
+'kerberos, default value 'simple'
+
+**oozie.test.kerberos.keytab.file**= : indicates the location of the keytab file, default value
+'${user.home}/oozie.keytab'
+
+**oozie.test.kerberos.realm**= : indicates the Kerberos real, default value 'LOCALHOST'
+
+**oozie.test.kerberos.oozie.principal**= : indicates the Kerberos principal for oozie, default value
+'${user.name}/localhost'
+
+**oozie.test.kerberos.jobtracker.principal**= : indicates the Kerberos principal for the JobTracker, default value
+'mapred/localhost'
+
+**oozie.test.kerberos.namenode.principal**= : indicates the Kerberos principal for the NameNode, default value
+'hdfs/localhost'
+
+**oozie.test.user.oozie**= : specifies the user ID used to start Oozie server in testcases, default value
+is `${user.name}`.
+
+**oozie.test.user.test**= : specifies primary user ID used as the user submitting jobs to Oozie Server in testcases,
+default value is `test`.
+
+**oozie.test.user.test2**= : specifies secondary user ID used as the user submitting jobs to Oozie Server in testcases,
+default value is `test2`.
+
+**oozie.test.user.test3**= : specifies secondary user ID used as the user submitting jobs to Oozie Server in testcases,
+default value is `test3`.
+
+**oozie.test.group**= : specifies group ID used as group when submitting jobs to Oozie Server in testcases,
+default value is `testg`.
+
+NOTE: The users/group specified in **oozie.test.user.test2**, **oozie.test.user.test3** and **oozie.test.user.group**
+are used for the authorization testcases only.
+
+**oozie.test.dir**` : specifies the directory where the `oozietests` directory will be created, default value is `/tmp=.
+The `oozietests` directory is used by testcases when they need a local filesystem directory.
+
+**hadoop.log.dir**= : specifies the directory where Hadoop minicluster will write its logs during testcases, default
+value is `/tmp`.
+
+**test.exclude**` : specifies a testcase class (just the class name) to exclude for the tests run, for example =TestSubmitCommand`.
+
+**test.exclude.pattern**` : specifies one or more patterns for testcases to exclude, for example =**/Test*Command.java`.
+
+### Testing Map Reduce Pipes Action
+
+Pipes testcases require Hadoop's **wordcount-simple** pipes binary example to run. The **wordcount-simple** pipes binary
+should be compiled for the build platform and copied into Oozie's **core/src/test/resources/** directory. The binary file
+must be named **wordcount-simple**.
+
+If the **wordcount-simple** pipes binary file is not available the testcase will do a NOP and it will print to its output
+file the following message 'SKIPPING TEST: TestPipesMain, binary 'wordcount-simple' not available in the classpath'.
+
+There are 2 testcases that use the **wordcount-simple** pipes binary, **TestPipesMain** and **TestMapReduceActionExecutor**,
+the 'SKIPPING TEST..." message would appear in the testcase log file of both testcases.
+
+### Testing using dist_test and grind
+
+Testing using [dist_test](https://github.com/cloudera/dist_test) framework with
+[grind](https://github.com/cloudera/dist_test/blob/master/docs/grind.md) front end might not work using the default 3.0.2
+version of the maven dependency plugin. It is necessary to downgrade to version 2.10 using
+`-Dmaven-dependency-plugin.version=2.10` .
+
+Maven flags for grind can be specified using `GRIND_MAVEN_FLAGS` environment variable:
+
+
+```
+export GRIND_MAVEN_FLAGS=-Dmaven.dependency.plugin.version=2.10
+grind test --java-version 8
+```
+
+## Building an Oozie Distribution
+
+An Oozie distribution bundles an embedded Jetty server.
+
+The simplest way to build Oozie is to run the `mkdistro.sh` script:
+
+```
+$ bin/mkdistro.sh [-DskipTests]
+Running =mkdistro.sh= will create the binary distribution of Oozie. The following options are available to customise
+the versions of the dependencies:
+-Puber - Bundle required hadoop and hcatalog libraries in oozie war
+-Dhadoop.version=<version> - default 2.6.0
+-Ptez - Bundle tez jars in hive and pig sharelibs. Useful if you want to use tez
+as the execution engine for those applications.
+-Dpig.version=<version> - default 0.16.0
+-Dpig.classifier=<classifier> - default h2
+-Dsqoop.version=<version> - default 1.4.3
+-Dsqoop.classifier=<classifier> - default hadoop100
+-jetty.version=<version> - default 9.3.20.v20170531
+-Dopenjpa.version=<version> - default 2.2.2
+-Dxerces.version=<version> - default 2.10.0
+-Dcurator.version=<version> - default 2.5.0
+-Dhive.version=<version> - default 1.2.0
+-Dhbase.version=<version> - default 1.2.3
+-Dtez.version=<version> - default 0.8.4
+```
+
+**IMPORTANT:** Profile hadoop-3 must be activated if building against Hadoop 3
+
+The following properties should be specified when building a release:
+
+ * -DgenerateDocs : forces the generation of Oozie documentation
+ * -Dbuild.time= : timestamps the distribution
+ * -Dvc.revision= : specifies the source control revision number of the distribution
+ * -Dvc.url= : specifies the source control URL of the distribution
+
+The provided `bin/mkdistro.sh` script runs the above Maven invocation setting all these properties to the
+right values (the 'vc.*' properties are obtained from the local git repository).
+
+## IDE Setup
+
+Eclipse and IntelliJ can use directly Oozie Maven project files.
+
+The only special consideration is that the following source directories from the `client` module must be added to
+the `core` module source path:
+
+ * `client/src/main/java` : as source directory
+ * `client/src/main/resources` : as source directory
+ * `client/src/test/java` : as test-source directory
+ * `client/src/test/resources` : as test-source directory
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+
diff --git a/docs/src/site/markdown/ENG_Custom_Authentication.md b/docs/src/site/markdown/ENG_Custom_Authentication.md
new file mode 100644
index 0000000..c70355f
--- /dev/null
+++ b/docs/src/site/markdown/ENG_Custom_Authentication.md
@@ -0,0 +1,80 @@
+
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+# Creating Custom Authentication
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Hadoop-Auth Authentication Interfaces and classes
+
+1. `org.apache.hadoop.security.authentication.client.Authenticator:` Interface for client authentication mechanisms.
+
+ The following authenticators are provided in hadoop-auth:
+
+ * KerberosAuthenticator : the authenticator implements the Kerberos SPNEGO authentication sequence.
+ * PseudoAuthenticator : the authenticator implementation provides an authentication equivalent to Hadoop's Simple
+ authentication, it trusts the value of the 'user.name' Java System property.
+
+2. `org.apache.hadoop.security.authentication.server.AuthenticationHandler:` Interface for server authentication mechanisms.
+
+ * KerberosAuthenticationHandler : the authenticator handler implements the Kerberos SPNEGO authentication mechanism for HTTP.
+ * PseudoAuthenticationHandler : the authenticator handler provides a pseudo authentication mechanism that accepts the user
+ name specified as a query string parameter.
+
+3. `org.apache.hadoop.security.authentication.server.AuthenticationFilter:` A servlet filter enables protecting web application
+ resources with different authentication mechanisms provided by AuthenticationHandler. To enable the filter, web application
+ resources file (ex. web.xml) needs to include a filter class derived from `AuthenticationFilter`.
+
+ For more information have a look at the appropriate
+ [Hadoop documentation](https://hadoop.apache.org/docs/r2.7.2/hadoop-auth/index.html).
+
+## Provide Custom Authentication to Oozie Client
+
+Apache Oozie contains a default class `org.apache.oozie.client.AuthOozieClient` to support Kerberos HTTP SPNEGO authentication,
+pseudo/simple authentication and anonymous access for client connections.
+
+To provide other authentication mechanisms, an Oozie client should extend from `AuthOozieClient` and provide the following
+methods should be overridden by derived classes to provide custom authentication:
+
+ * getAuthenticator() : return corresponding Authenticator based on value specified by user at `auth` command option.
+ * createConnection() : create a singleton class at Authenticator to allow client set and get key-value configuration for
+ authentication.
+
+## Provide Custom Authentication to Oozie Server
+
+To accept custom authentication in Oozie server, a filter extends from AuthenticationFilter must be provided. This filter
+delegates to the configured authentication handler for authentication and once it obtains an `AuthenticationToken` from it, sets
+a signed HTTP cookie with the token. If HTTP cookie is provided with different key name, its cookie value can be retrieved by
+overriding `getToken()` method. Please note, only when `getToken()` return NULL, a custom authentication can be invoked and
+processed in `AuthenticationFilter.doFilter()`.
+
+The following method explains how to read it and return NULL token.
+
+```
+protected AuthenticationToken getToken(HttpServletRequest request) throws IOException, AuthenticationException {
+ String tokenStr = null;
+ Cookie[] cookies = request.getCookies();
+
+ if (cookies != null) {
+ for (Cookie cookie : cookies) {
+ if (cookie.getName().equals(AuthenticatedURL.AUTH_COOKIE)) {
+ tokenStr = cookie.getValue();
+ LOG.info("Got 'hadoop.auth' cookie from request = " + tokenStr);
+ if (tokenStr != null && !tokenStr.trim().isEmpty()) {
+ AuthenticationToken retToken = super.getToken(request);
+ return retToken;
+ }
+ } else if (cookie.getName().equals("NEWAUTH")) {
+ tokenStr = cookie.getValue();
+ // DO NOT return the token string so request can authenticated.
+ }
+ }
+ }
+ return null;
+ }
+```
+
+[::Go back to Oozie Documentation Index::](index.html)
+
+
diff --git a/docs/src/site/markdown/index.md b/docs/src/site/markdown/index.md
new file mode 100644
index 0000000..0216222
--- /dev/null
+++ b/docs/src/site/markdown/index.md
@@ -0,0 +1,109 @@
+
+
+# Oozie, Workflow Engine for Apache Hadoop
+
+Oozie v3 is a server based _Bundle Engine_ that provides a higher-level oozie abstraction that will batch a set of coordinator applications. The user will be able to start/stop/suspend/resume/rerun a set coordinator jobs in the bundle level resulting a better and easy operational control.
+
+Oozie v2 is a server based _Coordinator Engine_ specialized in running workflows based on time and data triggers.
+(e.g. wait for my input data to exist before running my workflow).
+
+Oozie v1 is a server based _Workflow Engine_ specialized in running workflow jobs with actions that
+execute Hadoop Map/Reduce and Pig jobs.
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Distribution Contents
+
+Oozie distribution consists of a single 'tar.gz' file containing:
+
+ * Readme, license, notice & [Release log](./release-log.txt) files.
+ * Oozie server: `oozie-server` directory.
+ * Scripts: `bin/` directory, client and server scripts.
+ * Binaries: `lib/` directory, client JAR files.
+ * Configuration: `conf/` server configuration directory.
+ * Archives:
+ * `oozie-client-*.tar.gz`: Client tools.
+ * `oozie.war`: Oozie WAR file.
+ * `docs.zip`: Documentation.
+ * `oozie-examples-*.tar.gz`: Examples.
+ * `oozie-sharelib-*.tar.gz`: Share libraries (with Streaming, Pig JARs).
+
+## Quick Start
+
+Enough reading already? Follow the steps in [Oozie Quick Start](DG_QuickStart.html) to get Oozie up and running.
+
+## Developer Documentation
+
+ * [Overview](DG_Overview.html)
+ * [Oozie Quick Start](DG_QuickStart.html)
+ * [Running the Examples](DG_Examples.html)
+ * [Workflow Functional Specification](WorkflowFunctionalSpec.html)
+ * [Coordinator Functional Specification](CoordinatorFunctionalSpec.html)
+ * [Bundle Functional Specification](BundleFunctionalSpec.html)
+ * [EL Expression Language Quick Reference](https://docs.oracle.com/javaee/7/tutorial/jsf-el.htm)
+ * [Command Line Tool](DG_CommandLineTool.html)
+ * [Workflow Re-runs Explained](DG_WorkflowReRun.html)
+ * [HCatalog Integration Explained](DG_HCatalogIntegration.html)
+ * [Oozie Client Javadocs](./client/apidocs/index.html)
+ * [Oozie Core Javadocs](./core/apidocs/index.html)
+ * [Oozie Web Services API](WebServicesAPI.html)
+ * [Action Authentication](DG_ActionAuthentication.html)
+
+### Action Extensions
+
+ * [Email Action](DG_EmailActionExtension.html)
+ * [Shell Action](DG_ShellActionExtension.html)
+ * [Hive Action](DG_HiveActionExtension.html)
+ * [Hive 2 Action](DG_Hive2ActionExtension.html)
+ * [Sqoop Action](DG_SqoopActionExtension.html)
+ * [Ssh Action](DG_SshActionExtension.html)
+ * [DistCp Action](DG_DistCpActionExtension.html)
+ * [Spark Action](DG_SparkActionExtension.html)
+ * [Writing a Custom Action Executor](DG_CustomActionExecutor.html)
+
+### Job Status and SLA Monitoring
+
+ * [JMS Notifications for Job and SLA](DG_JMSNotifications.html)
+ * [Configuring and Monitoring SLA](DG_SLAMonitoring.html)
+
+## Administrator Documentation
+
+ * [Oozie Install](AG_Install.html)
+ * [Oozie Logging](AG_OozieLogging.html)
+ * [Hadoop Configuration](AG_HadoopConfiguration.html)
+ * [Action Configuration](AG_ActionConfiguration.html)
+ * [Oozie Monitoring](AG_Monitoring.html)
+ * [Command Line Tool](DG_CommandLineTool.html)
+ * [Oozie Upgrade](AG_OozieUpgrade.html)
+
+<a name="LicenseInfo"></a>
+## Licensing Information
+
+Oozie is distributed under [Apache License 2.0](http://www.apache.org/licenses/LICENSE-2.0).
+
+For details on the license of the dependent components, refer to the
+[Dependencies Report, Licenses section](./dependencies.html#Licenses).
+
+Oozie bundles an embedded Jetty 9.x.
+
+Some of the components in the dependencies report don't mention their license in the published POM. They are:
+
+ * JDOM: [JDOM License](http://www.jdom.org/docs/faq.html#a0030) (Apache style).
+ * Oro: [Apache License 2.0](http://www.apache.org/licenses/LICENSE-2.0).
+
+
+
+## Engineering Documentation
+
+ * [Building Oozie](ENG_Building.html)
+ * [Dependencies Report](./dependencies.html)
+
+## MiniOozie Documentation
+
+ * [Testing User Oozie Applications Using MiniOozie](ENG_MiniOozie.html)
+
+## Oozie User Authentication Documentation
+
+ * [Create Custom Oozie Authentication](ENG_Custom_Authentication.html)
+
+
diff --git a/docs/src/site/twiki/AG_ActionConfiguration.twiki b/docs/src/site/twiki/AG_ActionConfiguration.twiki
index 8c032a7..e020d1a 100644
--- a/docs/src/site/twiki/AG_ActionConfiguration.twiki
+++ b/docs/src/site/twiki/AG_ActionConfiguration.twiki
@@ -1,24 +1,24 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Action Configuration
+[::Go back to Oozie Documentation Index::](index.html)
+
+# Action Configuration
Oozie supports providing default configuration for actions of a particular
action type and default configuration for all actions
----++ Hadoop Default Configuration Values
+## Hadoop Default Configuration Values
-Oozie supports action configuration equivalent to the component's =*-site.xml= and =*.properties= files.
+Oozie supports action configuration equivalent to the component's `*-site.xml` and `*.properties` files.
-The configuration property in the =oozie-site.xml= is =oozie.service.HadoopAccessorService.action.configurations=
-and its value must follow the pattern _[AUTHORITY=ACTION-CONF-DIR_,]*. Where _AUTHORITY_ is the _HOST:PORT_ of
+The configuration property in the `oozie-site.xml` is `oozie.service.HadoopAccessorService.action.configurations`
+and its value must follow the pattern _\[AUTHORITY=ACTION-CONF-DIR_,\]*. Where _AUTHORITY_ is the _HOST:PORT_ of
the Hadoop service (JobTracker/ResourceManager or HDFS). The _ACTION-CONF-DIR_ is the action configuration directory. If the specified directory is a relative path, it will be looked under the Oozie configuration directory. An absolute path can
also be specified. Oozie will load and process the action configuration files in the following order.
1. All files in _default_/*.xml (sorted by lexical name, files with names lexically lower have lesser precedence than the following ones), if present.
1. _default_.xml, if present.
- 1. All supported files in _actionname_/*, e.g. _actionname_/*.xml and _actionname_/*.properties (based on filename extension, sorted by lexical name, files with names lexically lower have lesser precedence than the following ones), if present.
+ 1. All supported files in _actionname_/\*, e.g. _actionname_/\*.xml and _actionname_/*.properties (based on filename extension, sorted by lexical name, files with names lexically lower have lesser precedence than the following ones), if present.
1. _actionname_.xml, if present.
@@ -26,7 +26,7 @@
1. All files in _default_/*.xml, if present
1. _default_.xml, if present.
- 1. All files in _hive_/*.xml and _hive_/*.properties, if present
+ 1. All files in _hive_/\*.xml and _hive_/\*.properties, if present
1. _hive_.xml, if present.
@@ -35,22 +35,23 @@
In addition to explicit authorities, a '*' wildcard is supported. The configuration file associated with the wildcard
will be used as default if there is no action configuration for the requested Hadoop service.
-For example, the configuration in the =oozie-site.xml= would look like:
+For example, the configuration in the `oozie-site.xml` would look like:
-<verbatim>
+
+```
...
<property>
<name>oozie.service.HadoopAccessorService.action.configurations</name>
<value>*=hadoop-conf,jt-bar:8021=bar-cluster,nn-bar:8020=bar-cluster</value>
</property>
...
-</verbatim>
+```
The action configuration files use the Hadoop configuration syntax.
By default Oozie does not define any default action configurations.
----++ Dependency deduplication
+## Dependency deduplication
Using Oozie with Hadoop 3 may require to have dependency file names distinguishable,
which means having two files on sharelib and in your app's dependencies with identical names, leads to job submission failure.
@@ -62,7 +63,8 @@
Real world example:
You have an application workflow which is uploaded to HDFS in /apps/app directory. You have your app.jar and dependency jars.
You also define a spark action in your workflow and set use system libs; the HDFS tree is similar to this:
-<verbatim>
+
+```
+ /apps/app/
- app.jar
- workflow.xml
@@ -75,11 +77,11 @@
- jackson-annotations-1.0.jar
+ oozie
- jackson-annotations-1.0.jar
-</verbatim>
+```
The deduplicator code will create the following list of files:
-=/apps/app/app.jar,/apps/app/libs/jackson-annotations-1.0.jar=
+`/apps/app/app.jar,/apps/app/libs/jackson-annotations-1.0.jar`
And no other files will be passed at job submission.
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/AG_HadoopConfiguration.twiki b/docs/src/site/twiki/AG_HadoopConfiguration.twiki
index 528bf4a..ab71d7c 100644
--- a/docs/src/site/twiki/AG_HadoopConfiguration.twiki
+++ b/docs/src/site/twiki/AG_HadoopConfiguration.twiki
@@ -1,14 +1,15 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Hadoop Configuration
+[::Go back to Oozie Documentation Index::](index.html)
----++ Hadoop Services Whitelisting
+# Hadoop Configuration
+
+## Hadoop Services Whitelisting
Oozie supports whitelisting Hadoop services (JobTracker, HDFS), via 2 configuration properties:
-<verbatim>
+
+```
...
<property>
<name>oozie.service.HadoopAccessorService.jobTracker.whitelist</name>
@@ -25,43 +26,45 @@
</description>
</property>
...
-</verbatim>
+```
-The value must follow the pattern =[AUTHORITY,...]=. Where =AUTHORITY= is the =HOST:PORT= of
+The value must follow the pattern `[AUTHORITY,...]`. Where `AUTHORITY` is the `HOST:PORT` of
the Hadoop service (JobTracker, HDFS).
If the value is empty any HOST:PORT is accepted. Empty is the default value.
----++ Hadoop Default Configuration Values
+## Hadoop Default Configuration Values
-Oozie supports Hadoop configuration equivalent to the Hadoop =*-site.xml= files.
+Oozie supports Hadoop configuration equivalent to the Hadoop `*-site.xml` files.
-The configuration property in the =oozie-site.xml= is =oozie.service.HadoopAccessorService.hadoop.configurations=
-and its value must follow the pattern =[<AUTHORITY>=<HADOOP_CONF_DIR>,]*=. Where =<AUTHORITY>= is the =HOST:PORT= of
-the Hadoop service (JobTracker, HDFS). The =<HADOOP_CONF_DIR>= is a Hadoop configuration directory. If the specified
+The configuration property in the `oozie-site.xml` is `oozie.service.HadoopAccessorService.hadoop.configurations`
+and its value must follow the pattern `[<AUTHORITY>=<HADOOP_CONF_DIR>,]*`. Where `<AUTHORITY>` is the `HOST:PORT` of
+the Hadoop service (JobTracker, HDFS). The `<HADOOP_CONF_DIR>` is a Hadoop configuration directory. If the specified
directory is a relative path, it will be looked under the Oozie configuration directory. And absolute path can
- also be specified. Oozie will load the Hadoop =*-site.xml= files in the following order: core-site.xml, hdfs-site.xml,
+ also be specified. Oozie will load the Hadoop `*-site.xml` files in the following order: core-site.xml, hdfs-site.xml,
mapred-site.xml, yarn-site.xml, hadoop-site.xml, ssl-client.xml.
In addition to explicit authorities, a '*' wildcard is supported. The configuration file associated with the wildcard
will be used as default if there is no configuration for the requested Hadoop service.
-For example, the configuration in the =oozie-site.xml= would look like:
+For example, the configuration in the `oozie-site.xml` would look like:
-<verbatim>
+
+```
...
<property>
<name>oozie.service.HadoopAccessorService.hadoop.configurations</name>
<value>*=hadoop-conf,jt-bar:8021=bar-cluster,nn-bar:8020=bar-cluster</value>
</property>
...
-</verbatim>
+```
The Hadoop configuration files use the Hadoop configuration syntax.
-By default Oozie defines =*=hadoop-conf= and the default values of the =hadoop-site.xml= file are:
+By default Oozie defines `*=hadoop-conf` and the default values of the `hadoop-site.xml` file are:
-<verbatim>
+
+```
<configuration>
<property>
<name>mapreduce.jobtracker.kerberos.principal</name>
@@ -80,12 +83,12 @@
<value>yarn</value>
</property>
</configuration>
-</verbatim>
+```
----++ Limitations
+## Limitations
All actions in a workflow application must interact with the same Hadoop JobTracker and NameNode.
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/AG_Install.twiki b/docs/src/site/twiki/AG_Install.twiki
index b8031c8..270b98f 100644
--- a/docs/src/site/twiki/AG_Install.twiki
+++ b/docs/src/site/twiki/AG_Install.twiki
@@ -1,51 +1,52 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Oozie Installation and Configuration
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Oozie Installation and Configuration
----++ Basic Setup
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
-Follow the instructions at [[DG_QuickStart][Oozie Quick Start]].
+## Basic Setup
----++ Environment Setup
+Follow the instructions at [Oozie Quick Start](DG_QuickStart.html).
-*IMPORTANT:* Oozie ignores any set value for =OOZIE_HOME=, Oozie computes its home automatically.
+## Environment Setup
-When running Oozie with its embedded Jetty server, the =conf/oozie-env.sh= file can be
+**IMPORTANT:** Oozie ignores any set value for `OOZIE_HOME`, Oozie computes its home automatically.
+
+When running Oozie with its embedded Jetty server, the `conf/oozie-env.sh` file can be
used to configure the following environment variables used by Oozie:
-*JETTY_OPTS* : settings for the Embedded Jetty that runs Oozie. Java System properties
+**JETTY_OPTS** : settings for the Embedded Jetty that runs Oozie. Java System properties
for Oozie should be specified in this variable. No default value.
-*OOZIE_CONFIG_FILE* : Oozie configuration file to load from Oozie configuration directory.
-Default value =oozie-site.xml=.
+**OOZIE_CONFIG_FILE** : Oozie configuration file to load from Oozie configuration directory.
+Default value `oozie-site.xml`.
-*OOZIE_LOGS* : Oozie logs directory. Default value =logs/= directory in the Oozie installation
+**OOZIE_LOGS** : Oozie logs directory. Default value `logs/` directory in the Oozie installation
directory.
-*OOZIE_LOG4J_FILE* : Oozie Log4J configuration file to load from Oozie configuration directory.
-Default value =oozie-log4j.properties=.
+**OOZIE_LOG4J_FILE** : Oozie Log4J configuration file to load from Oozie configuration directory.
+Default value `oozie-log4j.properties`.
-*OOZIE_LOG4J_RELOAD* : Reload interval of the Log4J configuration file, in seconds.
-Default value =10=
+**OOZIE_LOG4J_RELOAD** : Reload interval of the Log4J configuration file, in seconds.
+Default value `10`
-*OOZIE_CHECK_OWNER* : If set to =true=, Oozie setup/start/run/stop scripts will check that the
+**OOZIE_CHECK_OWNER** : If set to `true`, Oozie setup/start/run/stop scripts will check that the
owner of the Oozie installation directory matches the user invoking the script. The default
-value is undefined and interpreted as a =false=.
+value is undefined and interpreted as a `false`.
-*OOZIE_INSTANCE_ID* : The instance id of the Oozie server. When using HA, each server instance should have a unique instance id.
-Default value =${OOZIE_HTTP_HOSTNAME}=
+**OOZIE_INSTANCE_ID** : The instance id of the Oozie server. When using HA, each server instance should have a unique instance id.
+Default value `${OOZIE_HTTP_HOSTNAME}`
----++ Oozie Server Setup
+## Oozie Server Setup
-The =oozie-setup.sh= script prepares the embedded Jetty server to run Oozie.
+The `oozie-setup.sh` script prepares the embedded Jetty server to run Oozie.
-The =oozie-setup.sh= script options are:
+The `oozie-setup.sh` script options are:
-<verbatim>
+
+```
Usage : oozie-setup.sh <Command and OPTIONS>
sharelib create -fs FS_URI [-locallib SHARED_LIBRARY] [-extralib EXTRA_SHARED_LIBRARY] [-concurrency CONCURRENCY]
(create sharelib for oozie,
@@ -76,24 +77,25 @@
import <file> imports the oozie database from the zip file
created by export
(without options prints this usage information)
-</verbatim>
+```
-If a directory =libext/= is present in Oozie installation directory, the =oozie-setup.sh= script will
-include all JARs in Jetty's =webapp/WEB_INF/lib/= directory.
+If a directory `libext/` is present in Oozie installation directory, the `oozie-setup.sh` script will
+include all JARs in Jetty's `webapp/WEB_INF/lib/` directory.
-If the ExtJS ZIP file is present in the =libext/= directory, it will be added to the Jetty's =webapp/= directory as well.
-The ExtJS library file name be =ext-2.2.zip=.
+If the ExtJS ZIP file is present in the `libext/` directory, it will be added to the Jetty's `webapp/` directory as well.
+The ExtJS library file name be `ext-2.2.zip`.
----+++ Setting Up Oozie with an Alternate Tomcat
+### Setting Up Oozie with an Alternate Tomcat
-Use the =addtowar.sh= script to prepare the Oozie server only if Oozie will run with a different
+Use the `addtowar.sh` script to prepare the Oozie server only if Oozie will run with a different
servlet container than the embedded Jetty provided with the distribution.
-The =addtowar.sh= script adds Hadoop JARs, JDBC JARs and the ExtJS library to the Oozie WAR file.
+The `addtowar.sh` script adds Hadoop JARs, JDBC JARs and the ExtJS library to the Oozie WAR file.
-The =addtowar.sh= script options are:
+The `addtowar.sh` script options are:
-<verbatim>
+
+```
Usage : addtowar <OPTIONS>
Options: -inputwar INPUT_OOZIE_WAR
-outputwar OUTPUT_OOZIE_WAR
@@ -101,21 +103,21 @@
[-extjs EXTJS_PATH]
[-jars JARS_PATH] (multiple JAR path separated by ':')
[-secureWeb WEB_XML_PATH] (path to secure web.xml)
-</verbatim>
+```
-The original =oozie.war= file is in the Oozie server installation directory.
+The original `oozie.war` file is in the Oozie server installation directory.
-After the Hadoop JARs and the ExtJS library has been added to the =oozie.war= file Oozie is ready to run.
+After the Hadoop JARs and the ExtJS library has been added to the `oozie.war` file Oozie is ready to run.
-Delete any previous deployment of the =oozie.war= from the servlet container (if using Tomcat, delete
-=oozie.war= and =oozie= directory from Tomcat's =webapps/= directory)
+Delete any previous deployment of the `oozie.war` from the servlet container (if using Tomcat, delete
+`oozie.war` and `oozie` directory from Tomcat's `webapps/` directory)
-Deploy the prepared =oozie.war= file (the one that contains the Hadoop JARs and the ExtJS library) in the
-servlet container (if using Tomcat, copy the prepared =oozie.war= file to Tomcat's =webapps/= directory).
+Deploy the prepared `oozie.war` file (the one that contains the Hadoop JARs and the ExtJS library) in the
+servlet container (if using Tomcat, copy the prepared `oozie.war` file to Tomcat's `webapps/` directory).
-*IMPORTANT:* Only one Oozie instance can be deployed per Tomcat instance.
+**IMPORTANT:** Only one Oozie instance can be deployed per Tomcat instance.
----++ Database Configuration
+## Database Configuration
Oozie works with HSQL, Derby, MySQL, Oracle, PostgreSQL or SQL Server databases.
@@ -125,23 +127,24 @@
HSQL is normally used for test cases as it is an in-memory database and all data is lost every time Oozie is stopped.
-If using Derby, MySQL, Oracle, PostgreSQL, or SQL Server, the Oozie database schema must be created using the =ooziedb.sh= command
+If using Derby, MySQL, Oracle, PostgreSQL, or SQL Server, the Oozie database schema must be created using the `ooziedb.sh` command
line tool.
-If using MySQL, Oracle, or SQL Server, the corresponding JDBC driver JAR file must be copied to Oozie's =libext/= directory and
-it must be added to Oozie WAR file using the =bin/addtowar.sh= or the =oozie-setup.sh= scripts using the =-jars= option.
+If using MySQL, Oracle, or SQL Server, the corresponding JDBC driver JAR file must be copied to Oozie's `libext/` directory and
+it must be added to Oozie WAR file using the `bin/addtowar.sh` or the `oozie-setup.sh` scripts using the `-jars` option.
-*IMPORTANT:* It is recommended to set the database's timezone to GMT (consult your database's documentation on how to do this).
+**IMPORTANT:** It is recommended to set the database's timezone to GMT (consult your database's documentation on how to do this).
Databases don't handle Daylight Saving Time shifts correctly, and may cause problems if you run any Coordinators with actions
scheduled to materialize during the 1 hour period where we "fall back". For Derby, you can add '-Duser.timezone=GMT'
-to =JETTY_OPTS= in oozie-env.sh to set this. Alternatively, if using MySQL, you can have Oozie use GMT with MySQL without
+to `JETTY_OPTS` in oozie-env.sh to set this. Alternatively, if using MySQL, you can have Oozie use GMT with MySQL without
setting MySQL's timezone to GMT by adding 'useLegacyDatetimeCode=false&serverTimezone=GMT' arguments to the JDBC
-URL, =oozie.service.JPAService.jdbc.url=. Be advised that changing the timezone on an existing Oozie database while Coordinators
+URL, `oozie.service.JPAService.jdbc.url`. Be advised that changing the timezone on an existing Oozie database while Coordinators
are already running may cause Coordinators to shift by the offset of their timezone from GMT once after making this change.
The SQL database used by Oozie is configured using the following configuration properties (default values shown):
-<verbatim>
+
+```
oozie.db.schema.name=oozie
oozie.service.JPAService.create.db.schema=false
oozie.service.JPAService.validate.db.connection=false
@@ -150,19 +153,20 @@
oozie.service.JPAService.jdbc.username=sa
oozie.service.JPAService.jdbc.password=
oozie.service.JPAService.pool.max.active.conn=10
-</verbatim>
+```
-*NOTE:* If the =oozie.db.schema.create= property is set to =true= (default value is =false=) the Oozie tables
-will be created automatically without having to use the =ooziedb= command line tool. Setting this property to
- =true= it is recommended only for development.
+**NOTE:** If the `oozie.db.schema.create` property is set to `true` (default value is `false`) the Oozie tables
+will be created automatically without having to use the `ooziedb` command line tool. Setting this property to
+ `true` it is recommended only for development.
-*NOTE:* If the =oozie.db.schema.create= property is set to true, the =oozie.service.JPAService.validate.db.connection=
-property value is ignored and Oozie handles it as set to =false=.
+**NOTE:** If the `oozie.db.schema.create` property is set to true, the `oozie.service.JPAService.validate.db.connection`
+property value is ignored and Oozie handles it as set to `false`.
-Once =oozie-site.xml= has been configured with the database configuration execute the =ooziedb.sh= command line tool to
+Once `oozie-site.xml` has been configured with the database configuration execute the `ooziedb.sh` command line tool to
create the database:
-<verbatim>
+
+```
$ bin/ooziedb.sh create -sqlfile oozie.sql -run
Validate DB Connection.
@@ -182,18 +186,19 @@
The SQL commands have been written to: oozie.sql
$
-</verbatim>
+```
-NOTE: If using MySQL, Oracle, or SQL Server, copy the corresponding JDBC driver JAR file to the =libext/= directory before running
-the =ooziedb.sh= command line tool.
+NOTE: If using MySQL, Oracle, or SQL Server, copy the corresponding JDBC driver JAR file to the `libext/` directory before running
+the `ooziedb.sh` command line tool.
-NOTE: If instead using the '-run' option, the '-sqlfile <FILE>' option is used, then all the
+NOTE: If instead using the '-run' option, the `-sqlfile <FILE>` option is used, then all the
database changes will be written to the specified file and the database won't be modified.
-If using HSQL there is no need to use the =ooziedb= command line tool as HSQL is an in-memory database. Use the
+If using HSQL there is no need to use the `ooziedb` command line tool as HSQL is an in-memory database. Use the
following configuration properties in the oozie-site.xml:
-<verbatim>
+
+```
oozie.db.schema.name=oozie
oozie.service.JPAService.create.db.schema=true
oozie.service.JPAService.validate.db.connection=false
@@ -202,24 +207,25 @@
oozie.service.JPAService.jdbc.username=sa
oozie.service.JPAService.jdbc.password=
oozie.service.JPAService.pool.max.active.conn=10
-</verbatim>
+```
If you are interested in fine tuning how Oozie can retry database operations on failing database connectivity or errors, you can
set following properties to other values. Here are the default ones:
-<verbatim>
+
+```
oozie.service.JPAService.retry.initial-wait-time.ms=100
oozie.service.JPAService.retry.maximum-wait-time.ms=30000
oozie.service.JPAService.retry.max-retries=10
-</verbatim>
+```
-If you set either =oozie.service.JPAService.retry.max-retries= or =oozie.service.JPAService.retry.maximum-wait-time.ms= to =0=,
+If you set either `oozie.service.JPAService.retry.max-retries` or `oozie.service.JPAService.retry.maximum-wait-time.ms` to `0`,
no retry attempts will be made on any database connectivity issues. Exact settings for these properties depend also on how much load
is on Oozie regarding workflow and coordinator jobs.
-The database operation retry functionality kicks in when there is a =javax.persistence.PersistenceException= those root cause is not
-part of the normal everyday operation - filtered against a blacklist consisting of descendants like =NoSuchResultException=,
-=NonUniqueResultException=, and the like. This way Oozie won't retry database operations on errors that are more related to the
+The database operation retry functionality kicks in when there is a `javax.persistence.PersistenceException` those root cause is not
+part of the normal everyday operation - filtered against a blacklist consisting of descendants like `NoSuchResultException`,
+`NonUniqueResultException`, and the like. This way Oozie won't retry database operations on errors that are more related to the
current query, or otherwise part of the everyday life. This way it's ensured that this blacklist is database agnostic.
It has been tested with a MySQL / failing every minute 10 seconds / an Oozie coordinator job of an Oozie workflow consisting of four
@@ -227,20 +233,21 @@
To set up such a failing MySQL scenario following has to be performed:
- * Set =oozie.service.JPAService.connection.data.source= to =org.apache.oozie.util.db.BasicDataSourceWrapper=
- within =oozie-site.xml=
- * Set =oozie.service.JPAService.jdbc.driver= to =org.apache.oozie.util.db.FailingMySQLDriverWrapper= within =oozie-site.xml=
+ * Set `oozie.service.JPAService.connection.data.source` to `org.apache.oozie.util.db.BasicDataSourceWrapper`
+ within `oozie-site.xml`
+ * Set `oozie.service.JPAService.jdbc.driver` to `org.apache.oozie.util.db.FailingMySQLDriverWrapper` within `oozie-site.xml`
* Restart Oozie server
* Submit / start some workflows, coordinators etc.
- * See how Oozie is retrying on injected database errors by looking at the Oozie server logs, grepping =JPAException= instances
- with following message prefix: <verbatim>Deliberately failing to prepare statement.</verbatim>
+ * See how Oozie is retrying on injected database errors by looking at the Oozie server logs, grepping `JPAException` instances
+ with following message prefix: `Deliberately failing to prepare statement.`
----++ Database Migration
+## Database Migration
Oozie provides an easy way to switch between databases without losing any data. Oozie servers should be stopped during the
database migration process.
The export of the database can be done using the following command:
-<verbatim>
+
+```
$ bin/oozie-setup.sh export /tmp/oozie_db.zip
1 rows exported from OOZIE_SYS
50 rows exported from WF_JOBS
@@ -251,14 +258,15 @@
0 rows exported from BUNDLE_ACTIONS
0 rows exported from SLA_REGISTRATION
0 rows exported from SLA_SUMMARY
-</verbatim>
+```
-The database configuration is read from =oozie-site.xml=. After updating the configuration to point to the new database,
-the tables have to be created with ooziedb.sh in the [[AG_Install#Database_Configuration][Database configuration]]
+The database configuration is read from `oozie-site.xml`. After updating the configuration to point to the new database,
+the tables have to be created with ooziedb.sh in the [Database configuration](AG_Install.html#Database_Configuration)
section above.
Once the tables are created, they can be filled with data using the following command:
-<verbatim>
+
+```
$ bin/oozie-setup.sh import /tmp/oozie_db.zip
Loading to Oozie database version 3
50 rows imported to WF_JOBS
@@ -269,124 +277,128 @@
0 rows imported to BUNDLE_ACTIONS
0 rows imported to SLA_REGISTRATION
0 rows imported to SLA_SUMMARY
-</verbatim>
+```
NOTE: The database version of the zip must match the version of the Oozie database it's imported to.
After starting the Oozie server, the history and the currently running workflows should be available.
-*IMPORTANT:* The tool was primarily developed to make the migration from embedded databases (e.g. Derby) to standalone databases
+**IMPORTANT:** The tool was primarily developed to make the migration from embedded databases (e.g. Derby) to standalone databases
(e.g. MySQL, PosgreSQL, Oracle, MS SQL Server), though it will work between any supported databases.
-It is *not* optimized to handle databases over 1 Gb. If the database size is larger, it should be purged before migration.
+It is **not** optimized to handle databases over 1 Gb. If the database size is larger, it should be purged before migration.
----++ Oozie Configuration
+## Oozie Configuration
-By default, Oozie configuration is read from Oozie's =conf/= directory
+By default, Oozie configuration is read from Oozie's `conf/` directory
The Oozie configuration is distributed in 3 different files:
- * =oozie-site.xml= : Oozie server configuration
- * =oozie-log4j.properties= : Oozie logging configuration
- * =adminusers.txt= : Oozie admin users list
+ * `oozie-site.xml` : Oozie server configuration
+ * `oozie-log4j.properties` : Oozie logging configuration
+ * `adminusers.txt` : Oozie admin users list
----+++ Oozie Configuration Properties
+### Oozie Configuration Properties
-All Oozie configuration properties and their default values are defined in the =oozie-default.xml= file.
+All Oozie configuration properties and their default values are defined in the `oozie-default.xml` file.
Oozie resolves configuration property values in the following order:
* If a Java System property is defined, it uses its value
- * Else, if the Oozie configuration file (=oozie-site.xml=) contains the property, it uses its value
- * Else, it uses the default value documented in the =oozie-default.xml= file
+ * Else, if the Oozie configuration file (`oozie-site.xml`) contains the property, it uses its value
+ * Else, it uses the default value documented in the `oozie-default.xml` file
-*NOTE:* The =oozie-default.xml= file found in Oozie's =conf/= directory is not used by Oozie, it is there
+**NOTE:** The `oozie-default.xml` file found in Oozie's `conf/` directory is not used by Oozie, it is there
for reference purposes only.
----+++ Precedence of Configuration Properties
+### Precedence of Configuration Properties
For compatibility reasons across Hadoop / Oozie versions, some configuration properties can be defined using multiple keys
in the launcher configuration. Beginning with Oozie 5.0.0, some of them can be overridden, some others will be prepended to default
configuration values.
----++++ Overriding Configuration Values
+#### Overriding Configuration Values
-Overriding happens for following configuration entries with =oozie.launcher= prefix, by switching =oozie.launcher.override=
+Overriding happens for following configuration entries with `oozie.launcher` prefix, by switching `oozie.launcher.override`
(on by default).
For those, following is the general approach:
+
* check whether a YARN compatible entry is present. If yes, use it to override default value
* check whether a MapReduce v2 compatible entry is present. If yes, use it to override default value
* check whether a MapReduce v1 compatible entry is present. If yes, use it to override default value
* use default value
Such properties are (legend: YARN / MapReduce v2 / MapReduce v1):
+
* max attempts of the MapReduce Application Master:
* N / A
- * =mapreduce.map.maxattempts=
- * =mapred.map.max.attempts=
+ * `mapreduce.map.maxattempts`
+ * `mapred.map.max.attempts`
* memory amount in MB of the MapReduce Application Master:
- * =yarn.app.mapreduce.am.resource.mb=
- * =mapreduce.map.memory.mb=
- * =mapred.job.map.memory.mb=
+ * `yarn.app.mapreduce.am.resource.mb`
+ * `mapreduce.map.memory.mb`
+ * `mapred.job.map.memory.mb`
* CPU vcore count of the MapReduce Application Master:
- * =yarn.app.mapreduce.am.resource.cpu-vcores=
- * =mapreduce.map.cpu.vcores=
+ * `yarn.app.mapreduce.am.resource.cpu-vcores`
+ * `mapreduce.map.cpu.vcores`
* N / A
* logging level of the MapReduce Application Master:
* N / A
- * =mapreduce.map.log.level=
- * =mapred.map.child.log.level=
+ * `mapreduce.map.log.level`
+ * `mapred.map.child.log.level`
* MapReduce Application Master JVM options:
- * =yarn.app.mapreduce.am.command-opts=
- * =mapreduce.map.java.opts=
- * =mapred.child.java.opts=
+ * `yarn.app.mapreduce.am.command-opts`
+ * `mapreduce.map.java.opts`
+ * `mapred.child.java.opts`
* MapReduce Application Master environment variable settings:
- * =yarn.app.mapreduce.am.env=
- * =mapreduce.map.env=
- * =mapred.child.env=
+ * `yarn.app.mapreduce.am.env`
+ * `mapreduce.map.env`
+ * `mapred.child.env`
* MapReduce Application Master job priority:
* N / A
- * =mapreduce.job.priority=
- * =mapred.job.priority=
+ * `mapreduce.job.priority`
+ * `mapred.job.priority`
* MapReduce Application Master job queue name:
* N / A
- * =mapreduce.job.queuename=
- * =mapred.job.queue.name=
+ * `mapreduce.job.queuename`
+ * `mapred.job.queue.name`
* MapReduce View ACL settings:
* N / A
- * =mapreduce.job.acl-view-job=
+ * `mapreduce.job.acl-view-job`
* N / A
* MapReduce Modify ACL settings:
* N / A
- * =mapreduce.job.acl-modify-job=
+ * `mapreduce.job.acl-modify-job`
* N / A
This list can be extended or modified by adding new configuration entries or updating existing values
-beginning with =oozie.launcher.override.= within =oozie-site.xml=. Examples can be found in =oozie-default.xml=.
+beginning with `oozie.launcher.override.` within `oozie-site.xml`. Examples can be found in `oozie-default.xml`.
----++++ Prepending Configuration Values
+#### Prepending Configuration Values
-Prepending happens for following configuration entries with =oozie.launcher= prefix, by switching =oozie.launcher.prepend=
+Prepending happens for following configuration entries with `oozie.launcher` prefix, by switching `oozie.launcher.prepend`
(on by default).
For those, following is the general approach:
+
* check whether a YARN compatible entry is present. If yes, use it to prepend to default value
* use default value
Such properties are (legend: YARN only):
- * MapReduce Application Master JVM options: =yarn.app.mapreduce.am.admin-command-opts=
- * MapReduce Application Master environment settings: =yarn.app.mapreduce.am.admin.user.env=
+
+ * MapReduce Application Master JVM options: `yarn.app.mapreduce.am.admin-command-opts`
+ * MapReduce Application Master environment settings: `yarn.app.mapreduce.am.admin.user.env`
This list can be extended or modified by adding new configuration entries or updating existing values
-beginning with =oozie.launcher.prepend.= within =oozie-site.xml=. Examples can be found in =oozie-default.xml=.
+beginning with `oozie.launcher.prepend.` within `oozie-site.xml`. Examples can be found in `oozie-default.xml`.
----+++ Logging Configuration
+### Logging Configuration
-By default, Oozie log configuration is defined in the =oozie-log4j.properties= configuration file.
+By default, Oozie log configuration is defined in the `oozie-log4j.properties` configuration file.
If the Oozie log configuration file changes, Oozie reloads the new settings automatically.
-By default, Oozie logs to Oozie's =logs/= directory.
+By default, Oozie logs to Oozie's `logs/` directory.
Oozie logs in 4 different files:
@@ -395,19 +407,19 @@
* oozie-instrumentation.log: instrumentation data, every 60 seconds (configurable)
* oozie-audit.log: audit messages, workflow jobs changes
-The embedded Jetty and embedded Derby log files are also written to Oozie's =logs/= directory.
+The embedded Jetty and embedded Derby log files are also written to Oozie's `logs/` directory.
----+++ Oozie User Authentication Configuration
+### Oozie User Authentication Configuration
Oozie supports Kerberos HTTP SPNEGO authentication, pseudo/simple authentication and anonymous access
for client connections.
-Anonymous access (*default*) does not require the user to authenticate and the user ID is obtained from
+Anonymous access (**default**) does not require the user to authenticate and the user ID is obtained from
the job properties on job submission operations, other operations are anonymous.
Pseudo/simple authentication requires the user to specify the user name on the request, this is done by
-the PseudoAuthenticator class by injecting the =user.name= parameter in the query string of all requests.
-The =user.name= parameter value is taken from the client process Java System property =user.name=.
+the PseudoAuthenticator class by injecting the `user.name` parameter in the query string of all requests.
+The `user.name` parameter value is taken from the client process Java System property `user.name`.
Kerberos HTTP SPNEGO authentication requires the user to perform a Kerberos HTTP SPNEGO authentication sequence.
@@ -419,7 +431,8 @@
Oozie user authentication is configured using the following configuration properties (default values shown):
-<verbatim>
+
+```
oozie.authentication.type=simple
oozie.authentication.token.validity=36000
oozie.authentication.signature.secret=
@@ -427,63 +440,66 @@
oozie.authentication.simple.anonymous.allowed=true
oozie.authentication.kerberos.principal=HTTP/localhost@${local.realm}
oozie.authentication.kerberos.keytab=${oozie.service.HadoopAccessorService.keytab.file}
-</verbatim>
+```
-The =type= defines authentication used for Oozie HTTP endpoint, the supported values are:
+The `type` defines authentication used for Oozie HTTP endpoint, the supported values are:
simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME#.
-The =token.validity= indicates how long (in seconds) an authentication token is valid before it has
+The `token.validity` indicates how long (in seconds) an authentication token is valid before it has
to be renewed.
-The =signature.secret= is the signature secret for signing the authentication tokens. It is recommended to not set this, in which
+The `signature.secret` is the signature secret for signing the authentication tokens. It is recommended to not set this, in which
case Oozie will randomly generate one on startup.
-The =oozie.authentication.cookie.domain= The domain to use for the HTTP cookie that stores the
+The `oozie.authentication.cookie.domain` The domain to use for the HTTP cookie that stores the
authentication token. In order to authentication to work correctly across all Hadoop nodes web-consoles
the domain must be correctly set.
-The =simple.anonymous.allowed= indicates if anonymous requests are allowed. This setting is meaningful
+The `simple.anonymous.allowed` indicates if anonymous requests are allowed. This setting is meaningful
only when using 'simple' authentication.
-The =kerberos.principal= indicates the Kerberos principal to be used for HTTP endpoint.
+The `kerberos.principal` indicates the Kerberos principal to be used for HTTP endpoint.
The principal MUST start with 'HTTP/' as per Kerberos HTTP SPNEGO specification.
-The =kerberos.keytab= indicates the location of the keytab file with the credentials for the principal.
+The `kerberos.keytab` indicates the location of the keytab file with the credentials for the principal.
It should be the same keytab file Oozie uses for its Kerberos credentials for Hadoop.
----+++ Oozie Hadoop Authentication Configuration
+### Oozie Hadoop Authentication Configuration
Oozie works with Hadoop versions which support Kerberos authentication.
Oozie Hadoop authentication is configured using the following configuration properties (default values shown):
-<verbatim>
+
+```
oozie.service.HadoopAccessorService.kerberos.enabled=false
local.realm=LOCALHOST
oozie.service.HadoopAccessorService.keytab.file=${user.home}/oozie.keytab
oozie.service.HadoopAccessorService.kerberos.principal=${user.name}/localhost@{local.realm}
-</verbatim>
+```
The above default values are for a Hadoop 0.20 secure distribution (with support for Kerberos authentication).
To enable Kerberos authentication, the following property must be set:
-<verbatim>
+
+```
oozie.service.HadoopAccessorService.kerberos.enabled=true
-</verbatim>
+```
When using Kerberos authentication, the following properties must be set to the correct values (default values shown):
-<verbatim>
+
+```
local.realm=LOCALHOST
oozie.service.HadoopAccessorService.keytab.file=${user.home}/oozie.keytab
oozie.service.HadoopAccessorService.kerberos.principal=${user.name}/localhost@{local.realm}
-</verbatim>
+```
-*IMPORTANT:* When using Oozie with a Hadoop 20 with Security distribution, the Oozie user in Hadoop must be configured
+**IMPORTANT:** When using Oozie with a Hadoop 20 with Security distribution, the Oozie user in Hadoop must be configured
as a proxy user.
----+++ User ProxyUser Configuration
+### User ProxyUser Configuration
Oozie supports impersonation or proxyuser functionality (identical to Hadoop proxyuser capabilities and conceptually
similar to Unix 'sudo').
@@ -504,7 +520,7 @@
Both properties support the '*' wildcard as value. Although this is recommended only for testing/development.
----+++ User Authorization Configuration
+### User Authorization Configuration
Oozie has a basic authorization model:
@@ -519,70 +535,74 @@
Oozie security is set via the following configuration property (default value shown):
-<verbatim>
+
+```
oozie.service.AuthorizationService.security.enabled=false
-</verbatim>
+```
NOTE: the old ACL model where a group was provided is still supported if the following property is set
-in =oozie-site.xml=:
+in `oozie-site.xml`:
-<verbatim>
+
+```
oozie.service.AuthorizationService.default.group.as.acl=true
-</verbatim>
+```
----++++ Defining Admin Users
+#### Defining Admin Users
Admin users are determined from the list of admin groups, specified in
- =oozie.service.AuthorizationService.admin.groups= property. Use commas to separate multiple groups, spaces, tabs
+ `oozie.service.AuthorizationService.admin.groups` property. Use commas to separate multiple groups, spaces, tabs
and ENTER characters are trimmed.
If the above property for admin groups is not set, then defining the admin users can happen in the following manners.
-The list of admin users can be in the =conf/adminusers.txt= file. The syntax of this file is:
+The list of admin users can be in the `conf/adminusers.txt` file. The syntax of this file is:
* One user name per line
* Empty lines and lines starting with '#' are ignored
Admin users can also be defined in
-=oozie.serviceAuthorizationService.admin.users= property. Use commas to separate multiple admin users, spaces, tabs
+`oozie.serviceAuthorizationService.admin.users` property. Use commas to separate multiple admin users, spaces, tabs
and ENTER characters are trimmed.
In case there are admin users defined using both methods, the effective list of admin users will be the union
-of the admin users found in the adminusers.txt and those specified with =oozie.serviceAuthorizationService.admin.users=.
+of the admin users found in the adminusers.txt and those specified with `oozie.serviceAuthorizationService.admin.users`.
----++++ Defining Access Control Lists
+#### Defining Access Control Lists
Access Control Lists are defined in the following ways:
- * workflow job submission over CLI: configuration property =group.name= of =job.properties=
- * workflow job submission over HTTP: configuration property =group.name= of the XML submitted over HTTP
- * workflow job re-run: configuration property =oozie.job.acl= (preferred) or configuration property =group.name= of
- =job.properties=
- * coordinator job submission over CLI: configuration property =oozie.job.acl= (preferred) or configuration property =group.name=
- of =job.properties=
- * bundle job submission over CLI: configuration property =oozie.job.acl= (preferred) or configuration property =group.name= of
- =job.properties=
+
+ * workflow job submission over CLI: configuration property `group.name` of `job.properties`
+ * workflow job submission over HTTP: configuration property `group.name` of the XML submitted over HTTP
+ * workflow job re-run: configuration property `oozie.job.acl` (preferred) or configuration property `group.name` of
+ `job.properties`
+ * coordinator job submission over CLI: configuration property `oozie.job.acl` (preferred) or configuration property `group.name`
+ of `job.properties`
+ * bundle job submission over CLI: configuration property `oozie.job.acl` (preferred) or configuration property `group.name` of
+ `job.properties`
For all other workflow, coordinator, or bundle actions the ACL set in beforehand will be used as basis.
Once the ACL for the job is defined, Oozie will check over HDFS whether the user trying to perform a specific action is part of the
-necessary group(s). For implementation details please check out =org.apache.hadoop.security.Groups#getGroups(String user)=.
+necessary group(s). For implementation details please check out `org.apache.hadoop.security.Groups#getGroups(String user)`.
Note that it's enough that the submitting user be part of at least one group of the ACL. Note also that the ACL can contain user
names as well. If there is an ACL defined and the submitting user isn't part of any group or user name present in the ACL, an
-=AuthorizationException= is thrown.
+`AuthorizationException` is thrown.
-*Example: A typical ACL setup*
+**Example: A typical ACL setup**
-Detail of =job.properties= on workflow job submission:
-<verbatim>
+Detail of `job.properties` on workflow job submission:
+
+```
user.name=joe
group.name=marketing,admin,qa,root
-<verbatim>
+```
-HDFS group membership of HDFS user =joe= is =qa=. That is, the check to =org.apache.hadoop.security.Groups#getGroups("joe")= returns
-=qa=. Hence, ACL check will pass inside =AuthorizationService=, because the =user.name= provided belongs to at least of the ACL list
-elements provided as =group.name=.
+HDFS group membership of HDFS user `joe` is `qa`. That is, the check to `org.apache.hadoop.security.Groups#getGroups("joe")` returns
+`qa`. Hence, ACL check will pass inside `AuthorizationService`, because the `user.name` provided belongs to at least of the ACL list
+elements provided as `group.name`.
----+++ Oozie System ID Configuration
+### Oozie System ID Configuration
Oozie has a system ID that is is used to generate the Oozie temporary runtime directory, the workflow job IDs, and the
workflow action IDs.
@@ -591,53 +611,57 @@
to identify resources created/used by the different Oozie systems if they have different system IDs (default value
shown):
-<verbatim>
- oozie.system.id=oozie-${user.name}
-</verbatim>
----+++ Filesystem Configuration
+```
+ oozie.system.id=oozie-${user.name}
+```
+
+### Filesystem Configuration
Oozie lets you to configure the allowed Filesystems by using the following configuration property in oozie-site.xml:
-<verbatim>
+
+```
<property>
<name>oozie.service.HadoopAccessorService.supported.filesystems</name>
<value>hdfs</value>
</property>
-</verbatim>
+```
-The above value, =hdfs=, which is the default, means that Oozie will only allow HDFS filesystems to be used. Examples of other
+The above value, `hdfs`, which is the default, means that Oozie will only allow HDFS filesystems to be used. Examples of other
filesystems that Oozie is compatible with are: hdfs, hftp, webhdfs, and viewfs. Multiple filesystems can be specified as
comma-separated values. Putting a * will allow any filesystem type, effectively disabling this check.
----+++ HCatalog Configuration
+### HCatalog Configuration
-Refer to the [[DG_HCatalogIntegration][Oozie HCatalog Integration]] document for a overview of HCatalog and
+Refer to the [Oozie HCatalog Integration](DG_HCatalogIntegration.html) document for a overview of HCatalog and
integration of Oozie with HCatalog. This section explains the various settings to be configured in oozie-site.xml on
the Oozie server to enable Oozie to work with HCatalog.
-*Adding HCatalog jars to Oozie war:*
+**Adding HCatalog jars to Oozie war:**
For Oozie server to talk to HCatalog server, HCatalog and hive jars need to be in the server classpath.
hive-site.xml which has the configuration to talk to the HCatalog server also needs to be in the classpath or specified by the
following configuration property in oozie-site.xml:
-<verbatim>
+
+```
<property>
<name>oozie.service.HCatAccessorService.hcat.configuration</name>
<value>/local/filesystem/path/to/hive-site.xml</value>
</property>
-</verbatim>
+```
The hive-site.xml can also be placed in a location on HDFS and the above property can have a value
-of =hdfs://HOST:PORT/path/to/hive-site.xml= to point there instead of the local file system.
+of `hdfs://HOST:PORT/path/to/hive-site.xml` to point there instead of the local file system.
The oozie-[version]-hcataloglibs.tar.gz in the oozie distribution bundles the required hcatalog and hive jars that
needs to be placed in the Oozie server classpath. If using a version of HCatalog bundled in
Oozie hcataloglibs/, copy the corresponding HCatalog jars from hcataloglibs/ to the libext/ directory. If using a
different version of HCatalog, copy the required HCatalog jars from such version in the libext/ directory.
-This needs to be done before running the =oozie-setup.sh= script so that these jars get added for Oozie.
+This needs to be done before running the `oozie-setup.sh` script so that these jars get added for Oozie.
-*Configure HCatalog URI Handling:*
+**Configure HCatalog URI Handling:**
-<verbatim>
+
+```
<property>
<name>oozie.service.URIHandlerService.uri.handlers</name>
<value>org.apache.oozie.dependency.FSURIHandler,org.apache.oozie.dependency.HCatURIHandler</value>
@@ -645,16 +669,17 @@
Enlist the different uri handlers supported for data availability checks.
</description>
</property>
-</verbatim>
+```
The above configuration defines the different uri handlers which check for existence of data dependencies defined in a
-Coordinator. The default value is =org.apache.oozie.dependency.FSURIHandler=. FSURIHandler supports uris with
-schemes defined in the configuration =oozie.service.HadoopAccessorService.supported.filesystems= which are hdfs, hftp
+Coordinator. The default value is `org.apache.oozie.dependency.FSURIHandler`. FSURIHandler supports uris with
+schemes defined in the configuration `oozie.service.HadoopAccessorService.supported.filesystems` which are hdfs, hftp
and webhcat by default. HCatURIHandler supports uris with the scheme as hcat.
-*Configure HCatalog services:*
+**Configure HCatalog services:**
-<verbatim>
+
+```
<property>
<name>oozie.services.ext</name>
<value>
@@ -667,16 +692,17 @@
Class names must be separated by commas.
</description>
</property>
-</verbatim>
+```
PartitionDependencyManagerService and HCatAccessorService are required to work with HCatalog and support Coordinators
having HCatalog uris as data dependency. If the HCatalog server is configured to publish partition availability
notifications to a JMS compliant messaging provider like ActiveMQ, then JMSAccessorService needs to be added
-to =oozie.services.ext= to handle those notifications.
+to `oozie.services.ext` to handle those notifications.
-*Configure JMS Provider JNDI connection mapping for HCatalog:*
+**Configure JMS Provider JNDI connection mapping for HCatalog:**
-<verbatim>
+
+```
<property>
<name>oozie.service.HCatAccessorService.jmsconnections</name>
<value>
@@ -692,21 +718,22 @@
hcat://${1}.${2}.com:8020=java.naming.factory.initial#Dummy.Factory;java.naming.provider.url#tcp://broker.${2}.com:61616
</description>
</property>
-</verbatim>
+```
Currently HCatalog does not provide APIs to get the connection details to connect to the JMS Provider it publishes
notifications to. It only has APIs which provide the topic name in the JMS Provider to which the notifications are
published for a given database table. So the JMS Provider's connection properties needs to be manually configured
-in Oozie using the above setting. You can either provide a =default= JNDI configuration which will be used as the
+in Oozie using the above setting. You can either provide a `default` JNDI configuration which will be used as the
JMS Provider for all HCatalog servers, or can specify a configuration per HCatalog server URL or provide a
configuration based on a rule matching multiple HCatalog server URLs. For example: With the configuration of
-hcat://${1}.${2}.com:8020=java.naming.factory.initial#Dummy.Factory;java.naming.provider.url#tcp://broker.${2}.com:61616,
-request URL of hcat://server1.colo1.com:8020 will map to tcp://broker.colo1.com:61616, hcat://server2.colo2.com:8020
-will map to tcp://broker.colo2.com:61616 and so on.
+`hcat://${1}.${2}.com:8020=java.naming.factory.initial#Dummy.Factory;java.naming.provider.url#tcp://broker.${2}.com:61616`,
+request URL of `hcat://server1.colo1.com:8020 `will map to`tcp://broker.colo1.com:61616`, `hcat://server2.colo2.com:8020`
+will map to`tcp://broker.colo2.com:61616` and so on.
-*Configure HCatalog Polling Frequency:*
+**Configure HCatalog Polling Frequency:**
-<verbatim>
+
+```
<property>
<name>oozie.service.coord.push.check.requeue.interval
</name>
@@ -714,29 +741,30 @@
<description>Command re-queue interval for push dependencies (in millisecond).
</description>
</property>
-</verbatim>
+```
If there is no JMS Provider configured for a HCatalog Server, then oozie polls HCatalog based on the frequency defined
-in =oozie.service.coord.input.check.requeue.interval=. This config also applies to HDFS polling.
+in `oozie.service.coord.input.check.requeue.interval`. This config also applies to HDFS polling.
If there is a JMS provider configured for a HCatalog Server, then oozie polls HCatalog based on the frequency defined
-in =oozie.service.coord.push.check.requeue.interval= as a fallback.
-The defaults for =oozie.service.coord.input.check.requeue.interval= and =oozie.service.coord.push.check.requeue.interval=
+in `oozie.service.coord.push.check.requeue.interval` as a fallback.
+The defaults for `oozie.service.coord.input.check.requeue.interval` and `oozie.service.coord.push.check.requeue.interval`
are 1 minute and 10 minutes respectively.
----+++ Notifications Configuration
+### Notifications Configuration
Oozie supports publishing notifications to a JMS Provider for job status changes and SLA met and miss events. For
-more information on the feature, refer [[DG_JMSNotifications][JMS Notifications]] documentation. Oozie can also send email
+more information on the feature, refer [JMS Notifications](DG_JMSNotifications.html) documentation. Oozie can also send email
notifications on SLA misses.
- * *Message Broker Installation*: <br/>
+ * **Message Broker Installation**: <br/>
For Oozie to send/receive messages, a JMS-compliant broker should be installed. Apache ActiveMQ is a popular JMS-compliant
-broker usable for this purpose. See [[http://activemq.apache.org/getting-started.html][here]] for instructions on
+broker usable for this purpose. See [here](http://activemq.apache.org/getting-started.html) for instructions on
installing and running ActiveMQ.
- * *Services*: <br/>
-Add/modify =oozie.services.ext= property in =oozie-site.xml= to include the following services.
- <verbatim>
+ * **Services**: <br/>
+Add/modify `oozie.services.ext` property in `oozie-site.xml` to include the following services.
+
+```
<property>
<name>oozie.services.ext</name>
<value>
@@ -746,10 +774,11 @@
org.apache.oozie.sla.service.SLAService
</value>
</property>
- </verbatim>
+```
- * *Event Handlers*: <br/>
- <verbatim>
+ * **Event Handlers**: <br/>
+
+```
<property>
<name>oozie.service.EventHandlerService.event.listeners</name>
<value>
@@ -759,30 +788,33 @@
org.apache.oozie.sla.listener.SLAEmailEventListener
</value>
</property>
- </verbatim>
- It is also recommended to increase =oozie.service.SchedulerService.threads= to 15 for faster event processing and sending notifications. The services and their functions are as follows: <br/>
+```
+It is also recommended to increase `oozie.service.SchedulerService.threads` to 15 for faster event processing and sending notifications. The services and their functions are as follows: <br/>
JMSJobEventListener - Sends JMS job notifications <br/>
JMSSLAEventListener - Sends JMS SLA notifications <br/>
SLAEmailEventListener - Sends Email SLA notifications <br/>
SLAJobEventListener - Processes job events and calculates SLA. Does not send any notifications
- * *JMS properties*: <br/>
-Add =oozie.jms.producer.connection.properties= property in =oozie-site.xml=. Its value corresponds to an
-identifier (e.g. default) assigned to a semi-colon separated key#value list of properties from your JMS broker's
-=jndi.properties= file. The important properties are =java.naming.factory.initial= and =java.naming.provider.url=.
- As an example, if using ActiveMQ in local env, the property can be set to
- <verbatim>
+ * **JMS properties**: <br/>
+Add `oozie.jms.producer.connection.properties` property in `oozie-site.xml`. Its value corresponds to an
+identifier (e.g. default) assigned to a semi-colon separated key#value list of properties from your JMS broker's
+`jndi.properties` file. The important properties are `java.naming.factory.initial` and `java.naming.provider.url`.
+
+As an example, if using ActiveMQ in local env, the property can be set to
+
+```
<property>
<name>oozie.jms.producer.connection.properties</name>
<value>
java.naming.factory.initial#org.apache.activemq.jndi.ActiveMQInitialContextFactory;java.naming.provider.url#tcp://localhost:61616;connectionFactoryNames#ConnectionFactory
</value>
</property>
- </verbatim>
- * *JMS Topic name*: <br/>
+```
+ * **JMS Topic name**: <br/>
JMS consumers listen on a particular "topic". Hence Oozie needs to define a topic variable with which to publish messages
about the various jobs.
- <verbatim>
+
+```
<property>
<name>oozie.service.JMSTopicService.topic.name</name>
<value>
@@ -802,10 +834,11 @@
For jobs with no defined topic, default topic will be ${username}
</description>
</property>
- </verbatim>
+```
- Another related property is the topic prefix.
- <verbatim>
+Another related property is the topic prefix.
+
+```
<property>
<name>oozie.service.JMSTopicService.topic.prefix</name>
<value></value>
@@ -813,17 +846,17 @@
This can be used to append a prefix to the topic in oozie.service.JMSTopicService.topic.name. For eg: oozie.
</description>
</property>
- </verbatim>
+```
----+++ Setting Up Oozie with HTTPS (SSL)
+### Setting Up Oozie with HTTPS (SSL)
-*IMPORTANT*:
+**IMPORTANT**:
The default HTTPS configuration will cause all Oozie URLs to use HTTPS except for the JobTracker callback URLs. This is to simplify
configuration (no changes needed outside of Oozie), but this is okay because Oozie doesn't inherently trust the callbacks anyway;
they are used as hints.
-The related environment variables are explained at [[AG_Install#Environment_Setup][Environment Setup]].
+The related environment variables are explained at [Environment Setup](AG_Install.html#Environment_Setup).
You can use either a certificate from a Certificate Authority or a Self-Signed Certificate. Using a self-signed certificate
requires some additional configuration on each Oozie client machine. If possible, a certificate from a Certificate Authority is
@@ -831,138 +864,147 @@
There's also some additional considerations when using Oozie HA with HTTPS.
----++++To use a Self-Signed Certificate
+#### To use a Self-Signed Certificate
There are many ways to create a Self-Signed Certificate, this is just one way. We will be using
-the [[http://docs.oracle.com/javase/6/docs/technotes/tools/solaris/keytool.html][keytool]] program, which is
+the [keytool](http://docs.oracle.com/javase/6/docs/technotes/tools/solaris/keytool.html) program, which is
included with your JRE. If it's not on your path, you should be able to find it in $JAVA_HOME/bin.
-1. Run the following command (as the Oozie user) to create the keystore file, which will be named =.keystore= and located in the
-Oozie user's home directory.
-<verbatim>
-keytool -genkeypair -alias jetty -keyalg RSA -dname "CN=hostname" -storepass password -keypass password
-</verbatim>
-The =hostname= should be the host name of the Oozie Server or a wildcard on the subdomain it belongs to. Make sure to include
-the "CN=" part. You can change =storepass= and =keypass= values, but they should be the same. If you do want to use something
-other than password, you'll also need to change the value of the =oozie.https.keystore.pass= property in =oozie-site.xml= to
-match; =password= is the default.
+1. Run the following command (as the Oozie user) to create the keystore file, which will be named `.keystore` and located in the
+ Oozie user's home directory.
-For example, if your Oozie server was at oozie.int.example.com, then you would do this:
-<verbatim>
-keytool -genkeypair -alias jetty -keyalg RSA -dname "CN=oozie.int.example.com" -storepass password -keypass password
-</verbatim>
-If you're going to be using Oozie HA, it's simplest if you have a single certificate that all Oozie servers in the HA group can use.
-To do that, you'll need to use a wildcard on the subdomain it belongs to:
-<verbatim>
-keytool -genkeypair -alias jetty -keyalg RSA -dname "CN=*.int.example.com" -storepass password -keypass password
-</verbatim>
-The above would work on any server in the int.example.com domain.
+ ```
+ keytool -genkeypair -alias jetty -keyalg RSA -dname "CN=hostname" -storepass password -keypass password
+ ```
+ The `hostname` should be the host name of the Oozie Server or a wildcard on the subdomain it belongs to. Make sure to include
+ the "CN=" part. You can change `storepass` and `keypass` values, but they should be the same. If you do want to use something
+ other than password, you'll also need to change the value of the `oozie.https.keystore.pass` property in `oozie-site.xml` to
+ match; `password` is the default.
+
+ For example, if your Oozie server was at oozie.int.example.com, then you would do this:
+
+ ```
+ keytool -genkeypair -alias jetty -keyalg RSA -dname "CN=oozie.int.example.com" -storepass password -keypass password
+ ```
+ If you're going to be using Oozie HA, it's simplest if you have a single certificate that all Oozie servers in the HA group can use.
+ To do that, you'll need to use a wildcard on the subdomain it belongs to:
+
+ ```
+ keytool -genkeypair -alias jetty -keyalg RSA -dname "CN=*.int.example.com" -storepass password -keypass password
+ ```
+ The above would work on any server in the int.example.com domain.
2. Run the following command (as the Oozie user) to export a certificate file from the keystore file:
-<verbatim>
-keytool -exportcert -alias jetty -file path/to/anywhere/certificate.cert -storepass password
-</verbatim>
+
+ ```
+ keytool -exportcert -alias jetty -file path/to/anywhere/certificate.cert -storepass password
+ ```
3. Run the following command (as any user) to create a truststore containing the certificate we just exported:
-<verbatim>
-keytool -import -alias jetty -file path/to/certificate.cert -keystore /path/to/anywhere/oozie.truststore -storepass password2
-</verbatim>
-You'll need the =oozie.truststore= later if you're using the Oozie client (or other Java-based client); otherwise, you can skip
-this step. The =storepass= value here is only used to verify or change the truststore and isn't typically required when only
-reading from it; so it does not have to be given to users only using the client.
----++++To use a Certificate from a Certificate Authority
+ ```
+ keytool -import -alias jetty -file path/to/certificate.cert -keystore /path/to/anywhere/oozie.truststore -storepass password2
+ ```
+ You'll need the `oozie.truststore` later if you're using the Oozie client (or other Java-based client); otherwise, you can skip
+ this step. The `storepass` value here is only used to verify or change the truststore and isn't typically required when only
+ reading from it; so it does not have to be given to users only using the client.
+
+#### To use a Certificate from a Certificate Authority
1. You will need to make a request to a Certificate Authority in order to obtain a proper Certificate; please consult a Certificate
-Authority on this procedure. If you're going to be using Oozie HA, it's simplest if you have a single certificate that all Oozie
-servers in the HA group can use. To do that, you'll need to use a wild on the subdomain it belongs to (e.g. "*.int.example.com").
+ Authority on this procedure. If you're going to be using Oozie HA, it's simplest if you have a single certificate that all Oozie
+ servers in the HA group can use. To do that, you'll need to use a wild on the subdomain it belongs to (e.g. "*.int.example.com").
2. Once you have your .cert file, run the following command (as the Oozie user) to create a keystore file from your certificate:
-<verbatim>
-keytool -import -alias jetty -file path/to/certificate.cert
-</verbatim>
-The keystore file will be named =.keystore= and located in the Oozie user's home directory.
----++++Configure the Oozie Server to use SSL (HTTPS)
+ ```
+ keytool -import -alias jetty -file path/to/certificate.cert
+ ```
+ The keystore file will be named `.keystore` and located in the Oozie user's home directory.
+
+#### Configure the Oozie Server to use SSL (HTTPS)
1. Make sure the Oozie server isn't running
-2. Configure settings necessary for enabling SSL/TLS support in =oozie-site.xml=.
+2. Configure settings necessary for enabling SSL/TLS support in `oozie-site.xml`.
-2a. Set =oozie.https.enabled= to =true=. To revert back to HTTP, set =oozie.https.enabled= to =false=.
-2b. Set location and password for the keystore and location for truststore by setting =oozie.https.keystore.file=,
-=oozie.https.keystore.pass=, =oozie.https.truststore.file=.
+ 2a. Set `oozie.https.enabled` to `true`. To revert back to HTTP, set `oozie.https.enabled` to `false`.
-*Note:* =oozie.https.truststore.file= can be overridden by setting =javax.net.ssl.trustStore= system property.
+ 2b. Set location and password for the keystore and location for truststore by setting `oozie.https.keystore.file`,
+ `oozie.https.keystore.pass`, `oozie.https.truststore.file`.
-The default HTTPS port Oozie listens on for secure connections is 11443; it can be changed via =oozie.https.port=.
+ **Note:** `oozie.https.truststore.file` can be overridden by setting `javax.net.ssl.trustStore` system property.
-It is possible to specify other HTTPS settings via =oozie-site.xml=:
-- To include / exclude cipher suites, set =oozie.https.include.cipher.suites= / =oozie.https.exclude.cipher.suites=.
-- To include / exclude TLS protocols, set =oozie.https.include.protocols= / =oozie.https.exclude.protocols=.
-*Note:* Exclude is always preferred over include (i.e. if you both include and exclude an entity, it will be excluded).
+ The default HTTPS port Oozie listens on for secure connections is 11443; it can be changed via `oozie.https.port`.
+
+ It is possible to specify other HTTPS settings via `oozie-site.xml`:
+ - To include / exclude cipher suites, set `oozie.https.include.cipher.suites` / `oozie.https.exclude.cipher.suites`.
+ - To include / exclude TLS protocols, set `oozie.https.include.protocols` / `oozie.https.exclude.protocols`.
+ **Note:** Exclude is always preferred over include (i.e. if you both include and exclude an entity, it will be excluded).
3. Start the Oozie server
-*Note:* If using Oozie HA, make sure that each Oozie server has a copy of the .keystore file.
+ **Note:** If using Oozie HA, make sure that each Oozie server has a copy of the .keystore file.
----++++Configure the Oozie Client to connect using SSL (HTTPS)
+#### Configure the Oozie Client to connect using SSL (HTTPS)
The first two steps are only necessary if you are using a Self-Signed Certificate; the third is required either way.
Also, these steps must be done on every machine where you intend to use the Oozie Client.
1. Copy or download the oozie.truststore file onto the client machine
-2. When using any Java-based program, you'll need to pass =-Djavax.net.ssl.trustStore= to the JVM. To
-do this for the Oozie client:
-<verbatim>
-export OOZIE_CLIENT_OPTS='-Djavax.net.ssl.trustStore=/path/to/oozie.truststore'
-</verbatim>
+2. When using any Java-based program, you'll need to pass `-Djavax.net.ssl.trustStore` to the JVM. To
+ do this for the Oozie client:
-3. When using the Oozie Client, you will need to use https://oozie.server.hostname:11443/oozie instead of
-http://oozie.server.hostname:11000/oozie -- Java will not automatically redirect from the http address to the https address.
+ ```
+ export OOZIE_CLIENT_OPTS='-Djavax.net.ssl.trustStore=/path/to/oozie.truststore'
+ ```
----++++Connect to the Oozie Web UI using SSL (HTTPS)
+3. When using the Oozie Client, you will need to use `https://oozie.server.hostname:11443/oozie` instead of
+ `http://oozie.server.hostname:11000/oozie` -- Java will not automatically redirect from the http address to the https address.
-1. Use https://oozie.server.hostname:11443/oozie
-though most browsers should automatically redirect you if you use http://oozie.server.hostname:11000/oozie
+#### Connect to the Oozie Web UI using SSL (HTTPS)
-*IMPORTANT*: If using a Self-Signed Certificate, your browser will warn you that it can't verify the certificate or something
-similar. You will probably have to add your certificate as an exception.
+1. Use `https://oozie.server.hostname:11443/oozie`
+ though most browsers should automatically redirect you if you use `http://oozie.server.hostname:11000/oozie`
----++++Additional considerations for Oozie HA with SSL
+ **IMPORTANT**: If using a Self-Signed Certificate, your browser will warn you that it can't verify the certificate or something
+ similar. You will probably have to add your certificate as an exception.
+
+#### Additional considerations for Oozie HA with SSL
You'll need to configure the load balancer to do SSL pass-through. This will allow the clients talking to Oozie to use the
SSL certificate provided by the Oozie servers (so the load balancer does not need one). Please consult your load balancer's
-documentation on how to configure this. Make sure to point the load balancer at the https://HOST:HTTPS_PORT addresses for your
-Oozie servers. Clients can then connect to the load balancer at https://LOAD_BALANCER_HOST:PORT.
+documentation on how to configure this. Make sure to point the load balancer at the `https://HOST:HTTPS_PORT` addresses for your
+Oozie servers. Clients can then connect to the load balancer at `https://LOAD_BALANCER_HOST:PORT`.
-*Important:* Callbacks from the ApplicationMaster are done via http or https depending on what you enter for the
-=OOZIE_BASE_URL= property. If you are using a Certificate from a Certificate Authority, you can simply put the https address here.
+**Important:** Callbacks from the ApplicationMaster are done via http or https depending on what you enter for the
+`OOZIE_BASE_URL` property. If you are using a Certificate from a Certificate Authority, you can simply put the https address here.
If you are using a self-signed certificate, you have to do one of the following options (Option 1 is recommended):
Option 1) You'll need to follow the steps in
-the [[AG_Install#Configure_the_Oozie_Client_to_connect_using_SSL_HTTPS][Configure the Oozie Client to connect using SSL (HTTPS)]]
-section, but on the host of the ApplicationMaster. You can then set =OOZIE_BASE_URL= to the load balancer https address.
+the [Configure the Oozie Client to connect using SSL (HTTPS)](AG_Install.html#Configure_the_Oozie_Client_to_connect_using_SSL_HTTPS)
+section, but on the host of the ApplicationMaster. You can then set `OOZIE_BASE_URL` to the load balancer https address.
This will allow the ApplicationMaster to contact the Oozie server with https (like the Oozie client, they are also Java
programs).
Option 2) You'll need setup another load balancer, or another "pool" on the existing load balancer, with the http addresses of the
-Oozie servers. You can then set =OOZIE_BASE_URL= to the load balancer http address. Clients should use the https load balancer
+Oozie servers. You can then set `OOZIE_BASE_URL` to the load balancer http address. Clients should use the https load balancer
address. This will allow clients to use https while the ApplicationMaster uses http for callbacks.
----+++ Fine Tuning an Oozie Server
+### Fine Tuning an Oozie Server
-Refer to the [[./oozie-default.xml][oozie-default.xml]] for details.
+Refer to the [oozie-default.xml](./oozie-default.xml) for details.
----+++ Using Instrumentation instead of Metrics
+### Using Instrumentation instead of Metrics
As of version 4.1.0, Oozie includes a replacement for the Instrumentation based on Codahale's Metrics library. It includes a
number of improvements over the original Instrumentation included in Oozie. They both report most of the same information, though
the formatting is slightly different and there's some additional information in the Metrics version; the format of the output to the
oozie-instrumentation log is also different.
-As of version 5.0.0, =MetricsInstrumentationService= is the default one, it's enlisted in =oozie.services=:
- <verbatim>
+As of version 5.0.0, `MetricsInstrumentationService` is the default one, it's enlisted in `oozie.services`:
+
+```
<property>
<name>oozie.services</name>
<value>
@@ -971,11 +1013,12 @@
...
</value>
</property>
- </verbatim>
+```
-The deprecated =InstrumentationService= can be enabled by adding =InstrumentationService= reference to the list of
-=oozie.services.ext=:
- <verbatim>
+The deprecated `InstrumentationService` can be enabled by adding `InstrumentationService` reference to the list of
+`oozie.services.ext`:
+
+```
<property>
<name>oozie.services.ext</name>
<value>
@@ -984,19 +1027,20 @@
...
</value>
</property>
- </verbatim>
+```
-By default the =admin/instrumentation= REST endpoint is no longer be available and instead the =admin/metrics= endpoint can
-be used (see the [[WebServicesAPI#Oozie_Metrics][Web Services API]] documentation for more details); the Oozie Web UI also replaces
+By default the `admin/instrumentation` REST endpoint is no longer be available and instead the `admin/metrics` endpoint can
+be used (see the [Web Services API](WebServicesAPI.html#Oozie_Metrics) documentation for more details); the Oozie Web UI also replaces
the "Instrumentation" tab with a "Metrics" tab.
-If the deprecated =InstrumentationService= is used, the =admin/instrumentation= REST endpoint gets enabled, the =admin/metrics=
-REST endpoint is no longer available (see the [[WebServicesAPI#Oozie_Metrics][Web Services API]] documentation for more details);
+If the deprecated `InstrumentationService` is used, the `admin/instrumentation` REST endpoint gets enabled, the `admin/metrics`
+REST endpoint is no longer available (see the [Web Services API](WebServicesAPI.html#Oozie_Metrics) documentation for more details);
the Oozie Web UI also replaces the "Metrics" tab with the "Instrumentation" tab.
We can also publish the instrumentation metrics to the external server graphite or ganglia. For this the following
properties should be specified in oozie-site.xml :
- <verbatim>
+
+```
<property>
<name>oozie.external_monitoring.enable</name>
<value>false</value>
@@ -1029,11 +1073,12 @@
<name>oozie.external_monitoring.reporterIntervalSecs</name>
<value>60</value>
</property>
- </verbatim>
+```
We can also publish the instrumentation metrics via JMX interface. For this the following property should be specified
in oozie-site.xml :
- <verbatim>
+
+```
<property>
<name>oozie.jmx_monitoring.enable</name>
<value>false</value>
@@ -1041,201 +1086,211 @@
If the oozie functional metrics needs to be exposed via JMX interface, set it to true.
</description>
</property>>
- </verbatim>
+```
-#HA
----+++ High Availability (HA)
+<a name="HA"></a>
+### High Availability (HA)
Multiple Oozie Servers can be configured against the same database to provide High Availability (HA) of the Oozie service.
----++++ Pre-requisites
+#### Pre-requisites
1. A database that supports multiple concurrent connections. In order to have full HA, the database should also have HA support, or
-it becomes a single point of failure.
+ it becomes a single point of failure.
-*NOTE:* The default derby database does not support this
+ **NOTE:** The default derby database does not support this
2. A ZooKeeper ensemble.
-Apache ZooKeeper is a distributed, open-source coordination service for distributed applications; the Oozie servers use it for
-coordinating access to the database and communicating with each other. In order to have full HA, there should be at least 3
-ZooKeeper servers.
-More information on ZooKeeper can be found [[http://zookeeper.apache.org][here]].
+ Apache ZooKeeper is a distributed, open-source coordination service for distributed applications; the Oozie servers use it for
+ coordinating access to the database and communicating with each other. In order to have full HA, there should be at least 3
+ ZooKeeper servers.
+ More information on ZooKeeper can be found [here](http://zookeeper.apache.org).
3. Multiple Oozie servers.
-*IMPORTANT:* While not strictly required for all configuration properties, all of the servers should ideally have exactly the same
-configuration for consistency's sake.
+ **IMPORTANT:** While not strictly required for all configuration properties, all of the servers should ideally have exactly the same
+ configuration for consistency's sake.
4. A Loadbalancer, Virtual IP, or Round-Robin DNS.
-This is used to provide a single entry-point for users and for callbacks from the JobTracker/ResourceManager. The load balancer
-should be configured for round-robin between the Oozie servers to distribute the requests. Users (using either the Oozie client, a
-web browser, or the REST API) should connect through the load balancer. In order to have full HA, the load balancer should also
-have HA support, or it becomes a single point of failure.
+ This is used to provide a single entry-point for users and for callbacks from the JobTracker/ResourceManager. The load balancer
+ should be configured for round-robin between the Oozie servers to distribute the requests. Users (using either the Oozie client, a
+ web browser, or the REST API) should connect through the load balancer. In order to have full HA, the load balancer should also
+ have HA support, or it becomes a single point of failure.
----++++ Installation/Configuration Steps
+#### Installation/Configuration Steps
1. Install identically configured Oozie servers normally. Make sure they are all configured against the same database and make sure
-that you DO NOT start them yet.
+ that you DO NOT start them yet.
2. Add the following services to the extension services configuration property in oozie-site.xml in all Oozie servers. This will
-make Oozie use the ZooKeeper versions of these services instead of the default implementations.
+ make Oozie use the ZooKeeper versions of these services instead of the default implementations.
-<verbatim>
-<property>
- <name>oozie.services.ext</name>
- <value>
- org.apache.oozie.service.ZKLocksService,
- org.apache.oozie.service.ZKXLogStreamingService,
- org.apache.oozie.service.ZKJobsConcurrencyService,
- org.apache.oozie.service.ZKUUIDService
- </value>
-</property>
-</verbatim>
+
+ ```
+ <property>
+ <name>oozie.services.ext</name>
+ <value>
+ org.apache.oozie.service.ZKLocksService,
+ org.apache.oozie.service.ZKXLogStreamingService,
+ org.apache.oozie.service.ZKJobsConcurrencyService,
+ org.apache.oozie.service.ZKUUIDService
+ </value>
+ </property>
+ ```
3. Add the following property to oozie-site.xml in all Oozie servers. It should be a comma-separated list of host:port pairs of the
-ZooKeeper servers. The default value is shown below.
+ ZooKeeper servers. The default value is shown below.
-<verbatim>
-<property>
- <name>oozie.zookeeper.connection.string</name>
- <value>localhost:2181</value>
-</property>
-</verbatim>
+
+ ```
+ <property>
+ <name>oozie.zookeeper.connection.string</name>
+ <value>localhost:2181</value>
+ </property>
+ ```
4. (Optional) Add the following property to oozie-site.xml in all Oozie servers to specify the namespace to use. All of the Oozie
-Servers that are planning on talking to each other should have the same namespace. If there are multiple Oozie setups each doing
-their own HA, they should have their own namespace. The default value is shown below.
+ Servers that are planning on talking to each other should have the same namespace. If there are multiple Oozie setups each doing
+ their own HA, they should have their own namespace. The default value is shown below.
-<verbatim>
-<property>
- <name>oozie.zookeeper.namespace</name>
- <value>oozie</value>
-</property>
-</verbatim>
-5. Change the value of =OOZIE_BASE_URL= in oozie-site.xml to point to the loadbalancer or virtual IP, for example:
+ ```
+ <property>
+ <name>oozie.zookeeper.namespace</name>
+ <value>oozie</value>
+ </property>
+ ```
-<verbatim>
-<property>
- <name>oozie.base.url</name>
- <value>http://my.loadbalancer.hostname:11000/oozie</value>
-</property>
-</verbatim>
+5. Change the value of `OOZIE_BASE_URL` in oozie-site.xml to point to the loadbalancer or virtual IP, for example:
-6. (Optional) If using a secure cluster, see [[AG_Install#Security][Security]] below on configuring Kerberos with Oozie HA.
+
+ ```
+ <property>
+ <name>oozie.base.url</name>
+ <value>http://my.loadbalancer.hostname:11000/oozie</value>
+ </property>
+ ```
+
+6. (Optional) If using a secure cluster, see [Security](AG_Install.html#Security) below on configuring Kerberos with Oozie HA.
7. Start the ZooKeeper servers.
8. Start the Oozie servers.
-Note: If one of the Oozie servers becomes unavailable, querying Oozie for the logs from a job in the Web UI, REST API, or client may
-be missing information until that server comes back up.
+ Note: If one of the Oozie servers becomes unavailable, querying Oozie for the logs from a job in the Web UI, REST API, or client may
+ be missing information until that server comes back up.
----++++ Security
+#### Security
Oozie HA works with the existing Oozie security framework and settings. For HA features (log streaming, share lib, etc) to work
-properly in a secure setup, following property can be set on each server. If =oozie.server.authentication.type= is not set, then
-server-server authentication will fall back on =oozie.authentication.type=.
+properly in a secure setup, following property can be set on each server. If `oozie.server.authentication.type` is not set, then
+server-server authentication will fall back on `oozie.authentication.type`.
-<verbatim>
+
+```
<property>
<name>oozie.server.authentication.type</name>
<value>kerberos</value>
</property>
-</verbatim>
+```
Below are some additional steps and information specific to Oozie HA:
1. (Optional) To prevent unauthorized users or programs from interacting with or reading the znodes used by Oozie in ZooKeeper,
-you can tell Oozie to use Kerberos-backed ACLs. To enforce this for all of the Oozie-related znodes, simply add the following
-property to oozie-site.xml in all Oozie servers and set it to =true=. The default is =false=.
+ you can tell Oozie to use Kerberos-backed ACLs. To enforce this for all of the Oozie-related znodes, simply add the following
+ property to oozie-site.xml in all Oozie servers and set it to `true`. The default is `false`.
-<verbatim>
-<property>
- <name>oozie.zookeeper.secure</name>
- <value>true</value>
-</property>
-</verbatim>
-Note: The Kerberos principals of each of the Oozie servers should have the same primary name (i.e. in =primary/instance@REALM=, each
-server should have the same value for =primary=).
+ ```
+ <property>
+ <name>oozie.zookeeper.secure</name>
+ <value>true</value>
+ </property>
+ ```
-*Important:* Once this property is set to =true=, it will set the ACLs on all existing Oozie-related znodes to only allow Kerberos
-authenticated users with a principal that has the same primary as described above (also for any subsequently created new znodes).
-This means that if you ever want to turn this feature off, you will have to manually connect to ZooKeeper using a Kerberos principal
-with the same primary and either delete all znodes under and including the namespace (i.e. if =oozie.zookeeper.namespace= = =oozie=
-then that would be =/oozie=); alternatively, instead of deleting them all, you can manually set all of their ACLs to =world:anyone=.
-In either case, make sure that no Oozie servers are running while this is being done.
+ Note: The Kerberos principals of each of the Oozie servers should have the same primary name (i.e. in `primary/instance@REALM`, each
+ server should have the same value for `primary`).
-Also, in your zoo.cfg for ZooKeeper, make sure to set the following properties:
-<verbatim>
-authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider
-kerberos.removeHostFromPrincipal=true
-kerberos.removeRealmFromPrincipal=true
-</verbatim>
+ **Important:** Once this property is set to `true`, it will set the ACLs on all existing Oozie-related znodes to only allow Kerberos
+ authenticated users with a principal that has the same primary as described above (also for any subsequently created new znodes).
+ This means that if you ever want to turn this feature off, you will have to manually connect to ZooKeeper using a Kerberos principal
+ with the same primary and either delete all znodes under and including the namespace (i.e. if `oozie.zookeeper.namespace` ` `oozie=
+ then that would be `/oozie`); alternatively, instead of deleting them all, you can manually set all of their ACLs to `world:anyone`.
+ In either case, make sure that no Oozie servers are running while this is being done.
+
+ Also, in your zoo.cfg for ZooKeeper, make sure to set the following properties:
+
+ ```
+ authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider
+ kerberos.removeHostFromPrincipal=true
+ kerberos.removeRealmFromPrincipal=true
+ ```
2. Until Hadoop 2.5.0 and later, there is a known limitation where each Oozie server can only use one HTTP principal. However,
-for Oozie HA, we need to use two HTTP principals: =HTTP/oozie-server-host@realm= and =HTTP/load-balancer-host@realm=. This
-allows access to each Oozie server directly and through the load balancer. While users should always go through the load balancer,
-certain features (e.g. log streaming) require the Oozie servers to talk to each other directly; it can also be helpful for an
-administrator to talk directly to an Oozie server. So, if using a Hadoop version prior to 2.5.0, you will have to choose which
-HTTP principal to use as you cannot use both; it is recommended to choose =HTTP/load-balancer-host@realm= so users can connect
-through the load balancer. This will prevent Oozie servers from talking to each other directly, which will effectively disable
-log streaming.
+ for Oozie HA, we need to use two HTTP principals: `HTTP/oozie-server-host@realm` and `HTTP/load-balancer-host@realm`. This
+ allows access to each Oozie server directly and through the load balancer. While users should always go through the load balancer,
+ certain features (e.g. log streaming) require the Oozie servers to talk to each other directly; it can also be helpful for an
+ administrator to talk directly to an Oozie server. So, if using a Hadoop version prior to 2.5.0, you will have to choose which
+ HTTP principal to use as you cannot use both; it is recommended to choose `HTTP/load-balancer-host@realm` so users can connect
+ through the load balancer. This will prevent Oozie servers from talking to each other directly, which will effectively disable
+ log streaming.
-For Hadoop 2.5.0 and later:
+ For Hadoop 2.5.0 and later:
-2a. When creating the keytab used by Oozie, make sure to include Oozie's principal and the two HTTP principals mentioned above.
+ 2a. When creating the keytab used by Oozie, make sure to include Oozie's principal and the two HTTP principals mentioned above.
-2b. Set =oozie.authentication.kerberos.principal= to * (that is, an asterisks) so it will use both HTTP principals.
-For earlier versions of Hadoop:
+ 2b. Set `oozie.authentication.kerberos.principal` to * (that is, an asterisks) so it will use both HTTP principals.
-2a. When creating the keytab used by Oozie, make sure to include Oozie's principal and the load balancer HTTP principal
+ For earlier versions of Hadoop:
-2b. Set =oozie.authentication.kerberos.principal= to =HTTP/load-balancer-host@realm=.
+ 2a. When creating the keytab used by Oozie, make sure to include Oozie's principal and the load balancer HTTP principal
+
+
+ 2b. Set `oozie.authentication.kerberos.principal` to `HTTP/load-balancer-host@realm`.
3. With Hadoop 2.6.0 and later, a rolling random secret that is synchronized across all Oozie servers will be used for signing the
-Oozie auth tokens. This is done automatically when HA is enabled; no additional configuration is needed.
+ Oozie auth tokens. This is done automatically when HA is enabled; no additional configuration is needed.
-For earlier versions of Hadoop, each server will have a different random secret. This will still work but will likely result in
-additional calls to the KDC to authenticate users to the Oozie server (because the auth tokens will not be accepted by other
-servers, which will cause a fallback to Kerberos).
+ For earlier versions of Hadoop, each server will have a different random secret. This will still work but will likely result in
+ additional calls to the KDC to authenticate users to the Oozie server (because the auth tokens will not be accepted by other
+ servers, which will cause a fallback to Kerberos).
4. If you'd like to use HTTPS (SSL) with Oozie HA, there's some additional considerations that need to be made.
-See the [[AG_Install#Setting_Up_Oozie_with_HTTPS_SSL][Setting Up Oozie with HTTPS (SSL)]] section for more information.
+ See the [Setting Up Oozie with HTTPS (SSL)](AG_Install.html#Setting_Up_Oozie_with_HTTPS_SSL) section for more information.
----++++ JobId sequence
+#### JobId sequence
Oozie in HA mode, uses ZK to generate job id sequence. Job Ids are of following format.
-<Id sequence>-<yyMMddHHmmss(server start time)>-<system_id>-<W/C/B>
+`<Id sequence>-<yyMMddHHmmss(server start time)>-<system_id>-<W/C/B>`
-Where, <systemId> is configured as =oozie.system.id= (default is "oozie-" + "user.name")
+Where, `<systemId>` is configured as `oozie.system.id` (default is "oozie-" + "user.name")
W/C/B is suffix to job id indicating that generated job is a type of workflow or coordinator or bundle.
Maximum allowed character for job id sequence is 40. "Id sequence" is stored in ZK and reset to 0 once maximum job id sequence is
-reached. Maximum job id sequence is configured as =oozie.service.ZKUUIDService.jobid.sequence.max=, default value is 99999999990.
+reached. Maximum job id sequence is configured as `oozie.service.ZKUUIDService.jobid.sequence.max`, default value is 99999999990.
-<verbatim>
+
+```
<property>
<name>oozie.service.ZKUUIDService.jobid.sequence.max</name>
<value>99999999990</value>
</property>
-</verbatim>
+```
----++ Starting and Stopping Oozie
+## Starting and Stopping Oozie
Use the standard commands to start and stop Oozie.
----++ Oozie Command Line Installation
+## Oozie Command Line Installation
-Copy and expand the =oozie-client= TAR.GZ file bundled with the distribution. Add the =bin/= directory to the =PATH=.
+Copy and expand the `oozie-client` TAR.GZ file bundled with the distribution. Add the `bin/` directory to the `PATH`.
-Refer to the [[DG_CommandLineTool][Command Line Interface Utilities]] document for a full reference of the =oozie=
+Refer to the [Command Line Interface Utilities](DG_CommandLineTool.html) document for a full reference of the `oozie`
command line tool.
----++ Oozie Share Lib
+## Oozie Share Lib
The Oozie sharelib TAR.GZ file bundled with the distribution contains the necessary files to run Oozie map-reduce streaming, pig,
hive, sqooop, and distcp actions. There is also a sharelib for HCatalog. The sharelib is required for these actions to work; any
@@ -1246,21 +1301,23 @@
respective sharelib jars or the "oozie" sharelib instead. When false, the sharelib is required for ALL actions; when true, the
sharelib is only required for actions that need additional jars (the original list from above).
-<verbatim>
+
+```
<property>
<name>oozie.action.ship.launcher.jar</name>
<value>true</value>
</property>
-</verbatim>
+```
-Using sharelib CLI, sharelib files are copied to new lib_<timestamped> directory. At start, server picks the sharelib from latest
+Using sharelib CLI, sharelib files are copied to new lib_`<timestamped>` directory. At start, server picks the sharelib from latest
time-stamp directory. While starting, server also purges sharelib directory which are older than sharelib retention days
(defined as oozie.service.ShareLibService.temp.sharelib.retention.days and 7 days is default).
Sharelib mapping file can be also configured. Configured file is a key value mapping, where key will be the sharelib name for the
action and value is a comma separated list of DFS or local filesystem directories or jar files. Local filesystem refers to the local
filesystem of the node where the Oozie launcher is running. This can be configured in oozie-site.xml as :
- <verbatim>
+
+```
<!-- OOZIE -->
<property>
<name>oozie.service.ShareLibService.mapping.file</name>
@@ -1277,11 +1334,12 @@
oozie.hive=file:///usr/local/oozie/share/lib/hive/
</description>
</property>
- </verbatim>
+```
Example mapping file with local filesystem resources:
- <verbatim>
+
+```
<property>
<name>oozie.service.ShareLibService.mapping.file</name>
<value>
@@ -1296,7 +1354,7 @@
oozie.sqoop=file:///usr/localoozie/share/lib/sqoop
</value>
</property>
- </verbatim>
+```
If you are using local filesystem resources in the mapping file, make sure corresponding jars are already deployed to
all the nodes where Oozie launcher jobs will be executed, and the files are readable by the launchers. To do this, you
@@ -1307,79 +1365,83 @@
files. These files can be added either to workflow's lib/ directory, to the sharelib or in sharelib mapping file.
----++ Oozie Coordinators/Bundles Processing Timezone
+## Oozie Coordinators/Bundles Processing Timezone
-By default Oozie runs coordinator and bundle jobs using =UTC= timezone for datetime values specified in the application
+By default Oozie runs coordinator and bundle jobs using `UTC` timezone for datetime values specified in the application
XML and in the job parameter properties. This includes coordinator applications start and end times of jobs, coordinator
datasets initial-instance, and bundle applications kickoff times. In addition, coordinator dataset instance URI templates
will be resolved using datetime values of the Oozie processing timezone.
It is possible to set the Oozie processing timezone to a timezone that is an offset of UTC, alternate timezones must
-expressed in using a GMT offset ( =GMT+/-####= ). For example: =GMT+0530= (India timezone).
+expressed in using a GMT offset ( `GMT+/-#### ` ). For example: `GMT+0530` (India timezone).
-To change the default =UTC= timezone, use the =oozie.processing.timezone= property in the =oozie-site.xml=. For example:
+To change the default `UTC` timezone, use the `oozie.processing.timezone` property in the `oozie-site.xml`. For example:
-<verbatim>
+
+```
<configuration>
<property>
<name>oozie.processing.timezone</name>
<value>GMT+0530</value>
</property>
</configuration>
-</verbatim>
+```
-*IMPORTANT:* If using a processing timezone other than =UTC=, all datetime values in coordinator and bundle jobs must
-be expressed in the corresponding timezone, for example =2012-08-08T12:42+0530=.
+**IMPORTANT:** If using a processing timezone other than `UTC`, all datetime values in coordinator and bundle jobs must
+be expressed in the corresponding timezone, for example `2012-08-08T12:42+0530`.
-*NOTE:* It is strongly encouraged to use =UTC=, the default Oozie processing timezone.
+**NOTE:** It is strongly encouraged to use `UTC`, the default Oozie processing timezone.
For more details on using an alternate Oozie processing timezone, please refer to the
-[[CoordinatorFunctionalSpec#datetime][Coordinator Functional Specification, section '4. Datetime']]
+[Coordinator Functional Specification, section '4. Datetime'](CoordinatorFunctionalSpec.html#datetime)
-#UberJar
----++ MapReduce Workflow Uber Jars
+<a name="UberJar"></a>
+## MapReduce Workflow Uber Jars
For Map-Reduce jobs (not including streaming or pipes), additional jar files can also be included via an uber jar. An uber jar is a
jar file that contains additional jar files within a "lib" folder (see
-[[WorkflowFunctionalSpec#AppDeployment][Workflow Functional Specification]] for more information). Submitting a workflow with an uber jar
+[Workflow Functional Specification](WorkflowFunctionalSpec.html#AppDeployment) for more information). Submitting a workflow with an uber jar
requires at least Hadoop 2.2.0 or 1.2.0. As such, using uber jars in a workflow is disabled by default. To enable this feature, use
-the =oozie.action.mapreduce.uber.jar.enable= property in the =oozie-site.xml= (and make sure to use a supported version of Hadoop).
+the `oozie.action.mapreduce.uber.jar.enable` property in the `oozie-site.xml` (and make sure to use a supported version of Hadoop).
-<verbatim>
+
+```
<configuration>
<property>
<name>oozie.action.mapreduce.uber.jar.enable</name>
<value>true</value>
</property>
</configuration>
-</verbatim>
+```
----++ Advanced/Custom Environment Settings
+## Advanced/Custom Environment Settings
Oozie can be configured to use Unix standard filesystem hierarchy for its different files
(configuration, logs, data and temporary files).
-These settings must be done in the =bin/oozie-env.sh= script.
+These settings must be done in the `bin/oozie-env.sh` script.
-This script is sourced before the configuration =oozie-env.sh= and supports additional
+This script is sourced before the configuration `oozie-env.sh` and supports additional
environment variables (shown with their default values):
-<verbatim>
+
+```
export OOZIE_CONFIG=${OOZIE_HOME}/conf
export OOZIE_DATA={OOZIE_HOME}/data
export OOZIE_LOG={OOZIE_HOME}/logs
export JETTY_OUT=${OOZIE_LOGS}/jetty.out
export JETTY_PID=/tmp/oozie.pid
-</verbatim>
+```
Sample values to make Oozie follow Unix standard filesystem hierarchy:
-<verbatim>
+
+```
export OOZIE_CONFIG=/etc/oozie
export OOZIE_DATA=/var/lib/oozie
export OOZIE_LOG=/var/log/oozie
export JETTY_PID=/tmp/oozie.pid
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/AG_Monitoring.twiki b/docs/src/site/twiki/AG_Monitoring.twiki
index 425f554..4877dfc 100644
--- a/docs/src/site/twiki/AG_Monitoring.twiki
+++ b/docs/src/site/twiki/AG_Monitoring.twiki
@@ -1,33 +1,34 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Oozie Monitoring
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Oozie Monitoring
----++ Oozie Instrumentation
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Oozie Instrumentation
Oozie code is instrumented in several places to collect runtime metrics. The instrumentation data can be used to
determine the health of the system, performance of the system, and to tune the system.
This comes in two flavors:
+
* metrics (by default enabled since 5.0.0)
* instrumentation (deprecated and by default disabled since 5.0.0)
-The instrumentation is accessible via the Admin web-services API (see the [[WebServicesAPI#Oozie_Metrics][metrics]] and
-[[WebServicesAPI#Oozie_Instrumentation][instrumentation]] Web Services API documentations for more details) and is also written on
+The instrumentation is accessible via the Admin web-services API (see the [metrics](WebServicesAPI.html#Oozie_Metrics) and
+[instrumentation](WebServicesAPI.html#Oozie_Instrumentation) Web Services API documentations for more details) and is also written on
regular intervals to an instrumentation log.
Instrumentation data includes variables, samplers, timers and counters.
----+++ Variables
+### Variables
* oozie
* version: Oozie build version.
* configuration
- * config.dir: directory from where the configuration files are loaded. If null, all configuration files are loaded from the classpath. [[AG_Install#Oozie_Configuration][Configuration files are described here]].
+ * config.dir: directory from where the configuration files are loaded. If null, all configuration files are loaded from the classpath. [Configuration files are described here](AG_Install.html#Oozie_Configuration).
* config.file: the Oozie custom configuration for the instance.
* jvm
@@ -43,7 +44,7 @@
* from.classpath: whether the config file has been read from the classpath or from the config directory.
* reload.interval: interval at which the config file will be reloaded. 0 if the config file will never be reloaded, when loaded from the classpath is never reloaded.
----+++ Samplers - Poll data at a fixed interval (default 1 sec) and report an average utilization over a longer period of time (default 60 seconds).
+### Samplers - Poll data at a fixed interval (default 1 sec) and report an average utilization over a longer period of time (default 60 seconds).
Poll for data over fixed interval and generate an average over the time interval. Unless specified, all samplers in
Oozie work on a 1 minute interval.
@@ -64,14 +65,16 @@
* requests
* version
----+++ Counters - Maintain statistics about the number of times an event has occurred, for the running Oozie instance. The values are reset if the Oozie instance is restarted.
+### Counters - Maintain statistics about the number of times an event has occurred, for the running Oozie instance. The values are reset if the Oozie instance is restarted.
* action.executors - Counters related to actions.
* [action_type]#action.[operation_performed] (start, end, check, kill)
* [action_type]#ex.[exception_type] (transient, non-transient, error, failed)
- * e.g. <verbatim>
- ssh#action.end: 306
- ssh#action.start: 316 </verbatim>
+ * e.g.
+```
+ssh#action.end: 306
+ssh#action.start: 316
+```
* callablequeue - count of events in various execution queues.
* delayed.queued: Number of commands queued with a delay.
@@ -113,7 +116,7 @@
* version
* version-GET
----+++ Timers - Maintain information about the time spent in various operations.
+### Timers - Maintain information about the time spent in various operations.
* action.executors - Counters related to actions.
* [action_type]#action.[operation_performed] (start, end, check, kill)
@@ -156,39 +159,41 @@
* version
* version-GET
----++ Oozie JVM Thread Dump
- The =admin/jvminfo.jsp= servlet can be used to get some basic jvm stats and thread dump.
-For eg: http://localhost:11000/oozie/admin/jvminfo.jsp?cpuwatch=1000&threadsort=cpu. It takes the following optional
+## Oozie JVM Thread Dump
+The `admin/jvminfo.jsp` servlet can be used to get some basic jvm stats and thread dump.
+For eg: `http://localhost:11000/oozie/admin/jvminfo.jsp?cpuwatch=1000&threadsort=cpu`. It takes the following optional
query parameters:
+
* threadsort - The order in which the threads are sorted for display. Valid values are name, cpu, state. Default is state.
* cpuwatch - Time interval in milliseconds to monitor cpu usage of threads. Default value is 0.
----++ Monitoring Database Schema Integrity
+## Monitoring Database Schema Integrity
Oozie stores all of its state in a database. Hence, ensuring that the database schema is correct is very important to ensuring that
-Oozie is healthy and behaves correctly. To help with this, Oozie includes a =SchemaCheckerService= which periodically runs and
+Oozie is healthy and behaves correctly. To help with this, Oozie includes a `SchemaCheckerService` which periodically runs and
performs a series of checks on the database schema. More specifically, it checks the following:
+
* Existence of the required tables
* Existence of the required columns in each table
* Each column has the correct type and default value
* Existence of the required primary keys and indexes
-After each run, the =SchemaCheckerService= writes the result of the checks to the Oozie log and to the "schema-checker.status"
+After each run, the `SchemaCheckerService` writes the result of the checks to the Oozie log and to the "schema-checker.status"
instrumentation variable. If there's a problem, it will be logged at the ERROR level, while correct checks are logged at the DEBUG
level.
-By default, the =SchemaCheckerService= runs every 7 days. This can be configured
-by =oozie.service.SchemaCheckerService.check.interval=
+By default, the `SchemaCheckerService` runs every 7 days. This can be configured
+by `oozie.service.SchemaCheckerService.check.interval`
-By default, the =SchemaCheckerService= will consider "extra" tables, columns, and indexes to be incorrect. Advanced users who have
+By default, the `SchemaCheckerService` will consider "extra" tables, columns, and indexes to be incorrect. Advanced users who have
added additional tables, columns, and indexes can tell Oozie to ignore these by
-setting =oozie.service.SchemaCheckerService.ignore.extras= to =false=.
+setting `oozie.service.SchemaCheckerService.ignore.extras` to `false`.
-The =SchemaCheckerService= currently only supports MySQL, PostgreSQL, and Oracle databases. SQL Server and Derby are currently not
+The `SchemaCheckerService` currently only supports MySQL, PostgreSQL, and Oracle databases. SQL Server and Derby are currently not
supported.
When Oozie HA is enabled, only one of the Oozie servers will perform the checks.
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/AG_OozieLogging.twiki b/docs/src/site/twiki/AG_OozieLogging.twiki
deleted file mode 100644
index ecdcfd3..0000000
--- a/docs/src/site/twiki/AG_OozieLogging.twiki
+++ /dev/null
@@ -1,87 +0,0 @@
-<noautolink>
-
-[[index][::Go back to Oozie Documentation Index::]]
-
----+!! Oozie Logging
-
-%TOC%
-
----++ Default Oozie Logging
-
-Oozie's logging properties can be configured in its log4j properties file (default is =oozie-log4j.properties=). Most log messages
-are configured by default to be written to the =oozie= appender.
-
-The default configuration for the =oozie= appender is shown below.
-
----+++ Default Configuration
-
-<verbatim>
-log4j.appender.oozie=org.apache.log4j.rolling.RollingFileAppender
-log4j.appender.oozie.RollingPolicy=org.apache.oozie.util.OozieRollingPolicy
-log4j.appender.oozie.File=${oozie.log.dir}/oozie.log
-log4j.appender.oozie.Append=true
-log4j.appender.oozie.layout=org.apache.log4j.PatternLayout
-log4j.appender.oozie.layout.ConversionPattern=%d{ISO8601} %5p %c{1}:%L - %m%n
-log4j.appender.oozie.RollingPolicy.FileNamePattern=${log4j.appender.oozie.File}-%d{yyyy-MM-dd-HH}
-log4j.appender.oozie.RollingPolicy.MaxHistory=720
-</verbatim>
-
-In this configuration, the active log file will be named =oozie.log= and all old log files will be named =oozie.log-yyyy-MM-dd-HH=
-(where =yyyy-MM-dd-HH= is the time that that log file was created; e.g. 2012-07-21-05). All log files are in the same directory
-(whatever =oozie.log.dir= is assigned to). A maximum of 720 older log files will be retained. The active log file is rolled every
-hour, so 720 old logs means that they are kept for 30 days before being deleted.
-
-To keep all old logs instead of deleting them, =log4j.appender.oozie.RollingPolicy.MaxHistory= can be set to =-1=.
-Additionally, =log4j.appender.oozie.RollingPolicy= can be set to =org.apache.log4j.rolling.TimeBasedRollingPolicy=, which has the
-same exact behavior as =org.apache.oozie.util.OozieRollingPolicy= except that it does not delete old logs.
-
----+++ Restrictions
-
-In order for Oozie logging to work 100% correctly, the following restrictions must be observed (described below and in
-the =oozie-log4j.properties= file):
-
-* The appender that Oozie uses must be named "oozie" (i.e. =log4j.appender.oozie=)
-
-* =log4j.appender.oozie.RollingPolicy.FileNamePattern= must end with "-%d{yyyy-MM-dd-HH}.gz" or "-%d{yyyy-MM-dd-HH}".
-If it ends with ".gz" the old logs will be compressed when rolled
-
-* =log4j.appender.oozie.RollingPolicy.FileNamePattern= must start with the value of =log4j.appender.oozie.File=
-
----++ Previous Default Oozie Logging
-
-Oozie previously used the logging configuration shown below as the default for the =oozie= appender. The other appender that Oozie
-writes to still use a configuration similar to this.
-
----+++ Previous Default Configuration
-
-<verbatim>
-log4j.appender.oozie=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.oozie.File=${oozie.log.dir}/oozie.log
-log4j.appender.oozie.Append=true
-log4j.appender.oozie.layout=org.apache.log4j.PatternLayout
-log4j.appender.oozie.layout.ConversionPattern=%d{ISO8601} %5p %c{1}:%L - %m%n
-log4j.appender.oozie.DatePattern='.'yyyy-MM-dd-HH
-</verbatim>
-
-In this configuration, the active log file will be named =oozie.log= and all old log files will be named =oozie.log.yyyy-MM-dd-HH=
-(where =yyyy-MM-dd-HH= is the time that the log file was created; e.g. 2012-07-21-05). All log files are in the same directory
-(whatever =oozie.log.dir= is assigned to). All older log files are retained. The active log file is rolled every hour.
-
----+++ Restrictions
-
-In order for Oozie logging to work 100% correctly, the following restrictions must be observed (described below and in the
-=oozie-log4j.properties= file):
-
-* The appender that Oozie uses must be named "oozie" (i.e. =log4j.appender.oozie=)
-
-* =log4j.appender.oozie.DatePattern= must end with either "dd" or "HH". If it ends with "HH", the log will be rolled every hour;
-if it ends with "dd", the log will be rolled every day.
-
----++ Other Oozie Logging
-
-While Oozie can technically use any valid log4j Appender or configurations that violate the above restrictions, certain
-features related to logs may be disabled and/or not work correctly, and is thus not advised.
-
-[[index][::Go back to Oozie Documentation Index::]]
-
-</noautolink>
diff --git a/docs/src/site/twiki/AG_OozieUpgrade.twiki b/docs/src/site/twiki/AG_OozieUpgrade.twiki
index 3024064..ec24011 100644
--- a/docs/src/site/twiki/AG_OozieUpgrade.twiki
+++ b/docs/src/site/twiki/AG_OozieUpgrade.twiki
@@ -1,12 +1,12 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Oozie Upgrade
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Oozie Upgrade
----+ Preparation
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Preparation
Make sure there are not Workflows in RUNNING or SUSPENDED status, otherwise the database upgrade will fail.
@@ -14,29 +14,30 @@
Copy the oozie-site.xml from your current setup.
----+ Oozie Server Upgrade
+## Oozie Server Upgrade
Expand the new Oozie tarball in a new location.
-Edit the new =oozie-site.xml= setting all custom properties values from the old =oozie-site.xml=
+Edit the new `oozie-site.xml` setting all custom properties values from the old `oozie-site.xml`
IMPORTANT: From Oozie 2.x to Oozie 3.x the names of the database configuration properties have
-changed. Their prefix has changed from =oozie.service.StoreService.*= to =oozie.service.JPAService.*=.
+changed. Their prefix has changed from `oozie.service.StoreService.*` to `oozie.service.JPAService.*`.
Make sure you are using the new prefix.
-After upgrading the Oozie server, the =oozie-setup.sh= MUST be rerun before starting the
+After upgrading the Oozie server, the `oozie-setup.sh` MUST be rerun before starting the
upgraded Oozie server.
Oozie database migration is required when there Oozie database schema changes, like
upgrading from Oozie 2.x to Oozie 3.x.
Configure the oozie-site.xml with the correct database configuration properties as
-explained in the 'Database Configuration' section in [[AG_Install][Oozie Install]].
+explained in the 'Database Configuration' section in [Oozie Install](AG_Install.html).
-Once =oozie-site.xml= has been configured with the database configuration execute the =ooziedb.sh=
+Once `oozie-site.xml` has been configured with the database configuration execute the `ooziedb.sh`
command line tool to upgrade the database:
-<verbatim>
+
+```
$ bin/ooziedb.sh upgrade -run
Validate DB Connection.
@@ -64,22 +65,22 @@
The SQL commands have been written to: /tmp/ooziedb-5737263881793872034.sql
$
-</verbatim>
+```
The new version of the Oozie server is ready to be started.
-NOTE: If using MySQL or Oracle, copy the corresponding JDBC driver JAR file to the =libext/= directory before running
-the =ooziedb.sh= command line tool.
+NOTE: If using MySQL or Oracle, copy the corresponding JDBC driver JAR file to the `libext/` directory before running
+the `ooziedb.sh` command line tool.
-NOTE: If instead using the '-run' option, the '-sqlfile <FILE>' option is used, then all the
+NOTE: If instead using the '-run' option, the `-sqlfile <FILE>` option is used, then all the
database changes will be written to the specified file and the database won't be modified.
----+ Oozie Client Upgrade
+## Oozie Client Upgrade
While older Oozie clients work with newer Oozie server, to have access to all the
functionality of the Oozie server the same version of Oozie client should be installed
and used by users.
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/BundleFunctionalSpec.twiki b/docs/src/site/twiki/BundleFunctionalSpec.twiki
index 9749df5..5301429 100644
--- a/docs/src/site/twiki/BundleFunctionalSpec.twiki
+++ b/docs/src/site/twiki/BundleFunctionalSpec.twiki
@@ -1,110 +1,111 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie Bundle Specification
+# Oozie Bundle Specification
The goal of this document is to define a new oozie abstraction called bundle system specialized in submitting and maintaining a set of coordinator applications.
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
----++ Changelog
+## Changelog
----++ 1. Bundle Overview
+## 1. Bundle Overview
Bundle is a higher-level oozie abstraction that will batch a set of coordinator applications. The user will be able to start/stop/suspend/resume/rerun in the bundle level resulting a better and easy operational control.
-More specifically, the oozie *Bundle* system allows the user to define and execute a bunch of coordinator applications often called a data pipeline. There is no explicit dependency among the coordinator applications in a bundle. However, a user could use the data dependency of coordinator applications to create an implicit data application pipeline.
+More specifically, the oozie **Bundle** system allows the user to define and execute a bunch of coordinator applications often called a data pipeline. There is no explicit dependency among the coordinator applications in a bundle. However, a user could use the data dependency of coordinator applications to create an implicit data application pipeline.
----++ 2. Definitions
+## 2. Definitions
-*Kick-off-time:* The time when a bundle should start and submit coordinator applications.
+**Kick-off-time:** The time when a bundle should start and submit coordinator applications.
-*Bundle Application:* A bundle application defines a set of coordinator applications and when to start those. Normally, bundle applications are parameterized. A bundle application is written in XML.
+**Bundle Application:** A bundle application defines a set of coordinator applications and when to start those. Normally, bundle applications are parameterized. A bundle application is written in XML.
-*Bundle Job:* A bundle job is an executable instance of a bundle application. A job submission is done by submitting a job configuration that resolves all parameters in the application definition.
+**Bundle Job:** A bundle job is an executable instance of a bundle application. A job submission is done by submitting a job configuration that resolves all parameters in the application definition.
-*Bundle Definition Language:* The language used to describe bundle applications.
+**Bundle Definition Language:** The language used to describe bundle applications.
----++ 3. Expression Language for Parameterization
+## 3. Expression Language for Parameterization
Bundle application definitions can be parameterized with variables.
At job submission time all the parameters are resolved into concrete values.
-The parameterization of bundle definitions is done using JSP Expression Language syntax from the [[http://jcp.org/aboutJava/communityprocess/final/jsr152/][JSP 2.0 Specification (JSP.2.3)]], allowing not only to support variables as parameters but also complex expressions.
+The parameterization of bundle definitions is done using JSP Expression Language syntax from the [JSP 2.0 Specification (JSP.2.3)](http://jcp.org/aboutJava/communityprocess/final/jsr152/index.html), allowing not only to support variables as parameters but also complex expressions.
EL expressions can be used in XML attribute values and XML text element values. They cannot be used in XML element and XML attribute names.
----++ 4. Bundle Job
+## 4. Bundle Job
----+++ 4.1. Bundle Job Status
+### 4.1. Bundle Job Status
-At any time, a bundle job is in one of the following status: *PREP, RUNNING, RUNNINGWITHERROR, SUSPENDED, PREPSUSPENDED, SUSPENDEDWITHERROR, PAUSED, PAUSEDWITHERROR, PREPPAUSED, SUCCEEDED, DONEWITHERROR, KILLED, FAILED*.
+At any time, a bundle job is in one of the following status: **PREP, RUNNING, RUNNINGWITHERROR, SUSPENDED, PREPSUSPENDED, SUSPENDEDWITHERROR, PAUSED, PAUSEDWITHERROR, PREPPAUSED, SUCCEEDED, DONEWITHERROR, KILLED, FAILED**.
----+++ 4.2. Transitions of Bundle Job Status
+### 4.2. Transitions of Bundle Job Status
Valid bundle job status transitions are:
- * *PREP --> PREPSUSPENDED | PREPPAUSED | RUNNING | KILLED*
- * *RUNNING --> RUNNINGWITHERROR | SUSPENDED | PAUSED | SUCCEEDED | KILLED*
- * *RUNNINGWITHERROR --> RUNNING | SUSPENDEDWITHERROR | PAUSEDWITHERROR | DONEWITHERROR | FAILED | KILLED*
- * *PREPSUSPENDED --> PREP | KILLED*
- * *SUSPENDED --> RUNNING | KILLED*
- * *SUSPENDEDWITHERROR --> RUNNINGWITHERROR | KILLED*
- * *PREPPAUSED --> PREP | KILLED*
- * *PAUSED --> SUSPENDED | RUNNING | KILLED*
- * *PAUSEDWITHERROR --> SUSPENDEDWITHERROR | RUNNINGWITHERROR | KILLED*
+ * **PREP --> PREPSUSPENDED | PREPPAUSED | RUNNING | KILLED**
+ * **RUNNING --> RUNNINGWITHERROR | SUSPENDED | PAUSED | SUCCEEDED | KILLED**
+ * **RUNNINGWITHERROR --> RUNNING | SUSPENDEDWITHERROR | PAUSEDWITHERROR | DONEWITHERROR | FAILED | KILLED**
+ * **PREPSUSPENDED --> PREP | KILLED**
+ * **SUSPENDED --> RUNNING | KILLED**
+ * **SUSPENDEDWITHERROR --> RUNNINGWITHERROR | KILLED**
+ * **PREPPAUSED --> PREP | KILLED**
+ * **PAUSED --> SUSPENDED | RUNNING | KILLED**
+ * **PAUSEDWITHERROR --> SUSPENDEDWITHERROR | RUNNINGWITHERROR | KILLED**
----+++ 4.3. Details of Status Transitions
-When a bundle job is submitted, oozie parses the bundle job XML. Oozie then creates a record for the bundle with status *PREP* and returns a unique ID.
+### 4.3. Details of Status Transitions
+When a bundle job is submitted, oozie parses the bundle job XML. Oozie then creates a record for the bundle with status **PREP** and returns a unique ID.
-When a user requests to suspend a bundle job that is in *PREP* state, oozie puts the job in status *PREPSUSPENDED*. Similarly, when pause time reaches for a bundle job with *PREP* status, oozie puts the job in status *PREPPAUSED*.
+When a user requests to suspend a bundle job that is in **PREP** state, oozie puts the job in status **PREPSUSPENDED**. Similarly, when pause time reaches for a bundle job with **PREP** status, oozie puts the job in status **PREPPAUSED**.
-Conversely, when a user requests to resume a *PREPSUSPENDED* bundle job, oozie puts the job in status *PREP*. And when pause time is reset for a bundle job that is in *PREPPAUSED* state, oozie puts the job in status *PREP*.
+Conversely, when a user requests to resume a **PREPSUSPENDED** bundle job, oozie puts the job in status **PREP**. And when pause time is reset for a bundle job that is in **PREPPAUSED** state, oozie puts the job in status **PREP**.
There are two ways a bundle job could be started.
- * If =kick-off-time= (defined in the bundle xml) reaches. The default value is null which means starts coordinators NOW.
+* If `kick-off-time` (defined in the bundle xml) reaches. The default value is null which means starts coordinators NOW.
- * If user sends a start request to START the bundle.
+* If user sends a start request to START the bundle.
-When a bundle job starts, oozie puts the job in status *RUNNING* and it submits all the coordinator jobs. If any coordinator job goes to *FAILED/KILLED/DONEWITHERROR* state, the bundle job is put in *RUNNINGWITHERROR*
+When a bundle job starts, oozie puts the job in status **RUNNING** and it submits all the coordinator jobs. If any coordinator job goes to **FAILED/KILLED/DONEWITHERROR** state, the bundle job is put in **RUNNINGWITHERROR**
-When a user requests to kill a bundle job, oozie puts the job in status *KILLED* and it sends kill to all submitted coordinator jobs.
+When a user requests to kill a bundle job, oozie puts the job in status **KILLED** and it sends kill to all submitted coordinator jobs.
-When a user requests to suspend a bundle job that is in *RUNNING* status, oozie puts the job in status *SUSPENDED* and it suspends all submitted coordinator jobs. Similarly, when a user requests to suspend a bundle job that is in *RUNNINGWITHERROR* status, oozie puts the job in status *SUSPENDEDWITHERROR* and it suspends all submitted coordinator jobs.
+When a user requests to suspend a bundle job that is in **RUNNING** status, oozie puts the job in status **SUSPENDED** and it suspends all submitted coordinator jobs. Similarly, when a user requests to suspend a bundle job that is in **RUNNINGWITHERROR** status, oozie puts the job in status **SUSPENDEDWITHERROR** and it suspends all submitted coordinator jobs.
-When pause time reaches for a bundle job that is in *RUNNING* status, oozie puts the job in status *PAUSED*. When pause time reaches for a bundle job that is in *RUNNINGWITHERROR* status, oozie puts the job in status *PAUSEDWITHERROR*.
+When pause time reaches for a bundle job that is in **RUNNING** status, oozie puts the job in status **PAUSED**. When pause time reaches for a bundle job that is in **RUNNINGWITHERROR** status, oozie puts the job in status **PAUSEDWITHERROR**.
-Conversely, when a user requests to resume a *SUSPENDED* bundle job, oozie puts the job in status *RUNNING*. Similarly, when a user requests to resume a *SUSPENDEDWITHERROR* bundle job, oozie puts the job in status *RUNNINGWITHERROR*. And when pause time is reset for a bundle job and job status is *PAUSED*, oozie puts the job in status *RUNNING*. Similarly, when the pause time is reset for a bundle job and job status is *PAUSEDWITHERROR*, oozie puts the job in status *RUNNINGWITHERROR*
+Conversely, when a user requests to resume a **SUSPENDED** bundle job, oozie puts the job in status **RUNNING**. Similarly, when a user requests to resume a **SUSPENDEDWITHERROR** bundle job, oozie puts the job in status **RUNNINGWITHERROR**. And when pause time is reset for a bundle job and job status is **PAUSED**, oozie puts the job in status **RUNNING**. Similarly, when the pause time is reset for a bundle job and job status is **PAUSEDWITHERROR**, oozie puts the job in status **RUNNINGWITHERROR**
-When all the coordinator jobs finish, oozie updates the bundle status accordingly. If all coordinators reaches to the _same_ terminal state, bundle job status also move to the same status. For example, if all coordinators are *SUCCEEDED*, oozie puts the bundle job into *SUCCEEDED* status. However, if all coordinator jobs don't finish with the same status, oozie puts the bundle job into *DONEWITHERROR*.
-
+When all the coordinator jobs finish, oozie updates the bundle status accordingly. If all coordinators reaches to the _same_ terminal state, bundle job status also move to the same status. For example, if all coordinators are **SUCCEEDED**, oozie puts the bundle job into **SUCCEEDED** status. However, if all coordinator jobs don't finish with the same status, oozie puts the bundle job into **DONEWITHERROR**.
----+++ 4.3. Bundle Application Definition
+
+### 4.3. Bundle Application Definition
A bundle definition is defined in XML by a name, controls and one or more coordinator application specifications:
- * *%BLUE% name: %ENDCOLOR%* The name for the bundle job.
- * *%BLUE% controls: %ENDCOLOR%* The control specification for the bundle.
- * *%BLUE% kick-off-time: %ENDCOLOR%* It defines when the bundle job should start and submit the coordinator applications. This field is optional and the default is *NOW* that means the job should start right-a-way.
- * *%BLUE% coordinator: %ENDCOLOR%* Coordinator application specification. There should be at least one coordinator application in any bundle.
- * *%BLUE% name: %ENDCOLOR%* Name of the coordinator application. It can be used for referring this application through bundle to control such as kill, suspend, rerun.
- * *%BLUE% enabled: %ENDCOLOR%* Enabled can be used to enable or disable a coordinator. It is optional. The default value for enabled is true.
- * *%BLUE% app-path: %ENDCOLOR%* Path of the coordinator application definition in hdfs. This is a mandatory element.
- * *%BLUE% configuration: %ENDCOLOR%* A hadoop like configuration to parameterize corresponding coordinator application. This is optional.
- * *%BLUE% Parameterization: %ENDCOLOR%* Configuration properties that are a valid Java identifier, [A-Za-z_][0-9A-Za-z_]*, are available as =${NAME}= variables within the bundle application definition. Configuration properties that are not a valid Java identifier, for example =job.tracker=, are available via the =${bundle:conf(String name)}= function. Valid Java identifier properties are available via this function as well.
+ * **<font color="#0000ff"> name: </font>** The name for the bundle job.
+ * **<font color="#0000ff"> controls: </font>** The control specification for the bundle.
+ * **<font color="#0000ff"> kick-off-time: </font>** It defines when the bundle job should start and submit the coordinator applications. This field is optional and the default is **NOW** that means the job should start right-a-way.
+ * **<font color="#0000ff"> coordinator: </font>** Coordinator application specification. There should be at least one coordinator application in any bundle.
+ * **<font color="#0000ff"> name: </font>** Name of the coordinator application. It can be used for referring this application through bundle to control such as kill, suspend, rerun.
+ * **<font color="#0000ff"> enabled: </font>** Enabled can be used to enable or disable a coordinator. It is optional. The default value for enabled is true.
+ * **<font color="#0000ff"> app-path: </font>** Path of the coordinator application definition in hdfs. This is a mandatory element.
+ * **<font color="#0000ff"> configuration: </font>** A hadoop like configuration to parameterize corresponding coordinator application. This is optional.
+ * **<font color="#0000ff"> Parameterization: </font>** Configuration properties that are a valid Java identifier, [A-Za-z_][0-9A-Za-z_]*, are available as `${NAME}` variables within the bundle application definition. Configuration properties that are not a valid Java identifier, for example `job.tracker`, are available via the `${bundle:conf(String name)}` function. Valid Java identifier properties are available via this function as well.
-*%PURPLE% Syntax: %ENDCOLOR%*
+**<font color="#800080">Syntax: </font>**
-<verbatim>
- <bundle-app name=[NAME] xmlns='uri:oozie:bundle:0.1'>
+
+```
+ <bundle-app name=[NAME] xmlns='uri:oozie:bundle:0.1'>
<controls>
<kick-off-time>[DATETIME]</kick-off-time>
</controls>
@@ -119,16 +120,17 @@
</configuration>
</coordinator>
...
-</bundle-app>
-</verbatim>
+</bundle-app>
+```
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-*A Bundle Job that maintains two coordinator applications:*
+**A Bundle Job that maintains two coordinator applications:**
-<verbatim>
-<bundle-app name='APPNAME' xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance' xmlns='uri:oozie:bundle:0.1'>
+
+```
+<bundle-app name='APPNAME' xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance' xmlns='uri:oozie:bundle:0.1'>
<controls>
<kick-off-time>${kickOffTime}</kick-off-time>
</controls>
@@ -159,18 +161,19 @@
</configuration>
</coordinator>
</bundle-app>
-</verbatim>
+```
----+++ 4.4. Bundle Formal Parameters
-As of schema 0.2, a list of formal parameters can be provided which will allow Oozie to verify, at submission time, that said
-properties are actually specified (i.e. before the job is executed and fails). Default values can also be provided.
+### 4.4. Bundle Formal Parameters
+As of schema 0.2, a list of formal parameters can be provided which will allow Oozie to verify, at submission time, that said
+properties are actually specified (i.e. before the job is executed and fails). Default values can also be provided.
-*Example:*
+**Example:**
The previous Bundle Job application definition with formal parameters:
-<verbatim>
-<bundle-app name='APPNAME' xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance' xmlns='uri:oozie:bundle:0.2'>
+
+```
+<bundle-app name='APPNAME' xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance' xmlns='uri:oozie:bundle:0.2'>
<parameters>
<property>
<name>appPath</name>
@@ -210,17 +213,17 @@
</configuration>
</coordinator>
</bundle-app>
-</verbatim>
+```
-In the above example, if =appPath= is not specified, Oozie will print an error message instead of submitting the job. If
-=appPath2= is not specified, Oozie will use the default value, =hdfs://foo:8020/user/joe/job/job.properties=.
+In the above example, if `appPath` is not specified, Oozie will print an error message instead of submitting the job. If
+`appPath2` is not specified, Oozie will use the default value, `hdfs://foo:8020/user/joe/job/job.properties`.
----++ 5. User Propagation
+## 5. User Propagation
-When submitting a bundle job, the configuration must contain a =user.name= property. If security is enabled, Oozie must ensure that the value of the =user.name= property in the configuration match the user credentials present in the protocol (web services) request.
+When submitting a bundle job, the configuration must contain a `user.name` property. If security is enabled, Oozie must ensure that the value of the `user.name` property in the configuration match the user credentials present in the protocol (web services) request.
-When submitting a bundle job, the configuration may contain the =oozie.job.acl= property (the =group.name= property
+When submitting a bundle job, the configuration may contain the `oozie.job.acl` property (the `group.name` property
has been deprecated). If authorization is enabled, this property is treated as as the ACL for the job, it can contain
user and group IDs separated by commas.
@@ -228,15 +231,15 @@
Oozie must propagate the specified user and ACL to the system executing its children jobs (coordinator jobs).
----++ 6. Bundle Application Deployment
+## 6. Bundle Application Deployment
A bundle application consist exclusively of bundle application definition and associated coordinator application specifications. They must be installed in an HDFS directory. To submit a job for a bundle application, the full HDFS path to bundle application definition must be specified.
----+++ 6.1. Organizing Bundle Applications
+### 6.1. Organizing Bundle Applications
TBD.
----++ 7. Bundle Job Submission
+## 7. Bundle Job Submission
When a bundle job is submitted to Oozie, the submitter must specified all the required job properties plus the HDFS path to the bundle application definition for the job.
@@ -245,9 +248,10 @@
All the bundle job properties, the HDFS path for the bundle application, the 'user.name' and 'oozie.job.acl' must be
submitted to the Oozie using an XML configuration file (Hadoop XML configuration file).
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
-<verbatim>
+
+```
<?xml version="1.0" encoding="UTF-8"?>
<configuration>
<property>
@@ -260,23 +264,24 @@
</property>
...
</configuration>
-</verbatim>
+```
----++ 8. Bundle Rerun
----+++ Rerunning a Bundle Job
+## 8. Bundle Rerun
+### 8.1 Rerunning a Bundle Job
Oozie provides a way of rerunning a bundle job. The user could request to rerun a subset of coordinators within a bundle by defining a list of coordinator's names. In addition, a user could define a list of dates or ranges of dates (in UTC format) to rerun for those time windows.
-There is a way of asking whether to cleanup all output directories before rerun. By default, oozie will remove all output directories. Moreover, there is an option by which a user could ask to re-calculate the dynamic input directories defined by latest function in coordinators.
+There is a way of asking whether to cleanup all output directories before rerun. By default, oozie will remove all output directories. Moreover, there is an option by which a user could ask to re-calculate the dynamic input directories defined by latest function in coordinators.
----+++ Rerun Arguments
+### 8.2 Rerun Arguments
-<verbatim>
+
+```
$oozie job -rerun <bundle_Job_id> [-coordinator <list of coordinator name separate by comma>
[-date 2009-01-01T01:00Z::2009-05-31T23:59Z, 2009-11-10T01:00Z, 2009-12-31T22:00Z]
[-nocleanup] [-refresh]
-</verbatim>
+```
- * The =rerun= option reruns a bundle job that is *not* in (=KILLED=, =FAILED=, =PREP=, =PREPPAUSED=, =PREPSUSPENDED=).
- * Rerun a bundle job that is in =PAUSED= state will reset the paused time.
+ * The `rerun` option reruns a bundle job that is *not* in (`KILLED`, `FAILED`, `PREP`, `PREPPAUSED`, `PREPSUSPENDED`).
+ * Rerun a bundle job that is in `PAUSED` state will reset the paused time.
* The option -coordinator determines the name of coordinator that will be rerun. By default all coordinators are rerun.
* Multiple ranges can be used in -date. See the above examples.
* The dates specified in -date must be UTC.
@@ -285,18 +290,19 @@
* If -refresh is set, all dependencies will be re-checked; otherwise only missed dependencies will be checked for the corresponding coordinators.
-After the command is executed the rerun bundle job will be in =RUNNING= status.
+After the command is executed the rerun bundle job will be in `RUNNING` status.
-Refer to the [[DG_CoordinatorRerun][Rerunning Coordinator Actions]] for details on rerun of coordinator job.
+Refer to the [Rerunning Coordinator Actions](DG_CoordinatorRerun.html) for details on rerun of coordinator job.
----++ Appendixes
+## Appendixes
----+++ Appendix A, Oozie Bundle XML-Schema
+### Appendix A, Oozie Bundle XML-Schema
----++++ Oozie Bundle Schema 0.1
+#### Oozie Bundle Schema 0.1
-<verbatim>
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:bundle="uri:oozie:bundle:0.1"
elementFormDefault="qualified" targetNamespace="uri:oozie:bundle:0.1">
@@ -340,11 +346,12 @@
</xs:sequence>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie Bundle Schema 0.2
+#### Oozie Bundle Schema 0.2
-<verbatim>
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:bundle="uri:oozie:bundle:0.2"
elementFormDefault="qualified" targetNamespace="uri:oozie:bundle:0.2">
@@ -403,9 +410,9 @@
</xs:sequence>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/CoordinatorFunctionalSpec.twiki b/docs/src/site/twiki/CoordinatorFunctionalSpec.twiki
index cd416d4..d31d1aa 100644
--- a/docs/src/site/twiki/CoordinatorFunctionalSpec.twiki
+++ b/docs/src/site/twiki/CoordinatorFunctionalSpec.twiki
@@ -1,74 +1,89 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie Coordinator Specification
+# Oozie Coordinator Specification
The goal of this document is to define a coordinator engine system specialized in submitting workflows based on time and data triggers.
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
----++ Changelog
----+++!! 03/JUL/2013
+## Changelog
- * #Appendix A, Added new coordinator schema 0.4, sla schema 0.2 and changed schemas ordering to newest first
----+++!! 07/JAN/2013
+**03/JUL/2013**
- * #6.8 Added section on new EL functions for datasets defined with HCatalog
----+++!! 26/JUL/2012
+ * Appendix A, Added new coordinator schema 0.4, sla schema 0.2 and changed schemas ordering to newest first
- * #Appendix A, updated XML schema 0.4 to include =parameters= element
- * #6.5 Updated to mention about =parameters= element as of schema 0.4
----+++!! 23/NOV/2011:
+**07/JAN/2013**
+
+ * 6.8 Added section on new EL functions for datasets defined with HCatalog
+
+**26/JUL/2012**
+
+ * Appendix A, updated XML schema 0.4 to include `parameters` element
+ * 6.5 Updated to mention about `parameters` element as of schema 0.4
+
+**23/NOV/2011:**
* Update execution order typo
----+++!! 05/MAY/2011:
+
+**05/MAY/2011:**
* Update coordinator schema 0.2
----+++!! 09/MAR/2011:
+
+**09/MAR/2011:**
* Update coordinator status
----+++!! 02/DEC/2010:
+
+**02/DEC/2010:**
* Update coordinator done-flag
----+++!! 26/AUG/2010:
+
+**26/AUG/2010:**
* Update coordinator rerun
----+++!! 09/JUN/2010:
+
+**09/JUN/2010:**
* Clean up unsupported functions
----+++!! 02/JUN/2010:
+
+**02/JUN/2010:**
* Update all EL functions in CoordFunctionalSpec with "coord:" prefix
----+++!! 02/OCT/2009:
+
+**02/OCT/2009:**
* Added Appendix A, Oozie Coordinator XML-Schema
* Change #5.3., Datasets definition supports 'include' element
----+++!! 29/SEP/2009:
- * Change #4.4.1, added =${coord:endOfDays(int n)}= EL function
- * Change #4.4.2, added =${coord:endOfMonths(int n)}= EL function
----+++!! 11/SEP/2009:
+**29/SEP/2009:**
- * Change #6.6.4. =${coord:tzOffset()}= EL function now returns offset in minutes. Added more explanation on behavior
+ * Change #4.4.1, added `${coord:endOfDays(int n)}` EL function
+ * Change #4.4.2, added `${coord:endOfMonths(int n)}` EL function
+
+**11/SEP/2009:**
+
+ * Change #6.6.4. `${coord:tzOffset()}` EL function now returns offset in minutes. Added more explanation on behavior
* Removed 'oozie' URL from action workflow invocation, per arch review feedback coord&wf run on the same instance
----+++!! 07/SEP/2009:
+
+**07/SEP/2009:**
* Full rewrite of sections #4 and #7
* Added sections #6.1.7, #6.6.2, #6.6.3 & #6.6.4
* Rewording through the spec definitions
* Updated all examples and syntax to latest changes
----+++!! 03/SEP/2009:
+
+**03/SEP/2009:**
* Change #2. Definitions. Some rewording in the definitions
- * Change #6.6.4. Replaced =${coord:next(int n)}= with =${coord:version(int n)}= EL Function
+ * Change #6.6.4. Replaced `${coord:next(int n)}` with `${coord:version(int n)}` EL Function
* Added #6.6.5. Dataset Instance Resolution for Instances Before the Initial Instance
----++ 1. Coordinator Overview
+## 1. Coordinator Overview
-Users typically run map-reduce, hadoop-streaming, hdfs and/or Pig jobs on the grid. Multiple of these jobs can be combined to form a workflow job. [[https://issues.apache.org/jira/browse/HADOOP-5303][Oozie, Hadoop Workflow System]] defines a workflow system that runs such jobs.
+Users typically run map-reduce, hadoop-streaming, hdfs and/or Pig jobs on the grid. Multiple of these jobs can be combined to form a workflow job. [Oozie, Hadoop Workflow System](https://issues.apache.org/jira/browse/HADOOP-5303) defines a workflow system that runs such jobs.
Commonly, workflow jobs are run based on regular time intervals and/or data availability. And, in some cases, they can be triggered by an external event.
@@ -76,54 +91,54 @@
It is also necessary to connect workflow jobs that run regularly, but at different time intervals. The outputs of multiple subsequent runs of a workflow become the input to the next workflow. For example, the outputs of last 4 runs of a workflow that runs every 15 minutes become the input of another workflow that runs every 60 minutes. Chaining together these workflows result it is referred as a data application pipeline.
-The Oozie *Coordinator* system allows the user to define and execute recurrent and interdependent workflow jobs (data application pipelines).
+The Oozie **Coordinator** system allows the user to define and execute recurrent and interdependent workflow jobs (data application pipelines).
Real world data application pipelines have to account for reprocessing, late processing, catchup, partial processing, monitoring, notification and SLAs.
This document defines the functional specification for the Oozie Coordinator system.
----++ 2. Definitions
+## 2. Definitions
-*Actual time:* The actual time indicates the time when something actually happens.
+**Actual time:** The actual time indicates the time when something actually happens.
-*Nominal time:* The nominal time specifies the time when something should happen. In theory the nominal time and the actual time should match, however, in practice due to delays the actual time may occur later than the nominal time.
+**Nominal time:** The nominal time specifies the time when something should happen. In theory the nominal time and the actual time should match, however, in practice due to delays the actual time may occur later than the nominal time.
-*Dataset:* Collection of data referred to by a logical name. A dataset normally has several instances of data and each
+**Dataset:** Collection of data referred to by a logical name. A dataset normally has several instances of data and each
one of them can be referred individually. Each dataset instance is represented by a unique set of URIs.
-*Synchronous Dataset:* Synchronous datasets instances are generated at fixed time intervals and there is a dataset
+**Synchronous Dataset:** Synchronous datasets instances are generated at fixed time intervals and there is a dataset
instance associated with each time interval. Synchronous dataset instances are identified by their nominal time.
For example, in the case of a HDFS based dataset, the nominal time would be somewhere in the file path of the
-dataset instance: hdfs://foo:8020/usr/logs/2009/04/15/23/30. In the case of HCatalog table partitions, the nominal time
-would be part of some partition values: hcat://bar:8020/mydb/mytable/year=2009;month=04;dt=15;region=us.
+dataset instance: `hdfs://foo:8020/usr/logs/2009/04/15/23/30`. In the case of HCatalog table partitions, the nominal time
+would be part of some partition values: `hcat://bar:8020/mydb/mytable/year=2009;month=04;dt=15;region=us`.
-*Coordinator Action:* A coordinator action is a workflow job that is started when a set of conditions are met (input dataset instances are available).
+**Coordinator Action:** A coordinator action is a workflow job that is started when a set of conditions are met (input dataset instances are available).
-*Coordinator Application:* A coordinator application defines the conditions under which coordinator actions should be created (the frequency) and when the actions can be started. The coordinator application also defines a start and an end time. Normally, coordinator applications are parameterized. A Coordinator application is written in XML.
+**Coordinator Application:** A coordinator application defines the conditions under which coordinator actions should be created (the frequency) and when the actions can be started. The coordinator application also defines a start and an end time. Normally, coordinator applications are parameterized. A Coordinator application is written in XML.
-*Coordinator Job:* A coordinator job is an executable instance of a coordination definition. A job submission is done by submitting a job configuration that resolves all parameters in the application definition.
+**Coordinator Job:** A coordinator job is an executable instance of a coordination definition. A job submission is done by submitting a job configuration that resolves all parameters in the application definition.
-*Data pipeline:* A data pipeline is a connected set of coordinator applications that consume and produce interdependent datasets.
+**Data pipeline:** A data pipeline is a connected set of coordinator applications that consume and produce interdependent datasets.
-*Coordinator Definition Language:* The language used to describe datasets and coordinator applications.
+**Coordinator Definition Language:** The language used to describe datasets and coordinator applications.
-*Coordinator Engine:* A system that executes coordinator jobs.
+**Coordinator Engine:** A system that executes coordinator jobs.
----++ 3. Expression Language for Parameterization
+## 3. Expression Language for Parameterization
Coordinator application definitions can be parameterized with variables, built-in constants and built-in functions.
At execution time all the parameters are resolved into concrete values.
-The parameterization of workflow definitions it done using JSP Expression Language syntax from the [[http://jcp.org/aboutJava/communityprocess/final/jsr152/][JSP 2.0 Specification (JSP.2.3)]], allowing not only to support variables as parameters but also functions and complex expressions.
+The parameterization of workflow definitions it done using JSP Expression Language syntax from the [JSP 2.0 Specification (JSP.2.3)](http://jcp.org/aboutJava/communityprocess/final/jsr152/index.html), allowing not only to support variables as parameters but also functions and complex expressions.
EL expressions can be used in XML attribute values and XML text element values. They cannot be used in XML element and XML attribute names.
Refer to section #6.5 'Parameterization of Coordinator Applications' for more details.
----++ 4. Datetime, Frequency and Time-Period Representation
+## 4. Datetime, Frequency and Time-Period Representation
-Oozie processes coordinator jobs in a fixed timezone with no DST (typically =UTC=), this timezone is referred as 'Oozie
+Oozie processes coordinator jobs in a fixed timezone with no DST (typically `UTC`), this timezone is referred as 'Oozie
processing timezone'.
The Oozie processing timezone is used to resolve coordinator jobs start/end times, job pause times and the initial-instance
@@ -131,42 +146,42 @@
time-zone.
All the datetimes used in coordinator applications and job parameters to coordinator applications must be specified
-in the Oozie processing timezone. If Oozie processing timezone is =UTC=, the qualifier is *Z*. If Oozie processing
-time zone is other than =UTC=, the qualifier must be the GMT offset, =(+/-)####=.
+in the Oozie processing timezone. If Oozie processing timezone is `UTC`, the qualifier is **Z**. If Oozie processing
+time zone is other than `UTC`, the qualifier must be the GMT offset, `(+/-)####`.
-For example, a datetime in =UTC= is =2012-08-12T00:00Z=, the same datetime in =GMT+5:30= is =2012-08-12T05:30+0530=.
+For example, a datetime in `UTC` is `2012-08-12T00:00Z`, the same datetime in `GMT+5:30` is `2012-08-12T05:30+0530`.
-For simplicity, the rest of this specification uses =UTC= datetimes.
+For simplicity, the rest of this specification uses `UTC` datetimes.
-#datetime
----+++ 4.1. Datetime
+<a name="datetime"></a>
+### 4.1. Datetime
-If the Oozie processing timezone is =UTC=, all datetime values are always in
-[[http://en.wikipedia.org/wiki/Coordinated_Universal_Time][UTC]] down to a minute precision, 'YYYY-MM-DDTHH:mmZ'.
+If the Oozie processing timezone is `UTC`, all datetime values are always in
+[UTC](http://en.wikipedia.org/wiki/Coordinated_Universal_Time) down to a minute precision, 'YYYY-MM-DDTHH:mmZ'.
-For example =2009-08-10T13:10Z= is August 10th 2009 at 13:10 UTC.
+For example `2009-08-10T13:10Z` is August 10th 2009 at 13:10 UTC.
-If the Oozie processing timezone is a GMT offset =GMT(+/-)####=, all datetime values are always in
-[[http://en.wikipedia.org/wiki/ISO_8601][ISO 8601]] in the corresponding GMT offset down to a minute precision,
+If the Oozie processing timezone is a GMT offset `GMT(+/-)####`, all datetime values are always in
+[ISO 8601](http://en.wikipedia.org/wiki/ISO_8601) in the corresponding GMT offset down to a minute precision,
'YYYY-MM-DDTHH:mmGMT(+/-)####'.
-For example =2009-08-10T13:10+0530= is August 10th 2009 at 13:10 GMT+0530, India timezone.
+For example `2009-08-10T13:10+0530` is August 10th 2009 at 13:10 GMT+0530, India timezone.
----++++ 4.1.1 End of the day in Datetime Values
+#### 4.1.1 End of the day in Datetime Values
-It is valid to express the end of day as a '24:00' hour (i.e. =2009-08-10T24:00Z=).
+It is valid to express the end of day as a '24:00' hour (i.e. `2009-08-10T24:00Z`).
However, for all calculations and display, Oozie resolves such dates as the zero hour of the following day
-(i.e. =2009-08-11T00:00Z=).
+(i.e. `2009-08-11T00:00Z`).
----+++ 4.2. Timezone Representation
+### 4.2. Timezone Representation
There is no widely accepted standard to identify timezones.
Oozie Coordinator will understand the following timezone identifiers:
- * Generic NON-DST timezone identifier: =GMT[+/-]##:##= (i.e.: GMT+05:30)
- * UTC timezone identifier: =UTC= (i.e.: 2009-06-06T00:00Z)
+ * Generic NON-DST timezone identifier: `GMT[+/-]##:##` (i.e.: GMT+05:30)
+ * UTC timezone identifier: `UTC` (i.e.: 2009-06-06T00:00Z)
* ZoneInfo identifiers, with DST support, understood by Java JDK (about 600 IDs) (i.e.: America/Los_Angeles)
Due to DST shift from PST to PDT, it is preferred that GMT, UTC or Region/City timezone notation is used in
@@ -175,9 +190,9 @@
Oozie Coordinator must provide a tool for developers to list all supported timezone identifiers.
----+++ 4.3. Timezones and Daylight-Saving
+### 4.3. Timezones and Daylight-Saving
-While Oozie coordinator engine works in a fixed timezone with no DST (typically =UTC=), it provides DST support for coordinator applications.
+While Oozie coordinator engine works in a fixed timezone with no DST (typically `UTC`), it provides DST support for coordinator applications.
The baseline datetime for datasets and coordinator applications are expressed in UTC. The baseline datetime is the time of the first occurrence.
@@ -189,85 +204,89 @@
Section #7 'Handling Timezones and Daylight Saving Time' explains how coordinator applications can be written to handle timezones and daylight-saving-time properly.
----+++ 4.4. Frequency and Time-Period Representation
+### 4.4. Frequency and Time-Period Representation
Frequency is used to capture the periodic intervals at which datasets that are produced, and coordinator applications are scheduled to run.
This time periods representation is also used to specify non-recurrent time-periods, for example a timeout interval.
-For datasets and coordinator applications the frequency time-period is applied =N= times to the baseline datetime to compute recurrent times.
+For datasets and coordinator applications the frequency time-period is applied `N` times to the baseline datetime to compute recurrent times.
Frequency is always expressed in minutes.
-Because the number of minutes in day may vary for timezones that observe daylight saving time, constants cannot be use to express frequencies greater than a day for datasets and coordinator applications for such timezones. For such uses cases, Oozie coordinator provides 2 EL functions, =${coord:days(int n)}= and =${coord:months(int n)}=.
+Because the number of minutes in day may vary for timezones that observe daylight saving time, constants cannot be use to express frequencies greater than a day for datasets and coordinator applications for such timezones. For such uses cases, Oozie coordinator provides 2 EL functions, `${coord:days(int n)}` and `${coord:months(int n)}`.
Frequencies can be expressed using EL constants and EL functions that evaluate to an positive integer number.
Coordinator Frequencies can also be expressed using cron syntax.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-| *EL Constant* | *Value* | *Example* |
-| =${coord:minutes(int n)}= | _n_ | =${coord:minutes(45)}= --> =45= |
-| =${coord:hours(int n)}= | _n * 60_ | =${coord:hours(3)}= --> =180= |
-| =${coord:days(int n)}= | _variable_ | =${coord:days(2)}= --> minutes in 2 full days from the current date |
-| =${coord:months(int n)}= | _variable_ | =${coord:months(1)}= --> minutes in a 1 full month from the current date |
-| =${cron syntax}= | _variable_ | =${0,10 15 * * 2-6}= --> a job that runs every weekday at 3:00pm and 3:10pm UTC time|
+| **EL Constant** | **Value** | **Example** |
+| --- | --- | --- |
+| `${coord:minutes(int n)}` | _n_ | `${coord:minutes(45)}` --> `45` |
+| `${coord:hours(int n)}` | _n * 60_ | `${coord:hours(3)}` --> `180` |
+| `${coord:days(int n)}` | _variable_ | `${coord:days(2)}` --> minutes in 2 full days from the current date |
+| `${coord:months(int n)}` | _variable_ | `${coord:months(1)}` --> minutes in a 1 full month from the current date |
+| `${cron syntax}` | _variable_ | `${0,10 15 * * 2-6}` --> a job that runs every weekday at 3:00pm and 3:10pm UTC time|
-Note that, though =${coord:days(int n)}= and =${coord:months(int n)}= EL functions are used to calculate minutes precisely including
+Note that, though `${coord:days(int n)}` and `${coord:months(int n)}` EL functions are used to calculate minutes precisely including
variations due to daylight saving time for Frequency representation, when specified for coordinator timeout interval, one day is
calculated as 24 hours and one month is calculated as 30 days for simplicity.
----++++ 4.4.1. The coord:days(int n) and coord:endOfDays(int n) EL functions
+#### 4.4.1. The coord:days(int n) and coord:endOfDays(int n) EL functions
-The =${coord:days(int n)}= and =${coord:endOfDays(int n)}= EL functions should be used to handle day based frequencies.
+The `${coord:days(int n)}` and `${coord:endOfDays(int n)}` EL functions should be used to handle day based frequencies.
Constant values should not be used to indicate a day based frequency (every 1 day, every 1 week, etc) because the number of hours in
every day is not always the same for timezones that observe daylight-saving time.
-It is a good practice to use always these EL functions instead of using a constant expression (i.e. =24 * 60=) even if the timezone
+It is a good practice to use always these EL functions instead of using a constant expression (i.e. `24 * 60`) even if the timezone
for which the application is being written for does not support daylight saving time. This makes application foolproof to country
legislation changes and also makes applications portable across timezones.
----+++++ 4.4.1.1. The coord:days(int n) EL function
+##### 4.4.1.1. The coord:days(int n) EL function
-The =${coord:days(int n)}= EL function returns the number of minutes for 'n' complete days starting with the day of the specified nominal time for which the computation is being done.
+The `${coord:days(int n)}` EL function returns the number of minutes for 'n' complete days starting with the day of the specified nominal time for which the computation is being done.
-The =${coord:days(int n)}= EL function includes *all* the minutes of the current day, regardless of the time of the day of the current nominal time.
+The `${coord:days(int n)}` EL function includes **all** the minutes of the current day, regardless of the time of the day of the current nominal time.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-| *Starting Nominal UTC time* | *Timezone* | *Usage* | *Value* | *First Occurrence* | *Comments* |
-| =2009-01-01T08:00Z= | =UTC= | =${coord:days(1)}= | 1440 | =2009-01-01T08:00Z= | total minutes on 2009JAN01 UTC time |
-| =2009-01-01T08:00Z= | =America/Los_Angeles= | =${coord:days(1)}= | 1440 | =2009-01-01T08:00Z= | total minutes in 2009JAN01 PST8PDT time |
-| =2009-01-01T08:00Z= | =America/Los_Angeles= | =${coord:days(2)}= | 2880 | =2009-01-01T08:00Z= | total minutes in 2009JAN01 and 2009JAN02 PST8PDT time |
+| **Starting Nominal UTC time** | **Timezone** | **Usage** | **Value** | **First Occurrence** | **Comments** |
+| --- | --- | --- | --- | --- | --- |
+| `2009-01-01T08:00Z` | `UTC` | `${coord:days(1)}` | 1440 | `2009-01-01T08:00Z` | total minutes on 2009JAN01 UTC time |
+| `2009-01-01T08:00Z` | `America/Los_Angeles` | `${coord:days(1)}` | 1440 | `2009-01-01T08:00Z` | total minutes in 2009JAN01 PST8PDT time |
+| `2009-01-01T08:00Z` | `America/Los_Angeles` | `${coord:days(2)}` | 2880 | `2009-01-01T08:00Z` | total minutes in 2009JAN01 and 2009JAN02 PST8PDT time |
| |||||
-| =2009-03-08T08:00Z= | =UTC= | =${coord:days(1)}= | 1440 | =2009-03-08T08:00Z= | total minutes on 2009MAR08 UTC time |
-| =2009-03-08T08:00Z= | =Europe/London= | =${coord:days(1)}= | 1440 | =2009-03-08T08:00Z= | total minutes in 2009MAR08 BST1BDT time |
-| =2009-03-08T08:00Z= | =America/Los_Angeles= | =${coord:days(1)}= | 1380 | =2009-03-08T08:00Z= | total minutes in 2009MAR08 PST8PDT time <br/> (2009MAR08 is DST switch in the US) |
-| =2009-03-08T08:00Z= | =UTC= | =${coord:days(2)}= | 2880 | =2009-03-08T08:00Z= | total minutes in 2009MAR08 and 2009MAR09 UTC time |
-| =2009-03-08T08:00Z= | =America/Los_Angeles= | =${coord:days(2)}= | 2820 | =2009-03-08T08:00Z= | total minutes in 2009MAR08 and 2009MAR09 PST8PDT time <br/> (2009MAR08 is DST switch in the US) |
-| =2009-03-09T08:00Z= | =America/Los_Angeles= | =${coord:days(1)}= | 1440 | =2009-03-09T07:00Z= | total minutes in 2009MAR09 PST8PDT time <br/> (2009MAR08 is DST ON, frequency tick is earlier in UTC) |
+| `2009-03-08T08:00Z` | `UTC` | `${coord:days(1)}` | 1440 | `2009-03-08T08:00Z` | total minutes on 2009MAR08 UTC time |
+| `2009-03-08T08:00Z` | `Europe/London` | `${coord:days(1)}` | 1440 | `2009-03-08T08:00Z` | total minutes in 2009MAR08 BST1BDT time |
+| `2009-03-08T08:00Z` | `America/Los_Angeles` | `${coord:days(1)}` | 1380 | `2009-03-08T08:00Z` | total minutes in 2009MAR08 PST8PDT time <br/> (2009MAR08 is DST switch in the US) |
+| `2009-03-08T08:00Z` | `UTC` | `${coord:days(2)}` | 2880 | `2009-03-08T08:00Z` | total minutes in 2009MAR08 and 2009MAR09 UTC time |
+| `2009-03-08T08:00Z` | `America/Los_Angeles` | `${coord:days(2)}` | 2820 | `2009-03-08T08:00Z` | total minutes in 2009MAR08 and 2009MAR09 PST8PDT time <br/> (2009MAR08 is DST switch in the US) |
+| `2009-03-09T08:00Z` | `America/Los_Angeles` | `${coord:days(1)}` | 1440 | `2009-03-09T07:00Z` | total minutes in 2009MAR09 PST8PDT time <br/> (2009MAR08 is DST ON, frequency tick is earlier in UTC) |
-For all these examples, the first occurrence of the frequency will be at =08:00Z= (UTC time).
+For all these examples, the first occurrence of the frequency will be at `08:00Z` (UTC time).
----+++++ 4.4.1.2. The coord:endOfDays(int n) EL function
+##### 4.4.1.2. The coord:endOfDays(int n) EL function
-The =${coord:endOfDays(int n)}= EL function is identical to the =${coord:days(int n)}= except that it shifts the first occurrence to the end of the day for the specified timezone before computing the interval in minutes.
+The `${coord:endOfDays(int n)}` EL function is identical to the `${coord:days(int n)}` except that it shifts the first occurrence to the end of the day for the specified timezone before computing the interval in minutes.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-| *Starting Nominal UTC time* | *Timezone* | *Usage* | *Value* | *First Occurrence* | *Comments* |
-| =2009-01-01T08:00Z= | =UTC= | =${coord:endOfDays(1)}= | 1440 | =2009-01-02T00:00Z= | first occurrence in 2009JAN02 00:00 UTC time, <br/> first occurrence shifted to the end of the UTC day |
-| =2009-01-01T08:00Z= | =America/Los_Angeles= | =${coord:endOfDays(1)}= | 1440 | =2009-01-02T08:00Z= | first occurrence in 2009JAN02 08:00 UTC time, <br/> first occurrence shifted to the end of the PST8PDT day |
-| =2009-01-01T08:01Z= | =America/Los_Angeles= | =${coord:endOfDays(1)}= | 1440 | =2009-01-02T08:00Z= | first occurrence in 2009JAN02 08:00 UTC time, <br/> first occurrence shifted to the end of the PST8PDT day |
-| =2009-01-01T18:00Z= | =America/Los_Angeles= | =${coord:endOfDays(1)}= | 1440 | =2009-01-02T08:00Z= | first occurrence in 2009JAN02 08:00 UTC time, <br/> first occurrence shifted to the end of the PST8PDT day |
+| **Starting Nominal UTC time** | **Timezone** | **Usage** | **Value** | **First Occurrence** | **Comments** |
+| --- | --- | --- | --- | --- | --- |
+| `2009-01-01T08:00Z` | `UTC` | `${coord:endOfDays(1)}` | 1440 | `2009-01-02T00:00Z` | first occurrence in 2009JAN02 00:00 UTC time, <br/> first occurrence shifted to the end of the UTC day |
+| `2009-01-01T08:00Z` | `America/Los_Angeles` | `${coord:endOfDays(1)}` | 1440 | `2009-01-02T08:00Z` | first occurrence in 2009JAN02 08:00 UTC time, <br/> first occurrence shifted to the end of the PST8PDT day |
+| `2009-01-01T08:01Z` | `America/Los_Angeles` | `${coord:endOfDays(1)}` | 1440 | `2009-01-02T08:00Z` | first occurrence in 2009JAN02 08:00 UTC time, <br/> first occurrence shifted to the end of the PST8PDT day |
+| `2009-01-01T18:00Z` | `America/Los_Angeles` | `${coord:endOfDays(1)}` | 1440 | `2009-01-02T08:00Z` | first occurrence in 2009JAN02 08:00 UTC time, <br/> first occurrence shifted to the end of the PST8PDT day |
| |||||
-| =2009-03-07T09:00Z= | =America/Los_Angeles= | =${coord:endOfDays(1)}= | 1380 | =2009-03-08T08:00Z= | first occurrence in 2009MAR08 08:00 UTC time <br/> first occurrence shifted to the end of the PST8PDT day |
-| =2009-03-08T07:00Z= | =America/Los_Angeles= | =${coord:endOfDays(1)}= | 1440 | =2009-03-08T08:00Z= | first occurrence in 2009MAR08 08:00 UTC time <br/> first occurrence shifted to the end of the PST8PDT day |
-| =2009-03-09T07:00Z= | =America/Los_Angeles= | =${coord:endOfDays(1)}= | 1440 | =2009-03-10T07:00Z= | first occurrence in 2009MAR10 07:00 UTC time <br/> (2009MAR08 is DST switch in the US), <br/> first occurrence shifted to the end of the PST8PDT day |
+| `2009-03-07T09:00Z` | `America/Los_Angeles` | `${coord:endOfDays(1)}` | 1380 | `2009-03-08T08:00Z` | first occurrence in 2009MAR08 08:00 UTC time <br/> first occurrence shifted to the end of the PST8PDT day |
+| `2009-03-08T07:00Z` | `America/Los_Angeles` | `${coord:endOfDays(1)}` | 1440 | `2009-03-08T08:00Z` | first occurrence in 2009MAR08 08:00 UTC time <br/> first occurrence shifted to the end of the PST8PDT day |
+| `2009-03-09T07:00Z` | `America/Los_Angeles` | `${coord:endOfDays(1)}` | 1440 | `2009-03-10T07:00Z` | first occurrence in 2009MAR10 07:00 UTC time <br/> (2009MAR08 is DST switch in the US), <br/> first occurrence shifted to the end of the PST8PDT day |
-<verbatim>
+
+```
<coordinator-app name="hello-coord" frequency="${coord:days(1)}"
start="2009-01-02T08:00Z" end="2009-01-04T08:00Z" timezone="America/Los_Angeles"
xmlns="uri:oozie:coordinator:0.5">
@@ -317,48 +336,51 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
----++++ 4.4.2. The coord:months(int n) and coord:endOfMonths(int n) EL functions
+#### 4.4.2. The coord:months(int n) and coord:endOfMonths(int n) EL functions
-The =${coord:months(int n)}= and =${coord:endOfMonths(int n)}= EL functions should be used to handle month based frequencies.
+The `${coord:months(int n)}` and `${coord:endOfMonths(int n)}` EL functions should be used to handle month based frequencies.
Constant values cannot be used to indicate a month based frequency because the number of days in a month changes month to month and on leap years; plus the number of hours in every day of the month are not always the same for timezones that observe daylight-saving time.
----+++++ 4.4.2.1. The coord:months(int n) EL function
+##### 4.4.2.1. The coord:months(int n) EL function
-The =${coord:months(int n)}= EL function returns the number of minutes for 'n' complete months starting with the month of the current nominal time for which the computation is being done.
+The `${coord:months(int n)}` EL function returns the number of minutes for 'n' complete months starting with the month of the current nominal time for which the computation is being done.
-The =${coord:months(int n)}= EL function includes *all* the minutes of the current month, regardless of the day of the month of the current nominal time.
+The `${coord:months(int n)}` EL function includes **all** the minutes of the current month, regardless of the day of the month of the current nominal time.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-| *Starting Nominal UTC time* | *Timezone* | *Usage* | *Value* | *First Occurrence* | *Comments* |
-| =2009-01-01T08:00Z= | =UTC= | =${coord:months(1)}= | 44640 | =2009-01-01T08:00Z= |total minutes for 2009JAN UTC time |
-| =2009-01-01T08:00Z= | =America/Los_Angeles= | =${coord:months(1)}= | 44640 | =2009-01-01T08:00Z= | total minutes in 2009JAN PST8PDT time |
-| =2009-01-01T08:00Z= | =America/Los_Angeles= | =${coord:months(2)}= | 84960 | =2009-01-01T08:00Z= | total minutes in 2009JAN and 2009FEB PST8PDT time |
+| **Starting Nominal UTC time** | **Timezone** | **Usage** | **Value** | **First Occurrence** | **Comments** |
+| --- | --- | --- | --- | --- | --- |
+| `2009-01-01T08:00Z` | `UTC` | `${coord:months(1)}` | 44640 | `2009-01-01T08:00Z` |total minutes for 2009JAN UTC time |
+| `2009-01-01T08:00Z` | `America/Los_Angeles` | `${coord:months(1)}` | 44640 | `2009-01-01T08:00Z` | total minutes in 2009JAN PST8PDT time |
+| `2009-01-01T08:00Z` | `America/Los_Angeles` | `${coord:months(2)}` | 84960 | `2009-01-01T08:00Z` | total minutes in 2009JAN and 2009FEB PST8PDT time |
| |||||
-| =2009-03-08T08:00Z= | =UTC= | =${coord:months(1)}= | 44640 | =2009-03-08T08:00Z= | total minutes on 2009MAR UTC time |
-| =2009-03-08T08:00Z= | =Europe/London= | =${coord:months(1)}= | 44580 | =2009-03-08T08:00Z= | total minutes in 2009MAR BST1BDT time <br/> (2009MAR29 is DST switch in Europe) |
-| =2009-03-08T08:00Z= | =America/Los_Angeles= | =${coord:months(1)}= | 44580 | =2009-03-08T08:00Z= | total minutes in 2009MAR PST8PDT time <br/> (2009MAR08 is DST switch in the US) |
-| =2009-03-08T08:00Z= | =UTC= | =${coord:months(2)}= | 87840 | =2009-03-08T08:00Z= | total minutes in 2009MAR and 2009APR UTC time |
-| =2009-03-08T08:00Z= | =America/Los_Angeles= | =${coord:months(2)}= | 87780 | =2009-03-08T08:00Z= | total minutes in 2009MAR and 2009APR PST8PDT time <br/> (2009MAR08 is DST switch in US) |
+| `2009-03-08T08:00Z` | `UTC` | `${coord:months(1)}` | 44640 | `2009-03-08T08:00Z` | total minutes on 2009MAR UTC time |
+| `2009-03-08T08:00Z` | `Europe/London` | `${coord:months(1)}` | 44580 | `2009-03-08T08:00Z` | total minutes in 2009MAR BST1BDT time <br/> (2009MAR29 is DST switch in Europe) |
+| `2009-03-08T08:00Z` | `America/Los_Angeles` | `${coord:months(1)}` | 44580 | `2009-03-08T08:00Z` | total minutes in 2009MAR PST8PDT time <br/> (2009MAR08 is DST switch in the US) |
+| `2009-03-08T08:00Z` | `UTC` | `${coord:months(2)}` | 87840 | `2009-03-08T08:00Z` | total minutes in 2009MAR and 2009APR UTC time |
+| `2009-03-08T08:00Z` | `America/Los_Angeles` | `${coord:months(2)}` | 87780 | `2009-03-08T08:00Z` | total minutes in 2009MAR and 2009APR PST8PDT time <br/> (2009MAR08 is DST switch in US) |
----+++++ 4.4.2.2. The coord:endOfMonths(int n) EL function
+##### 4.4.2.2. The coord:endOfMonths(int n) EL function
-The =${coord:endOfMonths(int n)}= EL function is identical to the =${coord:months(int n)}= except that it shifts the first occurrence to the end of the month for the specified timezone before computing the interval in minutes.
+The `${coord:endOfMonths(int n)}` EL function is identical to the `${coord:months(int n)}` except that it shifts the first occurrence to the end of the month for the specified timezone before computing the interval in minutes.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-| *Starting Nominal UTC time* | *Timezone* | *Usage* | *Value* | *First Occurrence* | *Comments* |
-| =2009-01-01T00:00Z= | =UTC= | =${coord:endOfMonths(1)}= | 40320 | =2009-02-01T00:00Z= | first occurrence in 2009FEB 00:00 UTC time |
-| =2009-01-01T08:00Z= | =UTC= | =${coord:endOfMonths(1)}= | 40320 | =2009-02-01T00:00Z= | first occurrence in 2009FEB 00:00 UTC time |
-| =2009-01-31T08:00Z= | =UTC= | =${coord:endOfMonths(1)}= | 40320 | =2009-02-01T00:00Z= | first occurrence in 2009FEB 00:00 UTC time |
-| =2009-01-01T08:00Z= | =America/Los_Angeles= | =${coord:endOfMonths(1)}= | 40320 | =2009-02-01T08:00Z= | first occurrence in 2009FEB 08:00 UTC time |
-| =2009-02-02T08:00Z= | =America/Los_Angeles= | =${coord:endOfMonths(1)}= | 44580 | =2009-03-01T08:00Z= | first occurrence in 2009MAR 08:00 UTC time |
-| =2009-02-01T08:00Z= | =America/Los_Angeles= | =${coord:endOfMonths(1)}= | 44580 | =2009-03-01T08:00Z= | first occurrence in 2009MAR 08:00 UTC time |
+| **Starting Nominal UTC time** | **Timezone** | **Usage** | **Value** | **First Occurrence** | **Comments** |
+| --- | --- | --- | --- | --- | --- |
+| `2009-01-01T00:00Z` | `UTC` | `${coord:endOfMonths(1)}` | 40320 | `2009-02-01T00:00Z` | first occurrence in 2009FEB 00:00 UTC time |
+| `2009-01-01T08:00Z` | `UTC` | `${coord:endOfMonths(1)}` | 40320 | `2009-02-01T00:00Z` | first occurrence in 2009FEB 00:00 UTC time |
+| `2009-01-31T08:00Z` | `UTC` | `${coord:endOfMonths(1)}` | 40320 | `2009-02-01T00:00Z` | first occurrence in 2009FEB 00:00 UTC time |
+| `2009-01-01T08:00Z` | `America/Los_Angeles` | `${coord:endOfMonths(1)}` | 40320 | `2009-02-01T08:00Z` | first occurrence in 2009FEB 08:00 UTC time |
+| `2009-02-02T08:00Z` | `America/Los_Angeles` | `${coord:endOfMonths(1)}` | 44580 | `2009-03-01T08:00Z` | first occurrence in 2009MAR 08:00 UTC time |
+| `2009-02-01T08:00Z` | `America/Los_Angeles` | `${coord:endOfMonths(1)}` | 44580 | `2009-03-01T08:00Z` | first occurrence in 2009MAR 08:00 UTC time |
-<verbatim>
+
+```
<coordinator-app name="hello-coord" frequency="${coord:months(1)}"
start="2009-01-02T08:00Z" end="2009-04-02T08:00Z" timezone="America/Los_Angeles"
xmlns="uri:oozie:coordinator:0.5">
@@ -408,25 +430,27 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
----++++ 4.4.3. The coord:endOfWeeks(int n) EL function
+#### 4.4.3. The coord:endOfWeeks(int n) EL function
-The =${coord:endOfWeeks(int n)}= EL function shifts the first occurrence to the start of the week for the specified
+The `${coord:endOfWeeks(int n)}` EL function shifts the first occurrence to the start of the week for the specified
timezone before computing the interval in minutes. The start of the week depends on the Java's implementation of
-[[https://docs.oracle.com/javase/8/docs/api/java/util/Calendar.html#getFirstDayOfWeek--][Calendar.getFirstDayOfWeek()]]
+[Calendar.getFirstDayOfWeek()](https://docs.oracle.com/javase/8/docs/api/java/util/Calendar.html#getFirstDayOfWeek--)
i.e. first day of the week is SUNDAY in the U.S., MONDAY in France.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-| *Starting Nominal UTC time* | *Timezone* | *Usage* | *Value* | *First Occurrence* | *Comments* |
-| =2017-01-04T00:00Z= | =UTC= | =${coord:endOfWeeks(1)}= | 10080 | =2017-01-08T00:00Z= | first occurrence on 2017JAN08 08:00 UTC time |
-| =2017-01-04T08:00Z= | =UTC= | =${coord:endOfWeeks(1)}= | 10080 | =2017-01-08T08:00Z= | first occurrence on 2017JAN08 08:00 UTC time |
-| =2017-01-06T08:00Z= | =UTC= | =${coord:endOfWeeks(1)}= | 10080 | =2017-01-08T08:00Z= | first occurrence on 2017JAN08 08:00 UTC time |
-| =2017-01-04T08:00Z= | =America/Los_Angeles= | =${coord:endOfWeeks(1)}= | 10080 | =2017-01-08T08:00Z= | first occurrence in 2017JAN08 08:00 UTC time |
-| =2017-01-06T08:00Z= | =America/Los_Angeles= | =${coord:endOfWeeks(1)}= | 10080 | =2017-01-08T08:00Z= | first occurrence in 2017JAN08 08:00 UTC time |
+| **Starting Nominal UTC time** | **Timezone** | **Usage** | **Value** | **First Occurrence** | **Comments** |
+| --- | --- | --- | --- | --- | --- |
+| `2017-01-04T00:00Z` | `UTC` | `${coord:endOfWeeks(1)}` | 10080 | `2017-01-08T00:00Z` | first occurrence on 2017JAN08 08:00 UTC time |
+| `2017-01-04T08:00Z` | `UTC` | `${coord:endOfWeeks(1)}` | 10080 | `2017-01-08T08:00Z` | first occurrence on 2017JAN08 08:00 UTC time |
+| `2017-01-06T08:00Z` | `UTC` | `${coord:endOfWeeks(1)}` | 10080 | `2017-01-08T08:00Z` | first occurrence on 2017JAN08 08:00 UTC time |
+| `2017-01-04T08:00Z` | `America/Los_Angeles` | `${coord:endOfWeeks(1)}` | 10080 | `2017-01-08T08:00Z` | first occurrence in 2017JAN08 08:00 UTC time |
+| `2017-01-06T08:00Z` | `America/Los_Angeles` | `${coord:endOfWeeks(1)}` | 10080 | `2017-01-08T08:00Z` | first occurrence in 2017JAN08 08:00 UTC time |
-<verbatim>
+
+```
<coordinator-app name="hello-coord" frequency="${coord:endOfWeeks(1)}"
start="2017-01-04T08:00Z" end="2017-12-31T08:00Z" timezone="America/Los_Angeles"
xmlns="uri:oozie:coordinator:0.5">
@@ -476,9 +500,9 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
----++++ 4.4.4. Cron syntax in coordinator frequency
+#### 4.4.4. Cron syntax in coordinator frequency
Oozie has historically allowed only very basic forms of scheduling: You could choose
to run jobs separated by a certain number of minutes, hours, days or weeks. That's
@@ -497,8 +521,9 @@
administrators to setup jobs and maintain software environment. Cron syntax generally consists of five fields, minutes,
hours, date of month, month, and day of week respectively although multiple variations do exist.
-<verbatim>
-<coordinator-app name="cron-coord" frequency="0/10 1/2 * * *" start="${start}" end="${end}" timezone="UTC"
+
+```
+<coordinator-app name="cron-coord" frequency="0/10 1/2 ** ** *" start="${start}" end="${end}" timezone="UTC"
xmlns="uri:oozie:coordinator:0.2">
<action>
<workflow>
@@ -520,18 +545,19 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
Cron expressions are comprised of 5 required fields. The fields respectively are described as follows:
-| *Field name* | *Allowed Values* | *Allowed Special Characters* |
-| =Minutes= | =0-59= | , - * / |
-| =Hours= | =0-23= | , - * / |
-| =Day-of-month= | =1-31= | , - * ? / L W |
-| =Month= | =1-12 or JAN-DEC= | , - * / |
-| =Day-of-Week= | =1-7 or SUN-SAT= | , - * ? / L #|
+| **Field name** | **Allowed Values** | **Allowed Special Characters** |
+| --- | --- | --- |
+| `Minutes` | `0-59` | , - * / |
+| `Hours` | `0-23` | , - * / |
+| `Day-of-month` | `1-31` | , - * ? / L W |
+| `Month` | `1-12 or JAN-DEC` | , - * / |
+| `Day-of-Week` | `1-7 or SUN-SAT` | , - * ? / L #|
-The '*' character is used to specify all values. For example, "*" in the minute field means "every minute".
+The '**' character is used to specify all values. For example, "**" in the minute field means "every minute".
The '?' character is allowed for the day-of-month and day-of-week fields. It is used to specify 'no specific value'.
This is useful when you need to specify something in one of the two fields, but not the other.
@@ -585,21 +611,22 @@
If a user specifies an invalid cron syntax to run something on Feb, 30th for example: "0 10 30 2 *", the coordinator job
will not be created and an invalid coordinator frequency parse exception will be thrown.
-If a user has a coordinator job that materializes no action during run time, for example: frequency of "0 10 * * *" with
+If a user has a coordinator job that materializes no action during run time, for example: frequency of "0 10 ** ** *" with
start time of 2013-10-18T21:00Z and end time of 2013-10-18T22:00Z, the coordinator job submission will be rejected and
an invalid coordinator attribute exception will be thrown.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-| *Cron Expression* | *Meaning* |
-| 10 9 * * * | Runs everyday at 9:10am |
-| 10,30,45 9 * * * | Runs everyday at 9:10am, 9:30am, and 9:45am |
-| =0 * 30 JAN 2-6= | Runs at 0 minute of every hour on weekdays and 30th of January |
-| =0/20 9-17 * * 2-5= | Runs every Mon, Tue, Wed, and Thurs at minutes 0, 20, 40 from 9am to 5pm |
-| 1 2 L-3 * * | Runs every third-to-last day of month at 2:01am |
-| =1 2 6W 3 ?= | Runs on the nearest weekday to March, 6th every year at 2:01am |
-| =1 2 * 3 3#2= | Runs every second Tuesday of March at 2:01am every year |
-| =0 10,13 * * MON-FRI= | Runs every weekday at 10am and 1pm |
+| **Cron Expression** | **Meaning** |
+| --- | --- |
+| 10 9 ** ** * | Runs everyday at 9:10am |
+| 10,30,45 9 ** ** * | Runs everyday at 9:10am, 9:30am, and 9:45am |
+| `0 * 30 JAN 2-6` | Runs at 0 minute of every hour on weekdays and 30th of January |
+| `0/20 9-17 ** ** 2-5` | Runs every Mon, Tue, Wed, and Thurs at minutes 0, 20, 40 from 9am to 5pm |
+| 1 2 L-3 ** ** | Runs every third-to-last day of month at 2:01am |
+| `1 2 6W 3 ?` | Runs on the nearest weekday to March, 6th every year at 2:01am |
+| `1 2 * 3 3#2` | Runs every second Tuesday of March at 2:01am every year |
+| `0 10,13 ** ** MON-FRI` | Runs every weekday at 10am and 1pm |
NOTES:
@@ -619,7 +646,7 @@
no effort has been made to determine which interpretation CronExpression chooses.
An example would be "0 14-6 ? * FRI-MON".
----++ 5. Dataset
+## 5. Dataset
A dataset is a collection of data referred to by a logical name.
@@ -631,7 +658,7 @@
A dataset instance is considered to be immutable while it is being consumed by coordinator jobs.
----+++ 5.1. Synchronous Datasets
+### 5.1. Synchronous Datasets
Instances of synchronous datasets are produced at regular time intervals, at an expected frequency. They are also referred to as "clocked datasets".
@@ -639,175 +666,190 @@
A synchronous dataset definition contains the following information:
- * *%BLUE% name: %ENDCOLOR%* The dataset name. It must be a valid Java identifier.
- * *%BLUE% frequency: %ENDCOLOR%* It represents the rate, in minutes at which data is _periodically_ created. The granularity is in minutes and can be expressed using EL expressions, for example: ${5 * HOUR}.
- * *%BLUE% initial-instance: %ENDCOLOR%* The UTC datetime of the initial instance of the dataset. The initial-instance also provides the baseline datetime to compute instances of the dataset using multiples of the frequency.
- * *%BLUE% timezone:%ENDCOLOR%* The timezone of the dataset.
- * *%BLUE% uri-template:%ENDCOLOR%* The URI template that identifies the dataset and can be resolved into concrete URIs to identify a particular dataset instance. The URI template is constructed using:
- * *%BLUE% constants %ENDCOLOR%* See the allowable EL Time Constants below. Ex: ${YEAR}/${MONTH}.
- * *%BLUE% variables %ENDCOLOR%* Variables must be resolved at the time a coordinator job is submitted to the coordinator engine. They are normally provided a job parameters (configuration properties). Ex: ${market}/${language}
- * *%BLUE% done-flag:%ENDCOLOR%* This flag denotes when a dataset instance is ready to be consumed.
+ * **<font color="#0000ff"> name: </font>** The dataset name. It must be a valid Java identifier.
+ * **<font color="#0000ff"> frequency: </font>*** It represents the rate, in minutes at which data is _periodically_ created. The granularity is in minutes and can be expressed using EL expressions, for example: ${5 ** HOUR}.
+ * **<font color="#0000ff"> initial-instance: </font>** The UTC datetime of the initial instance of the dataset. The initial-instance also provides the baseline datetime to compute instances of the dataset using multiples of the frequency.
+ * **<font color="#0000ff"> timezone:</font>** The timezone of the dataset.
+ * **<font color="#0000ff"> uri-template:</font>** The URI template that identifies the dataset and can be resolved into concrete URIs to identify a particular dataset instance. The URI template is constructed using:
+ * **<font color="#0000ff"> constants </font>** See the allowable EL Time Constants below. Ex: ${YEAR}/${MONTH}.
+ * **<font color="#0000ff"> variables </font>** Variables must be resolved at the time a coordinator job is submitted to the coordinator engine. They are normally provided a job parameters (configuration properties). Ex: ${market}/${language}
+ * **<font color="#0000ff"> done-flag:</font>** This flag denotes when a dataset instance is ready to be consumed.
* If the done-flag is omitted the coordinator will wait for the presence of a _SUCCESS file in the directory (Note: MapReduce jobs create this on successful completion automatically).
* If the done-flag is present but empty, then the existence of the directory itself indicates that the dataset is ready.
* If the done-flag is present but non-empty, Oozie will check for the presence of the named file within the directory, and will be considered ready (done) when the file exists.
The following EL constants can be used within synchronous dataset URI templates:
-| *EL Constant* | *Resulting Format* | *Comments* |
-| =YEAR= | _YYYY_ | 4 digits representing the year |
-| =MONTH= | _MM_ | 2 digits representing the month of the year, January = 1 |
-| =DAY= | _DD_ | 2 digits representing the day of the month |
-| =HOUR= | _HH_ | 2 digits representing the hour of the day, in 24 hour format, 0 - 23 |
-| =MINUTE= | _mm_ | 2 digits representing the minute of the hour, 0 - 59 |
+| **EL Constant** | **Resulting Format** | **Comments** |
+| --- | --- | --- |
+| `YEAR` | _YYYY_ | 4 digits representing the year |
+| `MONTH` | _MM_ | 2 digits representing the month of the year, January = 1 |
+| `DAY` | _DD_ | 2 digits representing the day of the month |
+| `HOUR` | _HH_ | 2 digits representing the hour of the day, in 24 hour format, 0 - 23 |
+| `MINUTE` | _mm_ | 2 digits representing the minute of the hour, 0 - 59 |
-*%PURPLE% Syntax: %ENDCOLOR%*
+**<font color="#800080">Syntax: </font>**
-<verbatim>
+
+```
<dataset name="[NAME]" frequency="[FREQUENCY]"
initial-instance="[DATETIME]" timezone="[TIMEZONE]">
<uri-template>[URI TEMPLATE]</uri-template>
<done-flag>[FILE NAME]</done-flag>
</dataset>
-</verbatim>
+```
IMPORTANT: The values of the EL constants in the dataset URIs (in HDFS) are expected in UTC. Oozie Coordinator takes care of the timezone conversion when performing calculations.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-1. *A dataset produced once every day at 00:15 PST8PDT and done-flag is set to empty:*
-
-<verbatim>
- <dataset name="logs" frequency="${coord:days(1)}"
- initial-instance="2009-02-15T08:15Z" timezone="America/Los_Angeles">
- <uri-template>
- hdfs://foo:8020/app/logs/${market}/${YEAR}${MONTH}/${DAY}/data
- </uri-template>
- <done-flag></done-flag>
- </dataset>
-</verbatim>
+1. **A dataset produced once every day at 00:15 PST8PDT and done-flag is set to empty:**
-The dataset would resolve to the following URIs and Coordinator looks for the existence of the directory itself:
-
-<verbatim>
- [market] will be replaced with user given property.
-
- hdfs://foo:8020/usr/app/[market]/2009/02/15/data
- hdfs://foo:8020/usr/app/[market]/2009/02/16/data
- hdfs://foo:8020/usr/app/[market]/2009/02/17/data
- ...
-</verbatim>
+ ```
+ <dataset name="logs" frequency="${coord:days(1)}"
+ initial-instance="2009-02-15T08:15Z" timezone="America/Los_Angeles">
+ <uri-template>
+ hdfs://foo:8020/app/logs/${market}/${YEAR}${MONTH}/${DAY}/data
+ </uri-template>
+ <done-flag></done-flag>
+ </dataset>
+ ```
-2. *A dataset available on the 10th of each month and done-flag is default '_SUCCESS':*
-
-<verbatim>
- <dataset name="stats" frequency="${coord:months(1)}"
- initial-instance="2009-01-10T10:00Z" timezone="America/Los_Angeles">
- <uri-template>hdfs://foo:8020/usr/app/stats/${YEAR}/${MONTH}/data</uri-template>
- </dataset>
-</verbatim>
-
-The dataset would resolve to the following URIs:
-
-<verbatim>
- hdfs://foo:8020/usr/app/stats/2009/01/data
- hdfs://foo:8020/usr/app/stats/2009/02/data
- hdfs://foo:8020/usr/app/stats/2009/03/data
- ...
-</verbatim>
-
-The dataset instances are not ready until '_SUCCESS' exists in each path:
-
-<verbatim>
- hdfs://foo:8020/usr/app/stats/2009/01/data/_SUCCESS
- hdfs://foo:8020/usr/app/stats/2009/02/data/_SUCCESS
- hdfs://foo:8020/usr/app/stats/2009/03/data/_SUCCESS
- ...
-</verbatim>
+ The dataset would resolve to the following URIs and Coordinator looks for the existence of the directory itself:
-3. *A dataset available at the end of every quarter and done-flag is 'trigger.dat':*
+ ```
+ [market] will be replaced with user given property.
-<verbatim>
- <dataset name="stats" frequency="${coord:months(3)}"
- initial-instance="2009-01-31T20:00Z" timezone="America/Los_Angeles">
- <uri-template>
- hdfs://foo:8020/usr/app/stats/${YEAR}/${MONTH}/data
- </uri-template>
- <done-flag>trigger.dat</done-flag>
- </dataset>
-</verbatim>
-
-The dataset would resolve to the following URIs:
-
-<verbatim>
- hdfs://foo:8020/usr/app/stats/2009/01/data
- hdfs://foo:8020/usr/app/stats/2009/04/data
- hdfs://foo:8020/usr/app/stats/2009/07/data
- ...
-</verbatim>
-
-The dataset instances are not ready until 'trigger.dat' exists in each path:
-
-<verbatim>
- hdfs://foo:8020/usr/app/stats/2009/01/data/trigger.dat
- hdfs://foo:8020/usr/app/stats/2009/04/data/trigger.dat
- hdfs://foo:8020/usr/app/stats/2009/07/data/trigger.dat
- ...
-</verbatim>
+ hdfs://foo:8020/usr/app/[market]/2009/02/15/data
+ hdfs://foo:8020/usr/app/[market]/2009/02/16/data
+ hdfs://foo:8020/usr/app/[market]/2009/02/17/data
+ ...
+ ```
-4. *Normally the URI template of a dataset has a precision similar to the frequency:*
+2. **A dataset available on the 10th of each month and done-flag is default '_SUCCESS':**
-<verbatim>
- <dataset name="logs" frequency="${coord:days(1)}"
- initial-instance="2009-01-01T10:30Z" timezone="America/Los_Angeles">
- <uri-template>
- hdfs://foo:8020/usr/app/logs/${YEAR}/${MONTH}/${DAY}/data
- </uri-template>
- </dataset>
-</verbatim>
-The dataset would resolve to the following URIs:
+ ```
+ <dataset name="stats" frequency="${coord:months(1)}"
+ initial-instance="2009-01-10T10:00Z" timezone="America/Los_Angeles">
+ <uri-template>hdfs://foo:8020/usr/app/stats/${YEAR}/${MONTH}/data</uri-template>
+ </dataset>
+ ```
-<verbatim>
- hdfs://foo:8020/usr/app/logs/2009/01/01/data
- hdfs://foo:8020/usr/app/logs/2009/01/02/data
- hdfs://foo:8020/usr/app/logs/2009/01/03/data
- ...
-</verbatim>
+ The dataset would resolve to the following URIs:
-5. *However, if the URI template has a finer precision than the dataset frequency:*
-<verbatim>
- <dataset name="logs" frequency="${coord:days(1)}"
- initial-instance="2009-01-01T10:30Z" timezone="America/Los_Angeles">
- <uri-template>
- hdfs://foo:8020/usr/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/data
- </uri-template>
- </dataset>
-</verbatim>
+ ```
+ hdfs://foo:8020/usr/app/stats/2009/01/data
+ hdfs://foo:8020/usr/app/stats/2009/02/data
+ hdfs://foo:8020/usr/app/stats/2009/03/data
+ ...
+ ```
-The dataset resolves to the following URIs with fixed values for the finer precision template variables:
+ The dataset instances are not ready until '_SUCCESS' exists in each path:
-<verbatim>
- hdfs://foo:8020/usr/app/logs/2009/01/01/10/30/data
- hdfs://foo:8020/usr/app/logs/2009/01/02/10/30/data
- hdfs://foo:8020/usr/app/logs/2009/01/03/10/30/data
- ...
-</verbatim>
----+++ 5.2. Dataset URI-Template types
+ ```
+ hdfs://foo:8020/usr/app/stats/2009/01/data/_SUCCESS
+ hdfs://foo:8020/usr/app/stats/2009/02/data/_SUCCESS
+ hdfs://foo:8020/usr/app/stats/2009/03/data/_SUCCESS
+ ...
+ ```
-Each dataset URI could be a HDFS path URI denoting a HDFS directory: hdfs://foo:8020/usr/logs/20090415 or a
-HCatalog partition URI identifying a set of table partitions: hcat://bar:8020/logsDB/logsTable/dt=20090415;region=US.
+
+3. **A dataset available at the end of every quarter and done-flag is 'trigger.dat':**
+
+
+ ```
+ <dataset name="stats" frequency="${coord:months(3)}"
+ initial-instance="2009-01-31T20:00Z" timezone="America/Los_Angeles">
+ <uri-template>
+ hdfs://foo:8020/usr/app/stats/${YEAR}/${MONTH}/data
+ </uri-template>
+ <done-flag>trigger.dat</done-flag>
+ </dataset>
+ ```
+
+ The dataset would resolve to the following URIs:
+
+
+ ```
+ hdfs://foo:8020/usr/app/stats/2009/01/data
+ hdfs://foo:8020/usr/app/stats/2009/04/data
+ hdfs://foo:8020/usr/app/stats/2009/07/data
+ ...
+ ```
+
+ The dataset instances are not ready until 'trigger.dat' exists in each path:
+
+
+ ```
+ hdfs://foo:8020/usr/app/stats/2009/01/data/trigger.dat
+ hdfs://foo:8020/usr/app/stats/2009/04/data/trigger.dat
+ hdfs://foo:8020/usr/app/stats/2009/07/data/trigger.dat
+ ...
+ ```
+
+
+4. **Normally the URI template of a dataset has a precision similar to the frequency:**
+
+
+ ```
+ <dataset name="logs" frequency="${coord:days(1)}"
+ initial-instance="2009-01-01T10:30Z" timezone="America/Los_Angeles">
+ <uri-template>
+ hdfs://foo:8020/usr/app/logs/${YEAR}/${MONTH}/${DAY}/data
+ </uri-template>
+ </dataset>
+ ```
+
+ The dataset would resolve to the following URIs:
+
+
+ ```
+ hdfs://foo:8020/usr/app/logs/2009/01/01/data
+ hdfs://foo:8020/usr/app/logs/2009/01/02/data
+ hdfs://foo:8020/usr/app/logs/2009/01/03/data
+ ...
+ ```
+
+5. **However, if the URI template has a finer precision than the dataset frequency:**
+
+
+ ```
+ <dataset name="logs" frequency="${coord:days(1)}"
+ initial-instance="2009-01-01T10:30Z" timezone="America/Los_Angeles">
+ <uri-template>
+ hdfs://foo:8020/usr/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}/data
+ </uri-template>
+ </dataset>
+ ```
+
+ The dataset resolves to the following URIs with fixed values for the finer precision template variables:
+
+
+ ```
+ hdfs://foo:8020/usr/app/logs/2009/01/01/10/30/data
+ hdfs://foo:8020/usr/app/logs/2009/01/02/10/30/data
+ hdfs://foo:8020/usr/app/logs/2009/01/03/10/30/data
+ ...
+ ```
+
+### 5.2. Dataset URI-Template types
+
+Each dataset URI could be a HDFS path URI denoting a HDFS directory: `hdfs://foo:8020/usr/logs/20090415` or a
+HCatalog partition URI identifying a set of table partitions: `hcat://bar:8020/logsDB/logsTable/dt=20090415;region=US`.
HCatalog enables table and storage management for Pig, Hive and MapReduce. The format to specify a HCatalog table partition URI is
-hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value];...
+`hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value];...`
For example,
-<verbatim>
+
+```
<dataset name="logs" frequency="${coord:days(1)}"
initial-instance="2009-02-15T08:15Z" timezone="America/Los_Angeles">
<uri-template>
@@ -815,19 +857,20 @@
</uri-template>
<done-flag></done-flag>
</dataset>
-</verbatim>
+```
----+++ 5.3. Asynchronous Datasets
+### 5.3. Asynchronous Datasets
* TBD
----+++ 5.4. Dataset Definitions
+### 5.4. Dataset Definitions
Dataset definitions are grouped in XML files.
-*IMPORTANT:* Please note that if an XML namespace version is specified for the coordinator-app element in the coordinator.xml file, no namespace needs to be defined separately for the datasets element (even if the dataset is defined in a separate file). Specifying it at multiple places might result in xml errors while submitting the coordinator job.
+**IMPORTANT:** Please note that if an XML namespace version is specified for the coordinator-app element in the coordinator.xml file, no namespace needs to be defined separately for the datasets element (even if the dataset is defined in a separate file). Specifying it at multiple places might result in xml errors while submitting the coordinator job.
-*%PURPLE% Syntax: %ENDCOLOR%*
+**<font color="#800080">Syntax: </font>**
-<verbatim>
+
+```
<!-- Synchronous datasets -->
<datasets>
<include>[SHARED_DATASETS]</include>
@@ -838,11 +881,12 @@
</dataset>
...
</datasets>
-</verbatim>
+```
-*%GREEN% Example: %ENDCOLOR%*
+**<font color="#008000"> Example: </font>**
-<verbatim>
+
+```
<datasets>
.
<include>hdfs://foo:8020/app/dataset-definitions/globallogs.xml</include>
@@ -860,74 +904,74 @@
</dataset>
.
</datasets>
-</verbatim>
+```
----++ 6. Coordinator Application
+## 6. Coordinator Application
----+++ 6.1. Concepts
+### 6.1. Concepts
----++++ 6.1.1. Coordinator Application
+#### 6.1.1. Coordinator Application
A coordinator application is a program that triggers actions (commonly workflow jobs) when a set of conditions are met. Conditions can be a time frequency, the availability of new dataset instances or other external events.
Types of coordinator applications:
- * *Synchronous:* Its coordinator actions are created at specified time intervals.
+ * **Synchronous:** Its coordinator actions are created at specified time intervals.
Coordinator applications are normally parameterized.
----++++ 6.1.2. Coordinator Job
+#### 6.1.2. Coordinator Job
To create a coordinator job, a job configuration that resolves all coordinator application parameters must be provided to the coordinator engine.
A coordinator job is a running instance of a coordinator application running from a start time to an end time. The start
time must be earlier than the end time.
-At any time, a coordinator job is in one of the following status: *PREP, RUNNING, RUNNINGWITHERROR, PREPSUSPENDED, SUSPENDED, SUSPENDEDWITHERROR, PREPPAUSED, PAUSED, PAUSEDWITHERROR, SUCCEEDED, DONEWITHERROR, KILLED, FAILED*.
+At any time, a coordinator job is in one of the following status: **PREP, RUNNING, RUNNINGWITHERROR, PREPSUSPENDED, SUSPENDED, SUSPENDEDWITHERROR, PREPPAUSED, PAUSED, PAUSEDWITHERROR, SUCCEEDED, DONEWITHERROR, KILLED, FAILED**.
Valid coordinator job status transitions are:
- * *PREP --> PREPSUSPENDED | PREPPAUSED | RUNNING | KILLED*
- * *RUNNING --> RUNNINGWITHERROR | SUSPENDED | PAUSED | SUCCEEDED | KILLED*
- * *RUNNINGWITHERROR --> RUNNING | SUSPENDEDWITHERROR | PAUSEDWITHERROR | DONEWITHERROR | KILLED | FAILED*
- * *PREPSUSPENDED --> PREP | KILLED*
- * *SUSPENDED --> RUNNING | KILLED*
- * *SUSPENDEDWITHERROR --> RUNNINGWITHERROR | KILLED*
- * *PREPPAUSED --> PREP | KILLED*
- * *PAUSED --> SUSPENDED | RUNNING | KILLED*
- * *PAUSEDWITHERROR --> SUSPENDEDWITHERROR | RUNNINGWITHERROR | KILLED*
- * *FAILED | KILLED --> IGNORED*
- * *IGNORED --> RUNNING*
+ * **PREP --> PREPSUSPENDED | PREPPAUSED | RUNNING | KILLED**
+ * **RUNNING --> RUNNINGWITHERROR | SUSPENDED | PAUSED | SUCCEEDED | KILLED**
+ * **RUNNINGWITHERROR --> RUNNING | SUSPENDEDWITHERROR | PAUSEDWITHERROR | DONEWITHERROR | KILLED | FAILED**
+ * **PREPSUSPENDED --> PREP | KILLED**
+ * **SUSPENDED --> RUNNING | KILLED**
+ * **SUSPENDEDWITHERROR --> RUNNINGWITHERROR | KILLED**
+ * **PREPPAUSED --> PREP | KILLED**
+ * **PAUSED --> SUSPENDED | RUNNING | KILLED**
+ * **PAUSEDWITHERROR --> SUSPENDEDWITHERROR | RUNNINGWITHERROR | KILLED**
+ * **FAILED | KILLED --> IGNORED**
+ * **IGNORED --> RUNNING**
-When a coordinator job is submitted, oozie parses the coordinator job XML. Oozie then creates a record for the coordinator with status *PREP* and returns a unique ID. The coordinator is also started immediately if pause time is not set.
+When a coordinator job is submitted, oozie parses the coordinator job XML. Oozie then creates a record for the coordinator with status **PREP** and returns a unique ID. The coordinator is also started immediately if pause time is not set.
-When a user requests to suspend a coordinator job that is in *PREP* state, oozie puts the job in status *PREPSUSPENDED*. Similarly, when pause time reaches for a coordinator job with *PREP* status, oozie puts the job in status *PREPPAUSED*.
+When a user requests to suspend a coordinator job that is in **PREP** state, oozie puts the job in status **PREPSUSPENDED**. Similarly, when pause time reaches for a coordinator job with **PREP** status, oozie puts the job in status **PREPPAUSED**.
-Conversely, when a user requests to resume a *PREPSUSPENDED* coordinator job, oozie puts the job in status *PREP*. And when pause time is reset for a coordinator job and job status is *PREPPAUSED*, oozie puts the job in status *PREP*.
+Conversely, when a user requests to resume a **PREPSUSPENDED** coordinator job, oozie puts the job in status **PREP**. And when pause time is reset for a coordinator job and job status is **PREPPAUSED**, oozie puts the job in status **PREP**.
-When a coordinator job starts, oozie puts the job in status *RUNNING* and start materializing workflow jobs based on job frequency. If any workflow job goes to *FAILED/KILLED/TIMEDOUT* state, the coordinator job is put in *RUNNINGWITHERROR*
+When a coordinator job starts, oozie puts the job in status **RUNNING** and start materializing workflow jobs based on job frequency. If any workflow job goes to **FAILED/KILLED/TIMEDOUT** state, the coordinator job is put in **RUNNINGWITHERROR**
-When a user requests to kill a coordinator job, oozie puts the job in status *KILLED* and it sends kill to all submitted workflow jobs.
+When a user requests to kill a coordinator job, oozie puts the job in status **KILLED** and it sends kill to all submitted workflow jobs.
-When a user requests to suspend a coordinator job that is in *RUNNING* status, oozie puts the job in status *SUSPENDED* and it suspends all submitted workflow jobs. Similarly, when a user requests to suspend a coordinator job that is in *RUNNINGWITHERROR* status, oozie puts the job in status *SUSPENDEDWITHERROR* and it suspends all submitted workflow jobs.
+When a user requests to suspend a coordinator job that is in **RUNNING** status, oozie puts the job in status **SUSPENDED** and it suspends all submitted workflow jobs. Similarly, when a user requests to suspend a coordinator job that is in **RUNNINGWITHERROR** status, oozie puts the job in status **SUSPENDEDWITHERROR** and it suspends all submitted workflow jobs.
-When pause time reaches for a coordinator job that is in *RUNNING* status, oozie puts the job in status *PAUSED*. Similarly, when pause time reaches for a coordinator job that is in *RUNNINGWITHERROR* status, oozie puts the job in status *PAUSEDWITHERROR*.
+When pause time reaches for a coordinator job that is in **RUNNING** status, oozie puts the job in status **PAUSED**. Similarly, when pause time reaches for a coordinator job that is in **RUNNINGWITHERROR** status, oozie puts the job in status **PAUSEDWITHERROR**.
-Conversely, when a user requests to resume a *SUSPENDED* coordinator job, oozie puts the job in status *RUNNING*. Also, when a user requests to resume a *SUSPENDEDWITHERROR* coordinator job, oozie puts the job in status *RUNNINGWITHERROR*. And when pause time is reset for a coordinator job and job status is *PAUSED*, oozie puts the job in status *RUNNING*. Also, when the pause time is reset for a coordinator job and job status is *PAUSEDWITHERROR*, oozie puts the job in status *RUNNINGWITHERROR*
+Conversely, when a user requests to resume a **SUSPENDED** coordinator job, oozie puts the job in status **RUNNING**. Also, when a user requests to resume a **SUSPENDEDWITHERROR** coordinator job, oozie puts the job in status **RUNNINGWITHERROR**. And when pause time is reset for a coordinator job and job status is **PAUSED**, oozie puts the job in status **RUNNING**. Also, when the pause time is reset for a coordinator job and job status is **PAUSEDWITHERROR**, oozie puts the job in status **RUNNINGWITHERROR**
-A coordinator job creates workflow jobs (commonly coordinator actions) only for the duration of the coordinator job and only if the coordinator job is in *RUNNING* status. If the coordinator job has been suspended, when resumed it will create all the coordinator actions that should have been created during the time it was suspended, actions will not be lost, they will delayed.
+A coordinator job creates workflow jobs (commonly coordinator actions) only for the duration of the coordinator job and only if the coordinator job is in **RUNNING** status. If the coordinator job has been suspended, when resumed it will create all the coordinator actions that should have been created during the time it was suspended, actions will not be lost, they will delayed.
When the coordinator job materialization finishes and all workflow jobs finish, oozie updates the coordinator status accordingly.
-For example, if all workflows are *SUCCEEDED*, oozie puts the coordinator job into *SUCCEEDED* status.
-If all workflows are *FAILED*, oozie puts the coordinator job into *FAILED* status. If all workflows are *KILLED*, the coordinator
-job status changes to KILLED. However, if any workflow job finishes with not *SUCCEEDED* and combination of *KILLED*, *FAILED* or
-*TIMEOUT*, oozie puts the coordinator job into *DONEWITHERROR*. If all coordinator actions are *TIMEDOUT*, oozie puts the
-coordinator job into *DONEWITHERROR*.
+For example, if all workflows are **SUCCEEDED**, oozie puts the coordinator job into **SUCCEEDED** status.
+If all workflows are **FAILED**, oozie puts the coordinator job into **FAILED** status. If all workflows are **KILLED**, the coordinator
+job status changes to KILLED. However, if any workflow job finishes with not **SUCCEEDED** and combination of **KILLED**, **FAILED** or
+**TIMEOUT**, oozie puts the coordinator job into **DONEWITHERROR**. If all coordinator actions are **TIMEDOUT**, oozie puts the
+coordinator job into **DONEWITHERROR**.
-A coordinator job in *FAILED* or *KILLED* status can be changed to *IGNORED* status. A coordinator job in *IGNORED* status can be changed to
- *RUNNING* status.
+A coordinator job in **FAILED** or **KILLED** status can be changed to **IGNORED** status. A coordinator job in **IGNORED** status can be changed to
+ **RUNNING** status.
----++++ 6.1.3. Coordinator Action
+#### 6.1.3. Coordinator Action
A coordinator job creates and executes coordinator actions.
@@ -935,169 +979,171 @@
Once an coordinator action is created (this is also referred as the action being materialized), the coordinator action will be in waiting until all required inputs for execution are satisfied or until the waiting times out.
----+++++ 6.1.3.1. Coordinator Action Creation (Materialization)
+##### 6.1.3.1. Coordinator Action Creation (Materialization)
A coordinator job has one driver event that determines the creation (materialization) of its coordinator actions (typically a workflow job).
* For synchronous coordinator jobs the driver event is the frequency of the coordinator job.
----+++++ 6.1.3.2. Coordinator Action Status
+##### 6.1.3.2. Coordinator Action Status
-Once a coordinator action has been created (materialized) the coordinator action qualifies for execution. At this point, the action status is *WAITING*.
+Once a coordinator action has been created (materialized) the coordinator action qualifies for execution. At this point, the action status is **WAITING**.
-A coordinator action in *WAITING* status must wait until all its input events are available before is ready for execution. When a coordinator action is ready for execution its status is *READY*.
+A coordinator action in **WAITING** status must wait until all its input events are available before is ready for execution. When a coordinator action is ready for execution its status is **READY**.
-A coordinator action in *WAITING* status may timeout before it becomes ready for execution. Then the action status is *TIMEDOUT*.
+A coordinator action in **WAITING** status may timeout before it becomes ready for execution. Then the action status is **TIMEDOUT**.
-A coordinator action may remain in *READY* status for a while, without starting execution, due to the concurrency execution policies of the coordinator job.
+A coordinator action may remain in **READY** status for a while, without starting execution, due to the concurrency execution policies of the coordinator job.
-A coordinator action in *READY* or *WAITING* status changes to *SKIPPED* status if the execution strategy is LAST_ONLY and the
+A coordinator action in **READY** or **WAITING** status changes to **SKIPPED** status if the execution strategy is LAST_ONLY and the
current time is past the next action's nominal time. See section 6.3 for more details.
-A coordinator action in *READY* or *WAITING* status changes to *SKIPPED* status if the execution strategy is NONE and the
+A coordinator action in **READY** or **WAITING** status changes to **SKIPPED** status if the execution strategy is NONE and the
current time is past the action's nominal time + 1 minute. See section 6.3 for more details.
-A coordinator action in *READY* status changes to *SUBMITTED* status if total current *RUNNING* and *SUBMITTED* actions are less than concurrency execution limit.
+A coordinator action in **READY** status changes to **SUBMITTED** status if total current **RUNNING** and **SUBMITTED** actions are less than concurrency execution limit.
-A coordinator action in *SUBMITTED* status changes to *RUNNING* status when the workflow engine start execution of the coordinator action.
+A coordinator action in **SUBMITTED** status changes to **RUNNING** status when the workflow engine start execution of the coordinator action.
-A coordinator action is in *RUNNING* status until the associated workflow job completes its execution. Depending on the workflow job completion status, the coordinator action will be in *SUCCEEDED*, *KILLED* or *FAILED* status.
+A coordinator action is in **RUNNING** status until the associated workflow job completes its execution. Depending on the workflow job completion status, the coordinator action will be in **SUCCEEDED**, **KILLED** or **FAILED** status.
-A coordinator action in *WAITING*, *READY*, *SUBMITTED* or *RUNNING* status can be killed, changing to *KILLED* status.
+A coordinator action in **WAITING**, **READY**, **SUBMITTED** or **RUNNING** status can be killed, changing to **KILLED** status.
-A coordinator action in *SUBMITTED* or *RUNNING* status can also fail, changing to *FAILED* status.
+A coordinator action in **SUBMITTED** or **RUNNING** status can also fail, changing to **FAILED** status.
-A coordinator action in *FAILED*, *KILLED*, or *TIMEDOUT* status can be changed to *IGNORED* status. A coordinator action in *IGNORED* status can be
- rerun, changing to *WAITING* status.
+A coordinator action in **FAILED**, **KILLED**, or **TIMEDOUT** status can be changed to **IGNORED** status. A coordinator action in **IGNORED** status can be
+ rerun, changing to **WAITING** status.
Valid coordinator action status transitions are:
- * *WAITING --> READY | TIMEDOUT | SKIPPED | KILLED*
- * *READY --> SUBMITTED | SKIPPED | KILLED*
- * *SUBMITTED --> RUNNING | KILLED | FAILED*
- * *RUNNING --> SUCCEEDED | KILLED | FAILED*
- * *FAILED | KILLED | TIMEDOUT --> IGNORED*
- * *IGNORED --> WAITING*
+ * **WAITING --> READY | TIMEDOUT | SKIPPED | KILLED**
+ * **READY --> SUBMITTED | SKIPPED | KILLED**
+ * **SUBMITTED --> RUNNING | KILLED | FAILED**
+ * **RUNNING --> SUCCEEDED | KILLED | FAILED**
+ * **FAILED | KILLED | TIMEDOUT --> IGNORED**
+ * **IGNORED --> WAITING**
----++++ 6.1.4. Input Events
+#### 6.1.4. Input Events
The Input events of a coordinator application specify the input conditions that are required in order to execute a coordinator action.
In the current specification input events are restricted to dataset instances availability.
-All the datasets instances defined as input events must be available for the coordinator action to be ready for execution ( *READY* status).
+All the datasets instances defined as input events must be available for the coordinator action to be ready for execution ( **READY** status).
Input events are normally parameterized. For example, the last 24 hourly instances of the 'searchlogs' dataset.
Input events can be refer to multiple instances of multiple datasets. For example, the last 24 hourly instances of the 'searchlogs' dataset and the last weekly instance of the 'celebrityRumours' dataset.
----++++ 6.1.5. Output Events
+#### 6.1.5. Output Events
A coordinator action can produce one or more dataset(s) instances as output.
Dataset instances produced as output by one coordinator actions may be consumed as input by another coordinator action(s) of other coordinator job(s).
-The chaining of coordinator jobs via the datasets they produce and consume is referred as a *data pipeline.*
+The chaining of coordinator jobs via the datasets they produce and consume is referred as a **data pipeline.**
In the current specification coordinator job output events are restricted to dataset instances.
----++++ 6.1.6. Coordinator Action Execution Policies
+#### 6.1.6. Coordinator Action Execution Policies
The execution policies for the actions of a coordinator job can be defined in the coordinator application.
* Timeout: A coordinator job can specify the timeout for its coordinator actions, this is, how long the coordinator action will be in *WAITING* or *READY* status before giving up on its execution.
- * Concurrency: A coordinator job can specify the concurrency for its coordinator actions, this is, how many coordinator actions are allowed to run concurrently ( *RUNNING* status) before the coordinator engine starts throttling them.
+ * Concurrency: A coordinator job can specify the concurrency for its coordinator actions, this is, how many coordinator actions are allowed to run concurrently ( **RUNNING** status) before the coordinator engine starts throttling them.
* Execution strategy: A coordinator job can specify the execution strategy of its coordinator actions when there is backlog of coordinator actions in the coordinator engine. The different execution strategies are 'oldest first', 'newest first', 'none' and 'last one only'. A backlog normally happens because of delayed input data, concurrency control or because manual re-runs of coordinator jobs.
* Throttle: A coordinator job can specify the materialization or creation throttle value for its coordinator actions, this is, how many maximum coordinator actions are allowed to be in WAITING state concurrently.
----++++ 6.1.7. Data Pipeline Application
+#### 6.1.7. Data Pipeline Application
Commonly, multiple workflow applications are chained together to form a more complex application.
-Workflow applications are run on regular basis, each of one of them at their own frequency. The data consumed and produced by these workflow applications is relative to the nominal time of workflow job that is processing the data. This is a *coordinator application*.
+Workflow applications are run on regular basis, each of one of them at their own frequency. The data consumed and produced by these workflow applications is relative to the nominal time of workflow job that is processing the data. This is a **coordinator application**.
-The output of multiple workflow jobs of a single workflow application is then consumed by a single workflow job of another workflow application, this is done on regular basis as well. These workflow jobs are triggered by recurrent actions of coordinator jobs. This is a set of *coordinator jobs* that inter-depend on each other via the data they produce and consume.
+The output of multiple workflow jobs of a single workflow application is then consumed by a single workflow job of another workflow application, this is done on regular basis as well. These workflow jobs are triggered by recurrent actions of coordinator jobs. This is a set of **coordinator jobs** that inter-depend on each other via the data they produce and consume.
-This set of interdependent *coordinator applications* is referred as a *data pipeline application*.
+This set of interdependent **coordinator applications** is referred as a **data pipeline application**.
----+++ 6.2. Synchronous Coordinator Application Example
+### 6.2. Synchronous Coordinator Application Example
- * The =checkouts= synchronous dataset is created every 15 minutes by an online checkout store.
- * The =hourlyRevenue= synchronous dataset is created every hour and contains the hourly revenue.
- * The =dailyRevenue= synchronous dataset is created every day and contains the daily revenue.
- * The =monthlyRevenue= synchronous dataset is created every month and contains the monthly revenue.
+ * The `checkouts` synchronous dataset is created every 15 minutes by an online checkout store.
+ * The `hourlyRevenue` synchronous dataset is created every hour and contains the hourly revenue.
+ * The `dailyRevenue` synchronous dataset is created every day and contains the daily revenue.
+ * The `monthlyRevenue` synchronous dataset is created every month and contains the monthly revenue.
- * The =revenueCalculator-wf= workflow consumes checkout data and produces as output the corresponding revenue.
- * The =rollUpRevenue-wf= workflow consumes revenue data and produces a consolidated output.
+ * The `revenueCalculator-wf` workflow consumes checkout data and produces as output the corresponding revenue.
+ * The `rollUpRevenue-wf` workflow consumes revenue data and produces a consolidated output.
- * The =hourlyRevenue-coord= coordinator job triggers, every hour, a =revenueCalculator-wf= workflow. It specifies as input the last 4 =checkouts= dataset instances and it specifies as output a new instance of the =hourlyRevenue= dataset.
- * The =dailyRollUpRevenue-coord= coordinator job triggers, every day, a =rollUpRevenue-wf= workflow. It specifies as input the last 24 =hourlyRevenue= dataset instances and it specifies as output a new instance of the =dailyRevenue= dataset.
- * The =monthlyRollUpRevenue-coord= coordinator job triggers, once a month, a =rollUpRevenue-wf= workflow. It specifies as input all the =dailyRevenue= dataset instance of the month and it specifies as output a new instance of the =monthlyRevenue= dataset.
+ * The `hourlyRevenue-coord` coordinator job triggers, every hour, a `revenueCalculator-wf` workflow. It specifies as input the last 4 `checkouts` dataset instances and it specifies as output a new instance of the `hourlyRevenue` dataset.
+ * The `dailyRollUpRevenue-coord` coordinator job triggers, every day, a `rollUpRevenue-wf` workflow. It specifies as input the last 24 `hourlyRevenue` dataset instances and it specifies as output a new instance of the `dailyRevenue` dataset.
+ * The `monthlyRollUpRevenue-coord` coordinator job triggers, once a month, a `rollUpRevenue-wf` workflow. It specifies as input all the `dailyRevenue` dataset instance of the month and it specifies as output a new instance of the `monthlyRevenue` dataset.
This example contains describes all the components that conform a data pipeline: datasets, coordinator jobs and coordinator actions (workflows).
The coordinator actions (the workflows) are completely agnostic of datasets and their frequencies, they just use them as input and output data (i.e. HDFS files or directories). Furthermore, as the example shows, the same workflow can be used to process similar datasets of different frequencies.
-The frequency of the =hourlyRevenue-coord= coordinator job is 1 hour, this means that every hour a coordinator action is created. A coordinator action will be executed only when the 4 =checkouts= dataset instances for the corresponding last hour are available, until then the coordinator action will remain as created (materialized), in *WAITING* status. Once the 4 dataset instances for the corresponding last hour are available, the coordinator action will be executed and it will start a =revenueCalculator-wf= workflow job.
+The frequency of the `hourlyRevenue-coord` coordinator job is 1 hour, this means that every hour a coordinator action is created. A coordinator action will be executed only when the 4 `checkouts` dataset instances for the corresponding last hour are available, until then the coordinator action will remain as created (materialized), in **WAITING** status. Once the 4 dataset instances for the corresponding last hour are available, the coordinator action will be executed and it will start a `revenueCalculator-wf` workflow job.
----+++ 6.3. Synchronous Coordinator Application Definition
+### 6.3. Synchronous Coordinator Application Definition
A synchronous coordinator definition is a is defined by a name, start time and end time, the frequency of creation of its coordinator actions, the input events, the output events and action control information:
- * *%BLUE% start: %ENDCOLOR%* The start datetime for the job. Starting at this time actions will be materialized. Refer to section #3 'Datetime Representation' for syntax details.
- * *%BLUE% end: %ENDCOLOR%* The end datetime for the job. When actions will stop being materialized. Refer to section #3 'Datetime Representation' for syntax details.
- * *%BLUE% timezone:%ENDCOLOR%* The timezone of the coordinator application.
- * *%BLUE% frequency: %ENDCOLOR%* The frequency, in minutes, to materialize actions. Refer to section #4 'Time Interval Representation' for syntax details.
+ * **<font color="#0000ff"> start: </font>** The start datetime for the job. Starting at this time actions will be materialized. Refer to section #3 'Datetime Representation' for syntax details.
+ * **<font color="#0000ff"> end: </font>** The end datetime for the job. When actions will stop being materialized. Refer to section #3 'Datetime Representation' for syntax details.
+ * **<font color="#0000ff"> timezone:</font>** The timezone of the coordinator application.
+ * **<font color="#0000ff"> frequency: </font>** The frequency, in minutes, to materialize actions. Refer to section #4 'Time Interval Representation' for syntax details.
* Control information:
- * *%BLUE% timeout: %ENDCOLOR%* The maximum time, in minutes, that a materialized action will be waiting for the additional conditions to be satisfied before being discarded. A timeout of =0= indicates that at the time of materialization all the other conditions must be satisfied, else the action will be discarded. A timeout of =0= indicates that if all the input events are not satisfied at the time of action materialization, the action should timeout immediately. A timeout of =-1= indicates no timeout, the materialized action will wait forever for the other conditions to be satisfied. The default value is =-1=. The timeout can only cause a =WAITING= action to transition to =TIMEDOUT=; once the data dependency is satisified, a =WAITING= action transitions to =READY=, and the timeout no longer has any affect, even if the action hasn't transitioned to =SUBMITTED= or =RUNNING= when it expires.
- * *%BLUE% concurrency: %ENDCOLOR%* The maximum number of actions for this job that can be running at the same time. This value allows to materialize and submit multiple instances of the coordinator app, and allows operations to catchup on delayed processing. The default value is =1=.
- * *%BLUE% execution: %ENDCOLOR%* Specifies the execution order if multiple instances of the coordinator job have satisfied their execution criteria. Valid values are:
- * =FIFO= (oldest first) *default*.
- * =LIFO= (newest first).
- * =LAST_ONLY= (see explanation below).
- * =NONE= (see explanation below).
- * *%BLUE% throttle: %ENDCOLOR%* The maximum coordinator actions are allowed to be in WAITING state concurrently. The default value is =12=.
- * *%BLUE% datasets: %ENDCOLOR%* The datasets coordinator application uses.
- * *%BLUE% input-events: %ENDCOLOR%* The coordinator job input events.
- * *%BLUE% data-in: %ENDCOLOR%* It defines one job input condition that resolves to one or more instances of a dataset.
- * *%BLUE% name: %ENDCOLOR%* input condition name.
- * *%BLUE% dataset: %ENDCOLOR%* dataset name.
- * *%BLUE% instance: %ENDCOLOR%* refers to a single dataset instance (the time for a synchronous dataset).
- * *%BLUE% start-instance: %ENDCOLOR%* refers to the beginning of an instance range (the time for a synchronous dataset).
- * *%BLUE% end-instance: %ENDCOLOR%* refers to the end of an instance range (the time for a synchronous dataset).
- * *%BLUE% output-events: %ENDCOLOR%* The coordinator job output events.
- * *%BLUE% data-out: %ENDCOLOR%* It defines one job output that resolves to a dataset instance.
- * *%BLUE% name: %ENDCOLOR%* output name.
- * *%BLUE% dataset: %ENDCOLOR%* dataset name.
- * *%BLUE% instance: %ENDCOLOR%* dataset instance that will be generated by coordinator action.
- * *%BLUE% nocleanup: %ENDCOLOR%* disable cleanup of the output dataset in rerun if true, even when nocleanup option is not used in CLI command.
- * *%BLUE% action: %ENDCOLOR%* The coordinator action to execute.
- * *%BLUE% workflow: %ENDCOLOR%* The workflow job invocation. Workflow job properties can refer to the defined data-in and data-out elements.
+ * **<font color="#0000ff"> timeout: </font>** The maximum time, in minutes, that a materialized action will be waiting for the additional conditions to be satisfied before being discarded. A timeout of `0` indicates that at the time of materialization all the other conditions must be satisfied, else the action will be discarded. A timeout of `0` indicates that if all the input events are not satisfied at the time of action materialization, the action should timeout immediately. A timeout of `-1` indicates no timeout, the materialized action will wait forever for the other conditions to be satisfied. The default value is `-1`. The timeout can only cause a `WAITING` action to transition to `TIMEDOUT`; once the data dependency is satisified, a `WAITING` action transitions to `READY`, and the timeout no longer has any affect, even if the action hasn't transitioned to `SUBMITTED` or `RUNNING` when it expires.
+ * **<font color="#0000ff"> concurrency: </font>** The maximum number of actions for this job that can be running at the same time. This value allows to materialize and submit multiple instances of the coordinator app, and allows operations to catchup on delayed processing. The default value is `1`.
+ * **<font color="#0000ff"> execution: </font>** Specifies the execution order if multiple instances of the coordinator job have satisfied their execution criteria. Valid values are:
+ ** `FIFO` (oldest first) **default*.
+ * `LIFO` (newest first).
+ * `LAST_ONLY` (see explanation below).
+ * `NONE` (see explanation below).
+ * **<font color="#0000ff"> throttle: </font>** The maximum coordinator actions are allowed to be in WAITING state concurrently. The default value is `12`.
+ * **<font color="#0000ff"> datasets: </font>** The datasets coordinator application uses.
+ * **<font color="#0000ff"> input-events: </font>** The coordinator job input events.
+ * **<font color="#0000ff"> data-in: </font>** It defines one job input condition that resolves to one or more instances of a dataset.
+ * **<font color="#0000ff"> name: </font>** input condition name.
+ * **<font color="#0000ff"> dataset: </font>** dataset name.
+ * **<font color="#0000ff"> instance: </font>** refers to a single dataset instance (the time for a synchronous dataset).
+ * **<font color="#0000ff"> start-instance: </font>** refers to the beginning of an instance range (the time for a synchronous dataset).
+ * **<font color="#0000ff"> end-instance: </font>** refers to the end of an instance range (the time for a synchronous dataset).
+ * **<font color="#0000ff"> output-events: </font>** The coordinator job output events.
+ * **<font color="#0000ff"> data-out: </font>** It defines one job output that resolves to a dataset instance.
+ * **<font color="#0000ff"> name: </font>** output name.
+ * **<font color="#0000ff"> dataset: </font>** dataset name.
+ * **<font color="#0000ff"> instance: </font>** dataset instance that will be generated by coordinator action.
+ * **<font color="#0000ff"> nocleanup: </font>** disable cleanup of the output dataset in rerun if true, even when nocleanup option is not used in CLI command.
+ * **<font color="#0000ff"> action: </font>** The coordinator action to execute.
+ * **<font color="#0000ff"> workflow: </font>** The workflow job invocation. Workflow job properties can refer to the defined data-in and data-out elements.
-*LAST_ONLY:* While =FIFO= and =LIFO= simply specify the order in which READY actions should be executed, =LAST_ONLY= can actually
-cause some actions to be SKIPPED and is a little harder to understand. When =LAST_ONLY= is set, an action that is =WAITING=
-or =READY= will be =SKIPPED= when the current time is past the next action's nominal time. For example, suppose action 1 and 2
-are both =READY=, the current time is 5:00pm, and action 2's nominal time is 5:10pm. In 10 minutes from now, at 5:10pm, action 1
-will become SKIPPED, assuming it doesn't transition to =SUBMITTED= (or a terminal state) before then. This sounds similar to the
+**LAST_ONLY:** While `FIFO` and `LIFO` simply specify the order in which READY actions should be executed, `LAST_ONLY` can actually
+cause some actions to be SKIPPED and is a little harder to understand. When `LAST_ONLY` is set, an action that is `WAITING`
+or `READY` will be `SKIPPED` when the current time is past the next action's nominal time. For example, suppose action 1 and 2
+are both `READY`, the current time is 5:00pm, and action 2's nominal time is 5:10pm. In 10 minutes from now, at 5:10pm, action 1
+will become SKIPPED, assuming it doesn't transition to `SUBMITTED` (or a terminal state) before then. This sounds similar to the
timeout control, but there are some important differences:
- * The timeout time is configurable while the =LAST_ONLY= time is effectively the frequency.
- * Reaching the timeout causes an action to transition to =TIMEDOUT=, which will cause the Coordinator Job to become =RUNNINGWITHERROR= and eventually =DONEWITHERROR=. With =LAST_ONLY=, an action becomes =SKIPPED= and the Coordinator Job remains =RUNNING= and eventually =DONE=.
- * The timeout is looking satisfying the data dependency, while =LAST_ONLY= is looking at the action itself. This means that the timeout can only cause a transition from =WAITING=, while =LAST_ONLY= can cause a transition from =WAITING= or =READY=.
-=LAST_ONLY= is useful if you want a recurring job, but do not actually care about the individual instances and just
+ * The timeout time is configurable while the `LAST_ONLY` time is effectively the frequency.
+ * Reaching the timeout causes an action to transition to `TIMEDOUT`, which will cause the Coordinator Job to become `RUNNINGWITHERROR` and eventually `DONEWITHERROR`. With `LAST_ONLY`, an action becomes `SKIPPED` and the Coordinator Job remains `RUNNING` and eventually `DONE`.
+ * The timeout is looking satisfying the data dependency, while `LAST_ONLY` is looking at the action itself. This means that the timeout can only cause a transition from `WAITING`, while `LAST_ONLY` can cause a transition from `WAITING` or `READY`.
+
+`LAST_ONLY` is useful if you want a recurring job, but do not actually care about the individual instances and just
always want the latest action. For example, if you have a coordinator running every 10 minutes and take Oozie down for 1 hour, when
-Oozie comes back, there would normally be 6 actions =WAITING= or =READY= to run. However, with =LAST_ONLY=, only the current one
-will go to =SUBMITTED= and then =RUNNING=; the others will go to =SKIPPED=.
+Oozie comes back, there would normally be 6 actions `WAITING` or `READY` to run. However, with `LAST_ONLY`, only the current one
+will go to `SUBMITTED` and then `RUNNING`; the others will go to `SKIPPED`.
-*NONE:* Similar to =LAST_ONLY= except instead of looking at the next action's nominal time, it looks
-at =oozie.coord.execution.none.tolerance= in oozie-site.xml (default is 1 minute). When =NONE= is set, an action that is =WAITING=
-or =READY= will be =SKIPPED= when the current time is more than the configured number of minutes (tolerance) past that action's
-nominal time. For example, suppose action 1 and 2 are both =READY=, the current time is 5:20pm, and both actions' nominal times are
-before 5:19pm. Both actions will become =SKIPPED=, assuming they don't transition to =SUBMITTED= (or a terminal state) before then.
+**NONE:** Similar to `LAST_ONLY` except instead of looking at the next action's nominal time, it looks
+at `oozie.coord.execution.none.tolerance` in oozie-site.xml (default is 1 minute). When `NONE` is set, an action that is `WAITING`
+or `READY` will be `SKIPPED` when the current time is more than the configured number of minutes (tolerance) past that action's
+nominal time. For example, suppose action 1 and 2 are both `READY`, the current time is 5:20pm, and both actions' nominal times are
+before 5:19pm. Both actions will become `SKIPPED`, assuming they don't transition to `SUBMITTED` (or a terminal state) before then.
-*%PURPLE% Syntax: %ENDCOLOR%*
+**<font color="#800080">Syntax: </font>**
-<verbatim>
+
+```
<coordinator-app name="[NAME]" frequency="[FREQUENCY]"
start="[DATETIME]" end="[DATETIME]" timezone="[TIMEZONE]"
xmlns="uri:oozie:coordinator:0.1">
@@ -1151,17 +1197,18 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-*1. A Coordinator Job that creates an executes a single coordinator action:*
+**1. A Coordinator Job that creates an executes a single coordinator action:**
The following example describes a synchronous coordinator application that runs once a day for 1 day at the end of the day. It consumes an instance of a daily 'logs' dataset and produces an instance of a daily 'siteAccessStats' dataset.
-*Coordinator application definition:*
+**Coordinator application definition:**
-<verbatim>
+
+```
<coordinator-app name="hello-coord" frequency="${coord:days(1)}"
start="2009-01-02T08:00Z" end="2009-01-02T08:00Z"
timezone="America/Los_Angeles"
@@ -1202,7 +1249,7 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
There are 2 synchronous datasets with a daily frequency and they are expected at the end of each PST8PDT day.
@@ -1210,7 +1257,8 @@
The workflow job invocation for the single coordinator action would resolve to:
-<verbatim>
+
+```
<workflow>
<app-path>hdfs://bar:8020/usr/joe/logsprocessor-wf</app-path>
<configuration>
@@ -1224,23 +1272,24 @@
</property>
</configuration>
</workflow>
-</verbatim>
+```
-IMPORTANT: Note Oozie works in UTC datetimes, all URI templates resolve to UTC datetime values. Because of the timezone difference between UTC and PST8PDT, the URIs resolves to =2009-01-02T08:00Z= (UTC) which is equivalent to 2009-01-01T24:00PST8PDT= (PST).
+IMPORTANT: Note Oozie works in UTC datetimes, all URI templates resolve to UTC datetime values. Because of the timezone difference between UTC and PST8PDT, the URIs resolves to `2009-01-02T08:00Z` (UTC) which is equivalent to 2009-01-01T24:00PST8PDT= (PST).
There is single input event, which resolves to January 1st PST8PDT instance of the 'logs' dataset. There is single output event, which resolves to January 1st PST8PDT instance of the 'siteAccessStats' dataset.
-The =${coord:dataIn(String name)}= and =${coord:dataOut(String name)}= EL functions resolve to the dataset instance URIs of the corresponding dataset instances. These EL functions are properly defined in a subsequent section.
+The `${coord:dataIn(String name)}` and `${coord:dataOut(String name)}` EL functions resolve to the dataset instance URIs of the corresponding dataset instances. These EL functions are properly defined in a subsequent section.
-Because the =${coord:dataIn(String name)}= and =${coord:dataOut(String name)}= EL functions resolve to URIs, which are HDFS URIs, the workflow job itself does not deal with dataset instances, just HDFS URIs.
+Because the `${coord:dataIn(String name)}` and `${coord:dataOut(String name)}` EL functions resolve to URIs, which are HDFS URIs, the workflow job itself does not deal with dataset instances, just HDFS URIs.
-*2. A Coordinator Job that executes its coordinator action multiple times:*
+**2. A Coordinator Job that executes its coordinator action multiple times:**
A more realistic version of the previous example would be a coordinator job that runs for a year creating a daily action an consuming the daily 'logs' dataset instance and producing the daily 'siteAccessStats' dataset instance.
The coordinator application is identical, except for the frequency, 'end' date and parameterization in the input and output events sections:
-<verbatim>
+
+```
<coordinator-app name="hello-coord" frequency="${coord:days(1)}"
start="2009-01-02T08:00Z" end="2010-01-02T08:00Z"
timezone="America/Los_Angeles"
@@ -1281,9 +1330,9 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
-The =${coord:current(int offset)}= EL function resolves to coordinator action creation time, that would be the current day at the time the coordinator action is created: =2009-01-02T08:00 ... 2010-01-01T08:00=. This EL function is properly defined in a subsequent section.
+The `${coord:current(int offset)}` EL function resolves to coordinator action creation time, that would be the current day at the time the coordinator action is created: `2009-01-02T08:00 ... 2010-01-01T08:00`. This EL function is properly defined in a subsequent section.
There is single input event, which resolves to the current day instance of the 'logs' dataset.
@@ -1291,7 +1340,8 @@
The workflow job invocation for the first coordinator action would resolve to:
-<verbatim>
+
+```
<workflow>
<app-path>hdfs://bar:8020/usr/joe/logsprocessor-wf</app-path>
<configuration>
@@ -1305,11 +1355,12 @@
</property>
</configuration>
</workflow>
-</verbatim>
+```
For the second coordinator action it would resolve to:
-<verbatim>
+
+```
<workflow>
<app-path>hdfs://bar:8020/usr/joe/logsprocessor-wf</app-path>
<configuration>
@@ -1323,11 +1374,11 @@
</property>
</configuration>
</workflow>
-</verbatim>
+```
And so on.
-*3. A Coordinator Job that executes its coordinator action multiple times and as input takes multiple dataset instances:*
+**3. A Coordinator Job that executes its coordinator action multiple times and as input takes multiple dataset instances:**
The following example is a variation of the example #2 where the synchronous coordinator application runs weekly. It consumes the of the last 7 instances of a daily 'logs' dataset and produces an instance of a weekly 'weeklySiteAccessStats' dataset.
@@ -1337,7 +1388,8 @@
The coordinator application frequency is weekly and it starts on the 7th day of the year:
-<verbatim>
+
+```
<coordinator-app name="hello2-coord" frequency="${coord:days(7)}"
start="2009-01-07T24:00Z" end="2009-12-12T24:00Z"
timezone="UTC"
@@ -1379,9 +1431,9 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
-The =${coord:current(int offset)}= EL function resolves to coordinator action creation time minus the specified offset multiplied by the dataset frequency. This EL function is properly defined in a subsequent section.
+The `${coord:current(int offset)}` EL function resolves to coordinator action creation time minus the specified offset multiplied by the dataset frequency. This EL function is properly defined in a subsequent section.
The input event, instead resolving to a single 'logs' dataset instance, it refers to a range of 7 dataset instances - the instance for 6 days ago, 5 days ago, ... and today's instance.
@@ -1389,7 +1441,8 @@
The workflow job invocation for the first coordinator action would resolve to:
-<verbatim>
+
+```
<workflow>
<app-path>hdfs://bar:8020/usr/joe/logsprocessor-wf</app-path>
<configuration>
@@ -1408,11 +1461,12 @@
</property>
</configuration>
</workflow>
-</verbatim>
+```
For the second coordinator action it would resolve to:
-<verbatim>
+
+```
<workflow>
<app-path>hdfs://bar:8020/usr/joe/logsprocessor-wf</app-path>
<configuration>
@@ -1431,20 +1485,20 @@
</property>
</configuration>
</workflow>
-</verbatim>
+```
And so on.
----+++ 6.4. Asynchronous Coordinator Application Definition
+### 6.4. Asynchronous Coordinator Application Definition
* TBD
----+++ 6.5. Parameterization of Coordinator Applications
+### 6.5. Parameterization of Coordinator Applications
When a coordinator job is submitted to Oozie, the submitter may specify as many coordinator job configuration properties as required (similar to Hadoop JobConf properties).
-Configuration properties that are a valid Java identifier, [A-Za-z_][0-9A-Za-z_]*, are available as =${NAME}= variables within the coordinator application definition.
+Configuration properties that are a valid Java identifier, [A-Za-z_][0-9A-Za-z_]*, are available as `${NAME}` variables within the coordinator application definition.
-Configuration Properties that are not a valid Java identifier, for example =job.tracker=, are available via the =${coord:conf(String name)}= function. Valid Java identifier properties are available via this function as well.
+Configuration Properties that are not a valid Java identifier, for example `job.tracker`, are available via the `${coord:conf(String name)}` function. Valid Java identifier properties are available via this function as well.
Using properties that are valid Java identifiers result in a more readable and compact definition.
@@ -1452,11 +1506,12 @@
If a configuration property used in the definitions is not provided with the job configuration used to submit a coordinator job, the value of the parameter will be undefined and the job submission will fail.
-*%GREEN% Example: %ENDCOLOR%*
+**<font color="#008000"> Example: </font>**
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="${jobStart}" end="${jobEnd}" timezone="${timezone}"
xmlns="uri:oozie:coordinator:0.1">
@@ -1480,29 +1535,30 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
In the above example there are 6 configuration parameters (variables) that have to be provided when submitting a job:
- * =jobStart= : start datetime for the job, in UTC
- * =jobEnd= : end datetime for the job, in UTC
- * =logsInitialInstance= : expected time of the first logs instance, in UTC
- * =timezone= : timezone for the job and the dataset
- * =market= : market to compute by this job, used in the uri-template
- * =language= : language to compute by this job, used in the uri-template
+ * `jobStart` : start datetime for the job, in UTC
+ * `jobEnd` : end datetime for the job, in UTC
+ * `logsInitialInstance` : expected time of the first logs instance, in UTC
+ * `timezone` : timezone for the job and the dataset
+ * `market` : market to compute by this job, used in the uri-template
+ * `language` : language to compute by this job, used in the uri-template
-IMPORTANT: Note that this example is not completely correct as it always consumes the last 24 instances of the 'logs' dataset. It is assumed that all days have 24 hours. For timezones that observe daylight saving this application will not work as expected as it will consume the wrong number of dataset instances in DST switch days. To be able to handle these scenarios, the =${coord:hoursInDays(int n)}= and =${coord:daysInMonths(int n)}= EL functions must be used (refer to section #6.6.2 and #6.6.3).
+IMPORTANT: Note that this example is not completely correct as it always consumes the last 24 instances of the 'logs' dataset. It is assumed that all days have 24 hours. For timezones that observe daylight saving this application will not work as expected as it will consume the wrong number of dataset instances in DST switch days. To be able to handle these scenarios, the `${coord:hoursInDays(int n)}` and `${coord:daysInMonths(int n)}` EL functions must be used (refer to section #6.6.2 and #6.6.3).
If the above 6 properties are not specified, the job will fail.
As of schema 0.4, a list of formal parameters can be provided which will allow Oozie to verify, at submission time, that said
properties are actually specified (i.e. before the job is executed and fails). Default values can also be provided.
-*Example:*
+**Example:**
The previous parameterized coordinator application definition with formal parameters:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="${jobStart}" end="${jobEnd}" timezone="${timezone}"
xmlns="uri:oozie:coordinator:0.1">
@@ -1535,310 +1591,322 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
-In the above example, if =jobStart= is not specified, Oozie will print an error message instead of submitting the job. If
-=jobEnd= is not specified, Oozie will use the default value, =2012-12-01T22:00Z=.
+In the above example, if `jobStart` is not specified, Oozie will print an error message instead of submitting the job. If
+`jobEnd` is not specified, Oozie will use the default value, `2012-12-01T22:00Z`.
----+++ 6.6. Parameterization of Dataset Instances in Input and Output Events
+### 6.6. Parameterization of Dataset Instances in Input and Output Events
A coordinator application job typically launches several coordinator actions during its lifetime. A coordinator action typically uses its creation (materialization) time to resolve the specific datasets instances required for its input and output events.
The following EL functions are the means for binding the coordinator action creation time to the datasets instances of its input and output events.
----++++ 6.6.1. coord:current(int n) EL Function for Synchronous Datasets
+#### 6.6.1. coord:current(int n) EL Function for Synchronous Datasets
-=${coord:current(int n)}= represents the n<sup>th</sup> dataset instance for a *synchronous* dataset, relative to the coordinator action creation (materialization) time. The coordinator action creation (materialization) time is computed based on the coordinator job start time and its frequency. The n<sup>th</sup> dataset instance is computed based on the dataset's initial-instance datetime, its frequency and the (current) coordinator action creation (materialization) time.
+`${coord:current(int n)}` represents the n<sup>th</sup> dataset instance for a **synchronous** dataset, relative to the coordinator action creation (materialization) time. The coordinator action creation (materialization) time is computed based on the coordinator job start time and its frequency. The n<sup>th</sup> dataset instance is computed based on the dataset's initial-instance datetime, its frequency and the (current) coordinator action creation (materialization) time.
-=n= can be a negative integer, zero or a positive integer.
+`n` can be a negative integer, zero or a positive integer.
-=${coord:current(int n)}= returns the nominal datetime for n<sup>th</sup> dataset instance relative to the coordinator action creation (materialization) time.
+`${coord:current(int n)}` returns the nominal datetime for n<sup>th</sup> dataset instance relative to the coordinator action creation (materialization) time.
-=${coord:current(int n)}= performs the following calculation:
+`${coord:current(int n)}` performs the following calculation:
-<verbatim>
+
+```
DS_II : dataset initial-instance (datetime)
DS_FREQ: dataset frequency (minutes)
CA_NT: coordinator action creation (materialization) nominal time
coord:current(int n) = DS_II + DS_FREQ * ( (CA_NT - DS_II) div DS_FREQ + n)
-</verbatim>
+```
NOTE: The formula above is not 100% correct, because DST changes the calculation has to account for hour shifts. Oozie Coordinator must make the correct calculation accounting for DST hour shifts.
-When a positive integer is used with the =${coord:current(int n)}=, it refers to a dataset instance in the future from the coordinator action creation (materialization) time. This can be useful when creating dataset instances for future use by other systems.
+When a positive integer is used with the `${coord:current(int n)}`, it refers to a dataset instance in the future from the coordinator action creation (materialization) time. This can be useful when creating dataset instances for future use by other systems.
-The datetime returned by =${coord:current(int n)}= returns the exact datetime for the computed dataset instance.
+The datetime returned by `${coord:current(int n)}` returns the exact datetime for the computed dataset instance.
-*IMPORTANT:* The coordinator engine does use output events to keep track of new dataset instances. Workflow jobs triggered from coordinator actions can leverage the coordinator engine capability to synthesize dataset instances URIs to create output directories.
+**IMPORTANT:** The coordinator engine does use output events to keep track of new dataset instances. Workflow jobs triggered from coordinator actions can leverage the coordinator engine capability to synthesize dataset instances URIs to create output directories.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-1. *=${coord:current(int n)}= datetime calculation:*
+1. **`${coord:current(int n)}` datetime calculation:**
-Datasets Definition:
+ Datasets Definition:
-<verbatim>
-<datasets>
-.
- <dataset name="logs" frequency="${coord:days(1)}"
- initial-instance="2009-01-01T24:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}</uri-template>
- </dataset>
-.
- <dataset name="weeklySiteAccessStats" frequency="${coord:days(7)}"
- initial-instance="2009-01-07T24:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/weeklystats/${YEAR}/${MONTH}/${DAY}</uri-template>
- </dataset>
-.
-</datasets>
-</verbatim>
-For a coordinator action creation time: =2009-05-29T24:00Z= the =${coord:current(int n)}= EL function would resolve to the following datetime values for the 'logs' and 'weeklySiteStats' datasets:
+ ```
+ <datasets>
+ .
+ <dataset name="logs" frequency="${coord:days(1)}"
+ initial-instance="2009-01-01T24:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}</uri-template>
+ </dataset>
+ .
+ <dataset name="weeklySiteAccessStats" frequency="${coord:days(7)}"
+ initial-instance="2009-01-07T24:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/weeklystats/${YEAR}/${MONTH}/${DAY}</uri-template>
+ </dataset>
+ .
+ </datasets>
+ ```
-| *${coord:current(int offset)}* | *Dataset 'logs'* | *Dataset 'weeklySiteAccessStats'* |
-| =${coord:current(0)}= | =2009-05-29T24:00Z= | =2009-05-27T24:00Z= |
-| =${coord:current(1)}= | =2009-05-30T24:00Z= | =2009-06-03T24:00Z= |
-| =${coord:current(-1)}= | =2009-05-28T24:00Z= | =2009-05-20T24:00Z= |
-| =${coord:current(-3)}= | =2009-05-26T24:00Z= | =2009-05-06T24:00Z= |
+ For a coordinator action creation time: `2009-05-29T24:00Z` the `${coord:current(int n)}` EL function would resolve to the following datetime values for the 'logs' and 'weeklySiteStats' datasets:
-Note, in the example above, how the datetimes resolved for the 2 datasets differ when the =${coord:current(int n)}= function is invoked with the same argument. This is because the =${coord:current(int n)}= function takes into consideration the initial-time and the frequency for the dataset for which is performing the calculation.
+ | **${coord:current(int offset)}** | **Dataset 'logs'** | **Dataset 'weeklySiteAccessStats'** |
+| --- | --- | --- |
+ | `${coord:current(0)}` | `2009-05-29T24:00Z` | `2009-05-27T24:00Z` |
+ | `${coord:current(1)}` | `2009-05-30T24:00Z` | `2009-06-03T24:00Z` |
+ | `${coord:current(-1)}` | `2009-05-28T24:00Z` | `2009-05-20T24:00Z` |
+ | `${coord:current(-3)}` | `2009-05-26T24:00Z` | `2009-05-06T24:00Z` |
-Datasets Definition file 'datasets.xml':
+ Note, in the example above, how the datetimes resolved for the 2 datasets differ when the `${coord:current(int n)}` function is invoked with the same argument. This is because the `${coord:current(int n)}` function takes into consideration the initial-time and the frequency for the dataset for which is performing the calculation.
-<verbatim>
-<datasets>
+ Datasets Definition file 'datasets.xml':
- <dataset name="logs" frequency="${coord:hours(1)}"
- initial-instance="2009-01-01T01:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}/${HOUR}</uri-template>
- </dataset>
-</datasets>
-</verbatim>
+ ```
+ <datasets>
-a. Coordinator application definition that creates a coordinator action once a day for a year, that is 365 coordinator actions:
+ <dataset name="logs" frequency="${coord:hours(1)}"
+ initial-instance="2009-01-01T01:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}/${HOUR}</uri-template>
+ </dataset>
-<verbatim>
- <coordinator-app name="app-coord" frequency="${coord:days(1)}"
- start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
- xmlns="uri:oozie:coordinator:0.1">
- <datasets>
- <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
- </datasets>
- <input-events>
- <data-in name="input" dataset="logs">
- <start-instance>${coord:current(-23)}</start-instance>
- <end-instance>${coord:current(0)}</end-instance>
- </data-in>
- </input-events>
- <action>
- <workflow>
- ...
- </workflow>
- </action>
- </coordinator-app>
-</verbatim>
+ </datasets>
+ ```
-Each coordinator action will require as input events the last 24 (-23 to 0) dataset instances for the 'logs' dataset. Because the dataset 'logs' is a hourly dataset, it means all its instances for the last 24 hours.
+ a. Coordinator application definition that creates a coordinator action once a day for a year, that is 365 coordinator actions:
-In this case, the dataset instances are used in a rolling window fashion.
-b. Coordinator application definition that creates a coordinator action once an hour for a year, that is 8760 (24*8760) coordinator actions:
+ ```
+ <coordinator-app name="app-coord" frequency="${coord:days(1)}"
+ start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
+ xmlns="uri:oozie:coordinator:0.1">
+ <datasets>
+ <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
+ </datasets>
+ <input-events>
+ <data-in name="input" dataset="logs">
+ <start-instance>${coord:current(-23)}</start-instance>
+ <end-instance>${coord:current(0)}</end-instance>
+ </data-in>
+ </input-events>
+ <action>
+ <workflow>
+ ...
+ </workflow>
+ </action>
+ </coordinator-app>
+ ```
-<verbatim>
- <coordinator-app name="app-coord" frequency="${coord:hours(1)}"
- start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
- xmlns="uri:oozie:coordinator:0.1">
- <datasets>
- <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
- </datasets>
- <input-events>
- <data-in name="input" dataset="logs">
- <start-instance>${coord:current(-23)}</start-instance>
- <end-instance>${coord:current(0)}</end-instance>
- </data-in>
- </input-events>
- <action>
- <workflow>
- ...
- </workflow>
- </action>
- </coordinator-app>
-</verbatim>
+ Each coordinator action will require as input events the last 24 (-23 to 0) dataset instances for the 'logs' dataset. Because the dataset 'logs' is a hourly dataset, it means all its instances for the last 24 hours.
-Each coordinator action will require as input events the last 24 (-23 to 0) dataset instances for the 'logs' dataset. Similarly to the previous coordinator application example, it means all its instances for the last 24 hours.
+ In this case, the dataset instances are used in a rolling window fashion.
-However, because the frequency is hourly instead of daily, each coordinator action will use the last 23 dataset instances used by the previous coordinator action plus a new one.
+ b. Coordinator application definition that creates a coordinator action once an hour for a year, that is 8760 (24*8760) coordinator actions:
-In this case, the dataset instances are used in a sliding window fashion.
-3. *Using =${coord:current(int n)}= to specify dataset instances created by a coordinator application:*
+ ```
+ <coordinator-app name="app-coord" frequency="${coord:hours(1)}"
+ start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
+ xmlns="uri:oozie:coordinator:0.1">
+ <datasets>
+ <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
+ </datasets>
+ <input-events>
+ <data-in name="input" dataset="logs">
+ <start-instance>${coord:current(-23)}</start-instance>
+ <end-instance>${coord:current(0)}</end-instance>
+ </data-in>
+ </input-events>
+ <action>
+ <workflow>
+ ...
+ </workflow>
+ </action>
+ </coordinator-app>
+ ```
-Datasets Definition file 'datasets.xml':
+ Each coordinator action will require as input events the last 24 (-23 to 0) dataset instances for the 'logs' dataset. Similarly to the previous coordinator application example, it means all its instances for the last 24 hours.
-<verbatim>
-<datasets>
-.
- <dataset name="logs" frequency="${coord:hours(1)}"
- initial-instance="2009-01-01T01:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}</uri-template>
- </dataset>
-.
- <dataset name="stats" frequency="${coord:days(1)}"
- initial-instance="2009-01-01T24:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}</uri-template>
- </dataset>
-.
-</datasets>
-</verbatim>
+ However, because the frequency is hourly instead of daily, each coordinator action will use the last 23 dataset instances used by the previous coordinator action plus a new one.
-Coordinator application definition:
+ In this case, the dataset instances are used in a sliding window fashion.
-<verbatim>
- <coordinator-app name="app-coord" frequency="${coord:days(1)}"
- start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
- xmlns="uri:oozie:coordinator:0.1">
- <datasets>
- <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
- </datasets>
- <input-events>
- <data-in name="input" dataset="logs">
- <start-instance>${coord:current(-23)}</start-instance>
- <end-instance>${coord:current(0)}</end-instance>
- </data-in>
- </input-events>
- <output-events>
- <data-out name="output" dataset="stats">
- <instance>${coord:current(0)}</instance>
- </data-out>
- </output-events>
- <action>
- <workflow>
- ...
- </workflow>
- </action>
- </coordinator-app>
-</verbatim>
+3. **Using `${coord:current(int n)}` to specify dataset instances created by a coordinator application:**
-This coordinator application creates a coordinator action once a day for a year, this is 365 coordinator actions.
+ Datasets Definition file 'datasets.xml':
-Each coordinator action will require as input events the last 24 (-23 to 0) dataset instances for the 'logs' dataset.
-Each coordinator action will create as output event a new dataset instance for the 'stats' dataset.
+ ```
+ <datasets>
+ .
+ <dataset name="logs" frequency="${coord:hours(1)}"
+ initial-instance="2009-01-01T01:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}</uri-template>
+ </dataset>
+ .
+ <dataset name="stats" frequency="${coord:days(1)}"
+ initial-instance="2009-01-01T24:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}</uri-template>
+ </dataset>
+ .
+ </datasets>
+ ```
-Note that the 'stats' dataset initial-instance and frequency match the coordinator application start and frequency.
+ Coordinator application definition:
-4. *Using =${coord:current(int n)}= to create a data-pipeline using a coordinator application:*
-This example shows how to chain together coordinator applications to create a data pipeline.
+ ```
+ <coordinator-app name="app-coord" frequency="${coord:days(1)}"
+ start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
+ xmlns="uri:oozie:coordinator:0.1">
+ <datasets>
+ <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
+ </datasets>
+ <input-events>
+ <data-in name="input" dataset="logs">
+ <start-instance>${coord:current(-23)}</start-instance>
+ <end-instance>${coord:current(0)}</end-instance>
+ </data-in>
+ </input-events>
+ <output-events>
+ <data-out name="output" dataset="stats">
+ <instance>${coord:current(0)}</instance>
+ </data-out>
+ </output-events>
+ <action>
+ <workflow>
+ ...
+ </workflow>
+ </action>
+ </coordinator-app>
+ ```
-Dataset definitions file 'datasets.xml':
+ This coordinator application creates a coordinator action once a day for a year, this is 365 coordinator actions.
-<verbatim>
- <!--- Dataset A - produced every 15 minutes. -->
-.
- <dataset name="15MinLogs" frequency="${coord:minutes(15)}"
- initial-instance="2009-01-01T00:15:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}</uri-template>
- </dataset>
-.
- <dataset name="1HourLogs" frequency="${coord:hours(1)}"
- initial-instance="2009-01-01T01:00:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}</uri-template>
- </dataset>
-.
- <dataset name="1DayLogs" frequency="${coord:hours(24)}"
- initial-instance="2009-01-01T24:00:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}</uri-template>
- </dataset>
-</verbatim>
+ Each coordinator action will require as input events the last 24 (-23 to 0) dataset instances for the 'logs' dataset.
-Coordinator application definitions. A data-pipeline with two coordinator-applications, one scheduled to run every hour, and another scheduled to run every day:
+ Each coordinator action will create as output event a new dataset instance for the 'stats' dataset.
-<verbatim>
- <coordinator-app name="app-coord-hourly" frequency="${coord:hours(1)}"
- start="2009-01-01T01:00Z" end="2009-12-31T24:00Z" timezone="UTC"
- xmlns="uri:oozie:coordinator:0.1">
- <datasets>
- <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
- </datasets>
- <input-events>
- <data-in name="input" dataset="15MinLogs">
- <start-instance>${coord:current(-3)}</start-instance>
- <end-instance>${coord:current(0)}</end-instance>
- </data-in>
- </input-events>
- <output-events>
- <data-out name="output" dataset="1HourLogs">
- <instance>${coord:current(0)}</instance>
- </data-out>
- </output-events>
- <action>
- <workflow>
- ...
- </workflow>
- </action>
- </coordinator-app>
-</verbatim>
+ Note that the 'stats' dataset initial-instance and frequency match the coordinator application start and frequency.
-<verbatim>
- <coordinator-app name="app-coord-daily" frequency="${coord:days(1)}"
- start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
- xmlns="uri:oozie:coordinator:0.1">
- <datasets>
- <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
- </datasets>
- <input-events>
- <data-in name="input" dataset="1HourLogs">
- <start-instance>${coord:current(-23)}</start-instance>
- <end-instance>${coord:current(0)}</end-instance>
- </data-in>
- </input-events>
- <output-events>
- <data-out name="output" dataset="1DayLogs">
- <instance>${coord:current(0)}</instance>
- </data-out>
- </output-events>
- <action>
- <workflow>
- ...
- </workflow>
- </action>
- </coordinator-app>
-</verbatim>
+4. **Using `${coord:current(int n)}` to create a data-pipeline using a coordinator application:**
-The 'app-coord-hourly' coordinator application runs every every hour, uses 4 instances of the dataset "15MinLogs" to create one instance of the dataset "1HourLogs"
+ This example shows how to chain together coordinator applications to create a data pipeline.
-The 'app-coord-daily' coordinator application runs every every day, uses 24 instances of "1HourLogs" to create one instance of "1DayLogs"
+ Dataset definitions file 'datasets.xml':
-The output datasets from the 'app-coord-hourly' coordinator application are the input to the 'app-coord-daily' coordinator application thereby forming a simple data-pipeline application.
----++++ 6.6.2. coord:offset(int n, String timeUnit) EL Function for Synchronous Datasets
+ ```
+ <!--- Dataset A - produced every 15 minutes. -->
+ .
+ <dataset name="15MinLogs" frequency="${coord:minutes(15)}"
+ initial-instance="2009-01-01T00:15:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}</uri-template>
+ </dataset>
+ .
+ <dataset name="1HourLogs" frequency="${coord:hours(1)}"
+ initial-instance="2009-01-01T01:00:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}</uri-template>
+ </dataset>
+ .
+ <dataset name="1DayLogs" frequency="${coord:hours(24)}"
+ initial-instance="2009-01-01T24:00:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}</uri-template>
+ </dataset>
+ ```
-=${coord:offset(int n, String timeUnit)}= represents the n<sup>th</sup> timeUnit, relative to the coordinator action creation
+ Coordinator application definitions. A data-pipeline with two coordinator-applications, one scheduled to run every hour, and another scheduled to run every day:
+
+
+ ```
+ <coordinator-app name="app-coord-hourly" frequency="${coord:hours(1)}"
+ start="2009-01-01T01:00Z" end="2009-12-31T24:00Z" timezone="UTC"
+ xmlns="uri:oozie:coordinator:0.1">
+ <datasets>
+ <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
+ </datasets>
+ <input-events>
+ <data-in name="input" dataset="15MinLogs">
+ <start-instance>${coord:current(-3)}</start-instance>
+ <end-instance>${coord:current(0)}</end-instance>
+ </data-in>
+ </input-events>
+ <output-events>
+ <data-out name="output" dataset="1HourLogs">
+ <instance>${coord:current(0)}</instance>
+ </data-out>
+ </output-events>
+ <action>
+ <workflow>
+ ...
+ </workflow>
+ </action>
+ </coordinator-app>
+ ```
+
+
+ ```
+ <coordinator-app name="app-coord-daily" frequency="${coord:days(1)}"
+ start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
+ xmlns="uri:oozie:coordinator:0.1">
+ <datasets>
+ <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
+ </datasets>
+ <input-events>
+ <data-in name="input" dataset="1HourLogs">
+ <start-instance>${coord:current(-23)}</start-instance>
+ <end-instance>${coord:current(0)}</end-instance>
+ </data-in>
+ </input-events>
+ <output-events>
+ <data-out name="output" dataset="1DayLogs">
+ <instance>${coord:current(0)}</instance>
+ </data-out>
+ </output-events>
+ <action>
+ <workflow>
+ ...
+ </workflow>
+ </action>
+ </coordinator-app>
+ ```
+
+ The 'app-coord-hourly' coordinator application runs every every hour, uses 4 instances of the dataset "15MinLogs" to create one instance of the dataset "1HourLogs"
+
+ The 'app-coord-daily' coordinator application runs every every day, uses 24 instances of "1HourLogs" to create one instance of "1DayLogs"
+
+ The output datasets from the 'app-coord-hourly' coordinator application are the input to the 'app-coord-daily' coordinator application thereby forming a simple data-pipeline application.
+
+#### 6.6.2. coord:offset(int n, String timeUnit) EL Function for Synchronous Datasets
+
+`${coord:offset(int n, String timeUnit)}` represents the n<sup>th</sup> timeUnit, relative to the coordinator action creation
(materialization) time. The coordinator action creation (materialization) time is computed based on the coordinator job start time
and its frequency.
-It is an alternative to the =${coord:current(int n)}= command (see previous section) and can be used anywhere
-=${coord:current(int n)}= can be used. The difference between the two functions is that =${coord:current(int n)}= computes an offset
-based on the n<sup>th</sup> multiple of the frequency, while =${coord:offset(int n, String timeUnit)}= computes an offset based on
-the n<sup>th</sup> multiple of =timeUnit=.
+It is an alternative to the `${coord:current(int n)}` command (see previous section) and can be used anywhere
+`${coord:current(int n)}` can be used. The difference between the two functions is that `${coord:current(int n)}` computes an offset
+based on the n<sup>th</sup> multiple of the frequency, while `${coord:offset(int n, String timeUnit)}` computes an offset based on
+the n<sup>th</sup> multiple of `timeUnit`.
-=n= can be a negative integer, zero or a positive integer.
+`n` can be a negative integer, zero or a positive integer.
-=timeUnit= can be any one of the following constants: ="MINUTE"=, ="HOUR"=, ="DAY"=, ="MONTH"=, ="YEAR"=
+`timeUnit` can be any one of the following constants: `"MINUTE"`, `"HOUR"`, `"DAY"`, `"MONTH"`, `"YEAR"`
-=${coord:offset(int n, String timeUnit)}= returns the nominal datetime for n<sup>th</sup> timeUnit relative to the coordinator
+`${coord:offset(int n, String timeUnit)}` returns the nominal datetime for n<sup>th</sup> timeUnit relative to the coordinator
action creation (materialization) time.
-When used directly, =${coord:offset(int n, String timeUnit)}= performs the following calculation:
+When used directly, `${coord:offset(int n, String timeUnit)}` performs the following calculation:
-<verbatim>
+
+```
DS_FREQ: dataset frequency (minutes)
CA_NT = coordinator action creation (materialization) nominal time
-coord:offset(int n, String timeUnit) = CA_NT + floor(timeUnit * n div DS_FREQ) * DS_FREQ
-</verbatim>
+coord:offset(int n, String timeUnit) = CA_NT + floor(timeUnit ** n div DS_FREQ) ** DS_FREQ
+```
NOTE: The formula above is not 100% correct, because DST changes the calculation has to account for hour shifts. Oozie Coordinator
must make the correct calculation accounting for DST hour shifts.
@@ -1849,131 +1917,138 @@
resolved datetime is "fastforwarded" to match the earliest instance after the resolved time.
See the next two examples for more information.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-1. *=${coord:offset(int n, String timeUnit)}= datetime calculation:*
+1. **`${coord:offset(int n, String timeUnit)}` datetime calculation:**
-Datasets Definition:
+ Datasets Definition:
-<verbatim>
-<datasets>
-.
- <dataset name="logs" frequency="${coord:days(1)}"
- initial-instance="2009-01-01T24:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}</uri-template>
- </dataset>
-.
- <dataset name="weeklySiteAccessStats" frequency="${coord:days(7)}"
- initial-instance="2009-01-07T24:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/weeklystats/${YEAR}/${MONTH}/${DAY}</uri-template>
- </dataset>
-.
-</datasets>
-</verbatim>
-For a coordinator action creation time: =2009-05-29T24:00Z= the =${coord:offset(int n, String timeUnit)}= EL function would resolve
-to the following datetime values for the 'logs' and 'weeklySiteStats' datasets:
+ ```
+ <datasets>
+ .
+ <dataset name="logs" frequency="${coord:days(1)}"
+ initial-instance="2009-01-01T24:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}</uri-template>
+ </dataset>
+ .
+ <dataset name="weeklySiteAccessStats" frequency="${coord:days(7)}"
+ initial-instance="2009-01-07T24:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/weeklystats/${YEAR}/${MONTH}/${DAY}</uri-template>
+ </dataset>
+ .
+ </datasets>
+ ```
-| *${coord:offset(int n, String timeUnit)}* | *Dataset 'logs'* | *Dataset 'weeklySiteAccessStats'* |
-| =${coord:offset(0, "MINUTE")}= <br/> =${coord:offset(0, "HOUR")}= <br/> =${coord:offset(0, "DAY")}= <br/> =${coord:offset(0, "MONTH")}= <br/> =${coord:offset(0, "YEAR")}= | =2009-05-29T24:00Z= | =2009-05-27T24:00Z= |
-| =${coord:offset(1440, "MINUTE")}= <br/> =${coord:offset(24, "HOUR")}= <br/> =${coord:offset(1, "DAY")}= | =2009-05-30T24:00Z= | =2009-05-27T24:00Z= |
-| =${coord:offset(-1440, "MINUTE")}= <br/> =${coord:offset(-24, "HOUR")}= <br/> =${coord:offset(-1, "DAY")}= | =2009-05-28T24:00Z= | =2009-05-20T24:00Z= |
-| =${coord:offset(-4320, "MINUTE")}= <br/> =${coord:offset(-72, "HOUR")}= <br/> =${coord:offset(-3, "DAY")}= | =2009-05-26T24:00Z= | =2009-05-20T24:00Z= |
-| =${coord:offset(11520, "MINUTE")}= <br/> =${coord:offset(192, "HOUR")}= <br/> =${coord:offset(8, "DAY")}= | =2009-06-06T24:00Z= | =2009-06-03T24:00Z= |
-| =${coord:offset(10, "MINUTE")}= | =2009-05-29T24:00Z= | =2009-05-27T24:00Z= |
+ For a coordinator action creation time: `2009-05-29T24:00Z` the `${coord:offset(int n, String timeUnit)}` EL function would resolve
+ to the following datetime values for the 'logs' and 'weeklySiteStats' datasets:
-Some things to note about the above example:
- 1. When =n= is 0, the =timeUnit= doesn't really matter because 0 minutes is the same as 0 hours, 0 days, etc
- 2. There are multiple ways to express the same value (e.g. ${coord:offset(24, "HOUR")}= is equivalent to =${coord:offset(1, "DAY")}=)
- 3. The datetimes resolved for the 2 datasets differ when the =${coord:offset(int n, String timeUnit)}= function is invoked with the same arguments. This is because the =${coord:offset(int n, String timeUnit)}= function takes into consideration the initial-time and the frequency for the dataset for which is performing the calculation.
- 4. As mentioned before, if the resolved time doesn't fall exactly on an instance, it will get "rewinded" to match the latest instance before the resolved time. For example, =${coord:offset(1, "DAY")}= is resolved to =2009-05-27T24:00Z= for the 'weeklysiteStats' dataset even though this is the same as =${coord:offset(0, "DAY")}=; this is because the frequency is 7 days, so =${coord:offset(1, "DAY")}= had to be "rewinded".
+ | **${coord:offset(int n, String timeUnit)}** | **Dataset 'logs'** | **Dataset 'weeklySiteAccessStats'** |
+| --- | --- | --- |
+ | `${coord:offset(0, "MINUTE")}` <br/> `${coord:offset(0, "HOUR")}` <br/> `${coord:offset(0, "DAY")}` <br/> `${coord:offset(0, "MONTH")}` <br/> `${coord:offset(0, "YEAR")}` | `2009-05-29T24:00Z` | `2009-05-27T24:00Z` |
+ | `${coord:offset(1440, "MINUTE")}` <br/> `${coord:offset(24, "HOUR")}` <br/> `${coord:offset(1, "DAY")}` | `2009-05-30T24:00Z` | `2009-05-27T24:00Z` |
+ | `${coord:offset(-1440, "MINUTE")}` <br/> `${coord:offset(-24, "HOUR")}` <br/> `${coord:offset(-1, "DAY")}` | `2009-05-28T24:00Z` | `2009-05-20T24:00Z` |
+ | `${coord:offset(-4320, "MINUTE")}` <br/> `${coord:offset(-72, "HOUR")}` <br/> `${coord:offset(-3, "DAY")}` | `2009-05-26T24:00Z` | `2009-05-20T24:00Z` |
+ | `${coord:offset(11520, "MINUTE")}` <br/> `${coord:offset(192, "HOUR")}` <br/> `${coord:offset(8, "DAY")}` | `2009-06-06T24:00Z` | `2009-06-03T24:00Z` |
+ | `${coord:offset(10, "MINUTE")}` | `2009-05-29T24:00Z` | `2009-05-27T24:00Z` |
-2. *"fastforwarding" in <start-instance> =${coord:offset(int n, String timeUnit)}= calculation:*
+ Some things to note about the above example:
-When specifying dataset instances, keep in mind that the resolved value of =${coord:offset(int n, String timeUnit)}= must line up
-with an offset of a multiple of the frequency when used in an 'instance' XML element.
-However, when used in ='start-instance'= and ='end-instance'= XML elements, this is not a requirement. In this case, the function
-will automatically resolve the range of instances to match the offset of a multiple of the frequency that would fall between the
-='start-instance'= and ='end-instance'= XML elements; in other words, ='start-instance'= XML element is "fastforwarded" while
-='end-instance'= XML element is "rewinded". So, in the example below, the frequency of the "logs" dataset is 1 hour while the
-='start-instance'= XML element is =${coord:offset(-90, "MINUTE")}= (-1.5 hours). If this were in an ='instance'= XML element, it
-would be "rewinded", but here it is effectively equivalent to =${coord:offset(-60, "MINUTE")}= or =${coord:current(-1)}= as we are
-dealing with a range.
+ 1. When `n` is 0, the `timeUnit` doesn't really matter because 0 minutes is the same as 0 hours, 0 days, etc
+ 2. There are multiple ways to express the same value (e.g. `${coord:offset(24, "HOUR")}` is equivalent to `${coord:offset(1, "DAY")}`)
+ 3. The datetimes resolved for the 2 datasets differ when the `${coord:offset(int n, String timeUnit)}` function is invoked with the same arguments. This is because the `${coord:offset(int n, String timeUnit)}` function takes into consideration the initial-time and the frequency for the dataset for which is performing the calculation.
+ 4. As mentioned before, if the resolved time doesn't fall exactly on an instance, it will get "rewinded" to match the latest instance before the resolved time. For example, `${coord:offset(1, "DAY")}` is resolved to `2009-05-27T24:00Z` for the 'weeklysiteStats' dataset even though this is the same as `${coord:offset(0, "DAY")}`; this is because the frequency is 7 days, so `${coord:offset(1, "DAY")}` had to be "rewinded".
-Datasets Definition file 'datasets.xml':
+2. **"fastforwarding" in \<start-instance\> `${coord:offset(int n, String timeUnit)}` calculation:**
-<verbatim>
-<datasets>
-.
- <dataset name="logs" frequency="${coord:hours(1)}"
- initial-instance="2009-01-01T01:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}</uri-template>
- </dataset>
-.
- <dataset name="stats" frequency="${coord:days(1)}"
- initial-instance="2009-01-01T24:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}</uri-template>
- </dataset>
-.
-</datasets>
-</verbatim>
+ When specifying dataset instances, keep in mind that the resolved value of `${coord:offset(int n, String timeUnit)}` must line up
+ with an offset of a multiple of the frequency when used in an 'instance' XML element.
+ However, when used in `'start-instance'` and `'end-instance'` XML elements, this is not a requirement. In this case, the function
+ will automatically resolve the range of instances to match the offset of a multiple of the frequency that would fall between the
+ `'start-instance'` and `'end-instance'` XML elements; in other words, `'start-instance'` XML element is "fastforwarded" while
+ `'end-instance'` XML element is "rewinded". So, in the example below, the frequency of the "logs" dataset is 1 hour while the
+ `'start-instance'` XML element is `${coord:offset(-90, "MINUTE")}` (-1.5 hours). If this were in an `'instance'` XML element, it
+ would be "rewinded", but here it is effectively equivalent to `${coord:offset(-60, "MINUTE")}` or `${coord:current(-1)}` as we are
+ dealing with a range.
-Coordinator application definition:
+ Datasets Definition file 'datasets.xml':
-<verbatim>
- <coordinator-app name="app-coord" frequency="${coord:days(1)}"
- start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
- xmlns="uri:oozie:coordinator:0.1">
- <datasets>
- <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
- </datasets>
- <input-events>
- <data-in name="input" dataset="logs">
- <start-instance>${coord:offset(-90, "MINUTE")}</start-instance>
- <end-instance>${coord:offset(0, "DAY")}</end-instance>
- </data-in>
- </input-events>
- <output-events>
- <data-out name="output" dataset="stats">
- <instance>${coord:offset(0, "DAY")}</instance>
- </data-out>
- </output-events>
- <action>
- <workflow>
- ...
- </workflow>
- </action>
- </coordinator-app>
-</verbatim>
----++++ 6.6.3. coord:hoursInDay(int n) EL Function for Synchronous Datasets
+ ```
+ <datasets>
+ .
+ <dataset name="logs" frequency="${coord:hours(1)}"
+ initial-instance="2009-01-01T01:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}</uri-template>
+ </dataset>
+ .
+ <dataset name="stats" frequency="${coord:days(1)}"
+ initial-instance="2009-01-01T24:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}</uri-template>
+ </dataset>
+ .
+ </datasets>
+ ```
-The =${coord:hoursInDay(int n)}= EL function returns the number of hours for the specified day, in a timezone/daylight-saving sensitive way.
+ Coordinator application definition:
-=n= is offset (in days) from the current nominal time. A negative value is the n<sup>th</sup> previous day. Zero is the current day. A positive number is the n<sup>th</sup> next day.
+
+ ```
+ <coordinator-app name="app-coord" frequency="${coord:days(1)}"
+ start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
+ xmlns="uri:oozie:coordinator:0.1">
+ <datasets>
+ <include>hdfs://foo:8020/app/dataset-definitions/datasets.xml</include>
+ </datasets>
+ <input-events>
+ <data-in name="input" dataset="logs">
+ <start-instance>${coord:offset(-90, "MINUTE")}</start-instance>
+ <end-instance>${coord:offset(0, "DAY")}</end-instance>
+ </data-in>
+ </input-events>
+ <output-events>
+ <data-out name="output" dataset="stats">
+ <instance>${coord:offset(0, "DAY")}</instance>
+ </data-out>
+ </output-events>
+ <action>
+ <workflow>
+ ...
+ </workflow>
+ </action>
+ </coordinator-app>
+ ```
+
+#### 6.6.3. coord:hoursInDay(int n) EL Function for Synchronous Datasets
+
+The `${coord:hoursInDay(int n)}` EL function returns the number of hours for the specified day, in a timezone/daylight-saving sensitive way.
+
+`n` is offset (in days) from the current nominal time. A negative value is the n<sup>th</sup> previous day. Zero is the current day. A positive number is the n<sup>th</sup> next day.
The returned value is calculated taking into account timezone daylight-saving information.
-Normally it returns =24=, only DST switch days for the timezone in question it will return either =23= or =25=.
+Normally it returns `24`, only DST switch days for the timezone in question it will return either `23` or `25`.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-| *Nominal UTC time* | *Timezone* | *EndOfFlag* | *Usage* | *Value* | *Comments* |
-| =2009-01-01T08:00Z= | =UTC= | =NO= |=${coord:hoursInDay(0)}= | 24 | hours in 2009JAN01 UTC |
-| =2009-01-01T08:00Z= | =America/Los_Angeles= | =NO= |=${coord:hoursInDay(0)}= | 24 | hours in 2009JAN01 PST8PDT time |
-| =2009-01-01T08:00Z= | =America/Los_Angeles= | =NO= |=${coord:hoursInDay(-1)}= | 24 | hours in 2008DEC31 PST8PDT time |
+| **Nominal UTC time** | **Timezone** | **EndOfFlag** | **Usage** | **Value** | **Comments** |
+| --- | --- | --- | --- | --- | --- |
+| `2009-01-01T08:00Z` | `UTC` | `NO` |`${coord:hoursInDay(0)}` | 24 | hours in 2009JAN01 UTC |
+| `2009-01-01T08:00Z` | `America/Los_Angeles` | `NO` |`${coord:hoursInDay(0)}` | 24 | hours in 2009JAN01 PST8PDT time |
+| `2009-01-01T08:00Z` | `America/Los_Angeles` | `NO` |`${coord:hoursInDay(-1)}` | 24 | hours in 2008DEC31 PST8PDT time |
| ||||| |
-| =2009-03-08T08:00Z= | =UTC= | =NO= | =${coord:hoursInDay(0)}= | 24 | hours in 2009MAR08 UTC time |
-| =2009-03-08T08:00Z= | =Europe/London= | =NO= | =${coord:hoursInDay(0)}= | 24 | hours in 2009MAR08 BST1BDT time |
-| =2009-03-08T08:00Z= | =America/Los_Angeles= | =NO= | =${coord:hoursInDay(0)}= | 23 | hours in 2009MAR08 PST8PDT time <br/> (2009MAR08 is DST switch in the US) |
-| =2009-03-08T08:00Z= | =America/Los_Angeles= | =NO= | =${coord:hoursInDay(1)}= | 24 | hours in 2009MAR09 PST8PDT time |
-| =2009-03-07T08:00Z= | =America/Los_Angeles= | =EndOfDay= | =${coord:hoursInDay(0)}= | 24 | hours in 2009MAR07 PST8PDT time |
-| =2009-03-07T08:00Z= | =America/Los_Angeles= | =EndOfDay= | =${coord:hoursInDay(1)}= | 23 | hours in 2009MAR08 PST8PDT time <br/> (2009MAR08 is DST switch in the US) |
+| `2009-03-08T08:00Z` | `UTC` | `NO` | `${coord:hoursInDay(0)}` | 24 | hours in 2009MAR08 UTC time |
+| `2009-03-08T08:00Z` | `Europe/London` | `NO` | `${coord:hoursInDay(0)}` | 24 | hours in 2009MAR08 BST1BDT time |
+| `2009-03-08T08:00Z` | `America/Los_Angeles` | `NO` | `${coord:hoursInDay(0)}` | 23 | hours in 2009MAR08 PST8PDT time <br/> (2009MAR08 is DST switch in the US) |
+| `2009-03-08T08:00Z` | `America/Los_Angeles` | `NO` | `${coord:hoursInDay(1)}` | 24 | hours in 2009MAR09 PST8PDT time |
+| `2009-03-07T08:00Z` | `America/Los_Angeles` | `EndOfDay` | `${coord:hoursInDay(0)}` | 24 | hours in 2009MAR07 PST8PDT time |
+| `2009-03-07T08:00Z` | `America/Los_Angeles` | `EndOfDay` | `${coord:hoursInDay(1)}` | 23 | hours in 2009MAR08 PST8PDT time <br/> (2009MAR08 is DST switch in the US) |
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="${jobStart}" end="${jobEnd}" timezone="${timezone}"
xmlns="uri:oozie:coordinator:0.1">
@@ -1997,47 +2072,49 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
-This example is the example of section #6.5 but with a minor change. The argument for the =${coord:current(int n)}= function in the 'start-instance' element, instead using =-23=, the example now uses =-(coord:hoursInDay(0) - 1)=.
+This example is the example of section #6.5 but with a minor change. The argument for the `${coord:current(int n)}` function in the 'start-instance' element, instead using `-23`, the example now uses `-(coord:hoursInDay(0) - 1)`.
This simple change fully enables this coordinator application to handle daily data (produced hourly) for any timezone, with timezones observing or not daylight saving.
-For timezones observing daylight saving, on the days of DST switch, the function will resolve to =23= or =25=, thus the dataset instances used will be for for the day in the DST sense.
+For timezones observing daylight saving, on the days of DST switch, the function will resolve to `23` or `25`, thus the dataset instances used will be for for the day in the DST sense.
-For timezones not observing daylight saving, it always returns =24=.
+For timezones not observing daylight saving, it always returns `24`.
----++++ 6.6.4. coord:daysInMonth(int n) EL Function for Synchronous Datasets
+#### 6.6.4. coord:daysInMonth(int n) EL Function for Synchronous Datasets
-The =${coord:daysInMonth(int n)}= EL function returns the number of days for month of the specified day.
+The `${coord:daysInMonth(int n)}` EL function returns the number of days for month of the specified day.
-=n= is offset (in months) from the current nominal time. A negative value is the n<sup>th</sup> previous month. Zero is the current month. A positive number is the n<sup>th</sup> next month.
+`n` is offset (in months) from the current nominal time. A negative value is the n<sup>th</sup> previous month. Zero is the current month. A positive number is the n<sup>th</sup> next month.
The returned value is calculated taking into account leap years information.
-The =${coord:daysInMonth(int n)}= EL function can be used to express monthly ranges for dataset instances.
+The `${coord:daysInMonth(int n)}` EL function can be used to express monthly ranges for dataset instances.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-| *Nominal UTC time* | *Timezone* |*EndOfFlag* | *Usage* | *Value* | *Comments* |
-| =2008-02-01T00:00Z= | =UTC= | =NO= | =${coord:daysInMonth(0)}= | 29 | days in 2008FEB UTC time |
-| =2009-02-01T00:00Z= | =UTC= | =NO= | =${coord:daysInMonth(0)}= | 28 | days in 2009FEB UTC time |
-| =2009-02-01T00:00Z= | =UTC= | =NO= | =${coord:daysInMonth(-1)}= | 31 | days in 2009JAN UTC time |
-| =2009-03-01T00:00Z= | =UTC= | =NO= | =${coord:daysInMonth(1)}= | 30 | days in 2009APR UTC time |
-| =2009-02-01T00:00Z= | =Americas/Los_Angeles= | =NO= |=${coord:daysInMonth(0)}= | 31 | days in 2009JAN PST8PDT time, note that the nominal time is UTC |
+| **Nominal UTC time** | **Timezone** |**EndOfFlag** | **Usage** | **Value** | **Comments** |
+| --- | --- | --- | --- | --- | --- |
+| `2008-02-01T00:00Z` | `UTC` | `NO` | `${coord:daysInMonth(0)}` | 29 | days in 2008FEB UTC time |
+| `2009-02-01T00:00Z` | `UTC` | `NO` | `${coord:daysInMonth(0)}` | 28 | days in 2009FEB UTC time |
+| `2009-02-01T00:00Z` | `UTC` | `NO` | `${coord:daysInMonth(-1)}` | 31 | days in 2009JAN UTC time |
+| `2009-03-01T00:00Z` | `UTC` | `NO` | `${coord:daysInMonth(1)}` | 30 | days in 2009APR UTC time |
+| `2009-02-01T00:00Z` | `Americas/Los_Angeles` | `NO` |`${coord:daysInMonth(0)}` | 31 | days in 2009JAN PST8PDT time, note that the nominal time is UTC |
|||||||
-| =2008-02-01T00:00Z= | =UTC= | =EndOfMonth= | =${coord:daysInMonth(0)}= | 29 | days in 2008FEB UTC time |
-| =2008-02-01T00:00Z= | =UTC= | =EndOfMonth= | =${coord:daysInMonth(-1)}= | 31 | days in 2008JAN UTC time |
-| =2009-02-01T00:00Z= | =UTC= | =EndOfMonth= | =${coord:daysInMonth(0)}= | 28 | days in 2009FEB UTC time |
-| =2009-02-01T00:00Z= | =UTC= | =EndOfMonth= | =${coord:daysInMonth(-1)}= | 31 | days in 2009JAN UTC time |
-| =2009-03-01T00:00Z= | =UTC= | =EndOfMonth= | =${coord:daysInMonth(1)}= | 30 | days in 2009APR UTC time |
-| =2009-02-01T00:00Z= | =Americas/Los_Angeles= | =EndOfMonth= |=${coord:daysInMonth(0)}= | 31 | days in 2009JAN PST8PDT time, note that the nominal time is UTC |
+| `2008-02-01T00:00Z` | `UTC` | `EndOfMonth` | `${coord:daysInMonth(0)}` | 29 | days in 2008FEB UTC time |
+| `2008-02-01T00:00Z` | `UTC` | `EndOfMonth` | `${coord:daysInMonth(-1)}` | 31 | days in 2008JAN UTC time |
+| `2009-02-01T00:00Z` | `UTC` | `EndOfMonth` | `${coord:daysInMonth(0)}` | 28 | days in 2009FEB UTC time |
+| `2009-02-01T00:00Z` | `UTC` | `EndOfMonth` | `${coord:daysInMonth(-1)}` | 31 | days in 2009JAN UTC time |
+| `2009-03-01T00:00Z` | `UTC` | `EndOfMonth` | `${coord:daysInMonth(1)}` | 30 | days in 2009APR UTC time |
+| `2009-02-01T00:00Z` | `Americas/Los_Angeles` | `EndOfMonth` |`${coord:daysInMonth(0)}` | 31 | days in 2009JAN PST8PDT time, note that the nominal time is UTC |
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:months(1)}"
start="2009-01-31T24:00Z" end="2009-12-31T24:00" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -2061,48 +2138,50 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
This example is a coordinator application that runs monthly, and consumes the daily feeds for the last month.
----++++ 6.6.5. coord:tzOffset() EL Function for Synchronous Datasets
+#### 6.6.5. coord:tzOffset() EL Function for Synchronous Datasets
-=${coord:tzOffset()}= EL function returns the difference in *minutes* between a dataset timezone and the coordinator job timezone at the current nominal time. This EL function is useful when dealing with datasets from multiple timezones, but execute in a different timezone.
+`${coord:tzOffset()}` EL function returns the difference in **minutes** between a dataset timezone and the coordinator job timezone at the current nominal time. This EL function is useful when dealing with datasets from multiple timezones, but execute in a different timezone.
-<verbatim>
+
+```
DS_TZ : dataset TZ offset in minutes at the current nominal time (UTC offset)
JOB_TZ: coordinator job UTC TZ offset in minutes at the current nominal time (UTC offset).
coord:tzOffset() = DS_TZ - JOB_TZ
-</verbatim>
+```
-For example: Los Angeles Winter offset (no DST) is =-480= (-08:00 hours). India offset is =-330= (+05:30 hours).
+For example: Los Angeles Winter offset (no DST) is `-480` (-08:00 hours). India offset is `-330` (+05:30 hours).
The value returned by this function may change because of the daylight saving rules of the 2 timezones. For example, between Continental Europe and The U.S. West coast, most of the year the timezone different is 9 hours, but there are a few day or weeks.
-IMPORTANT: While the offset is multiples of 60 for most timezones, it can be multiple of 30 mins when one of the timezones is has a =##:30= offset (i.e. India).
+IMPORTANT: While the offset is multiples of 60 for most timezones, it can be multiple of 30 mins when one of the timezones is has a `##:30` offset (i.e. India).
Refer to section #7, 3nd use case for a detailed example.
----++++ 6.6.6. coord:latest(int n) EL Function for Synchronous Datasets
+#### 6.6.6. coord:latest(int n) EL Function for Synchronous Datasets
-=${coord:latest(int n)}= represents the n<sup>th</sup> latest currently available instance of a *synchronous* dataset.
+`${coord:latest(int n)}` represents the n<sup>th</sup> latest currently available instance of a **synchronous** dataset.
-=${coord:latest(int n)}= is not relative to the coordinator action creation (materialization) time, it is the n<sup>th</sup> latest instance available when the action is started (when the workflow job is started).
+`${coord:latest(int n)}` is not relative to the coordinator action creation (materialization) time, it is the n<sup>th</sup> latest instance available when the action is started (when the workflow job is started).
-If a coordinator job is suspended, when resumed, all usages of =${coord:latest(int n)}= will be resolved to the currently existent instances.
+If a coordinator job is suspended, when resumed, all usages of `${coord:latest(int n)}` will be resolved to the currently existent instances.
Finally, it is not possible to represent the latest dataset when execution reaches a node in the workflow job. The resolution of latest dataset instances happens at action start time (workflow job start time).
-The parameter =n= can be a negative integer or zero. Where =0= means the latest instance available, =-1= means the second latest instance available, etc.
+The parameter `n` can be a negative integer or zero. Where `0` means the latest instance available, `-1` means the second latest instance available, etc.
-the =${coord:latest(int n)}= ignores gaps in dataset instances, it just looks for the latest n<sup>th</sup> instance available.
+the `${coord:latest(int n)}` ignores gaps in dataset instances, it just looks for the latest n<sup>th</sup> instance available.
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:hours(1)}"
start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -2126,11 +2205,12 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
If the available dataset instances in HDFS at time of a coordinator action being executed are:
-<verbatim>
+
+```
hdfs://bar:8020/app/logs/2009/01/01
hdfs://bar:8020/app/logs/2009/01/02
hdfs://bar:8020/app/logs/2009/01/03
@@ -2141,32 +2221,34 @@
(missing)
(missing)
hdfs://bar:8020/app/logs/2009/01/10
-</verbatim>
+```
Then, the dataset instances for the input events for the coordinator action will be:
-<verbatim>
+
+```
hdfs://bar:8020/app/logs/2009/01/05
hdfs://bar:8020/app/logs/2009/01/10
-</verbatim>
+```
----++++ 6.6.7. coord:future(int n, int limit) EL Function for Synchronous Datasets
+#### 6.6.7. coord:future(int n, int limit) EL Function for Synchronous Datasets
-=${coord:future(int n, int limit)}= represents the n<sup>th</sup> currently available future instance of a *synchronous* dataset while looking ahead for 'limit' number of instances.
+`${coord:future(int n, int limit)}` represents the n<sup>th</sup> currently available future instance of a **synchronous** dataset while looking ahead for 'limit' number of instances.
-=${coord:future(int n, int limit)}= is relative to the coordinator action creation (materialization) time. The coordinator action creation (materialization) time is computed based on the coordinator job start time and its frequency. The n<sup>th</sup> dataset instance is computed based on the dataset's initial-instance datetime, its frequency and the (current) coordinator action creation (materialization) time.
+`${coord:future(int n, int limit)}` is relative to the coordinator action creation (materialization) time. The coordinator action creation (materialization) time is computed based on the coordinator job start time and its frequency. The n<sup>th</sup> dataset instance is computed based on the dataset's initial-instance datetime, its frequency and the (current) coordinator action creation (materialization) time.
-=n= can be a zero or a positive integer. Where =0= means the immediate instance available, =1= means the second next instance available, etc.
+`n` can be a zero or a positive integer. Where `0` means the immediate instance available, `1` means the second next instance available, etc.
-=limit= should be a positive integer. Where =3= means search for n<sup>th</sup> next instance and should not check beyond =3= instance.
+`limit` should be a positive integer. Where `3` means search for n<sup>th</sup> next instance and should not check beyond `3` instance.
-The =${coord:future(int n, int limit)}= ignores gaps in dataset instances, it just looks for the next n<sup>th</sup> instance available.
+The `${coord:future(int n, int limit)}` ignores gaps in dataset instances, it just looks for the next n<sup>th</sup> instance available.
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:hours(1)}"
start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -2190,11 +2272,12 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
If the available dataset instances in HDFS at time of a coordinator action being executed are:
-<verbatim>
+
+```
hdfs://bar:8020/app/logs/2009/02/01
(missing)
(missing)
@@ -2212,28 +2295,30 @@
hdfs://bar:8020/app/logs/2009/02/14
(missing)
hdfs://bar:8020/app/logs/2009/02/16
-</verbatim>
+```
Then, the dataset instances for the input events for the coordinator action will be:
-<verbatim>
+
+```
hdfs://bar:8020/app/logs/2009/02/01
hdfs://bar:8020/app/logs/2009/02/07
-</verbatim>
+```
----++++ 6.6.8. coord:absolute(String timeStamp) EL Function for Synchronous Datasets
+#### 6.6.8. coord:absolute(String timeStamp) EL Function for Synchronous Datasets
-=${coord:absolute(String timeStamp)}= represents absolute dataset instance time. coord:absolute is only supported with range
+`${coord:absolute(String timeStamp)}` represents absolute dataset instance time. coord:absolute is only supported with range
where, start-instance is coord:absolute and end-instance is coord:current. Specifying a fixed date as the start instance is
useful if your processing needs to process all dataset instances from a specific instance to the current instance.
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:months(1)}"
start="2009-01-01T01:00Z" end="2009-12-31T00:00" timezone="UTC"
xmlns="uri:oozie:coordinator:0.4">
@@ -2254,299 +2339,317 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
Then, the dataset instances for the input events for the coordinator action at first run will be:
-<verbatim>
+
+```
hdfs://bar:8020/app/logs/2009/02/01
-</verbatim>
+```
The dataset instances for the input events for the coordinator action at second run will be:
-<verbatim>
+
+```
hdfs://bar:8020/app/logs/2009/02/01
hdfs://bar:8020/app/logs/2009/02/07
-</verbatim>
+```
----++++ 6.6.9. coord:endOfMonths(int n) EL Function for Synchronous Datasets
+#### 6.6.9. coord:endOfMonths(int n) EL Function for Synchronous Datasets
-=${coord:endOfMonths(int n)}= represents dataset instance at start of n <sup>th</sup> month. coord:endOfMonths is only
+`${coord:endOfMonths(int n)}` represents dataset instance at start of n <sup>th</sup> month. coord:endOfMonths is only
supported with range, where start-instance is coord:endOfMonths and end-instance is coord:current. Specifying start of
a month is useful if you want to process all the dataset instances from starting of a month to the current instance.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-1. *. =${coord:endOfMonths(int n)}= datetime calculation:*
+1. **. `${coord:endOfMonths(int n)}` datetime calculation:**
-Datasets Definition:
-
-<verbatim>
-<datasets>
-.
- <dataset name="logs" frequency="${coord:days(1)}"
- initial-instance="2009-01-01T00:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}</uri-template>
- </dataset>
-.
-</datasets>
-</verbatim>
-
-For a coordinator action creation time: =2009-05-29T00:00Z= the =${coord:endOfMonths(int n)}= EL function
-would resolve to the following datetime values for the 'logs' dataset:
-
-| *${coord:endOfMonths(int offset)}* | *Dataset 'logs'* | *Comments* |
-| =${coord:endOfMonths(-1)}= | =2009-05-01T00:00Z= | End of last month i.e. start of current month |
-| =${coord:endOfMonths(0)}= | =2009-06-01T00:00Z= | End of current month i.e. start of next month |
-| =${coord:endOfMonths(-2)}= | =2009-04-01T00:00Z= | |
-| =${coord:endOfMonths(-3)}= | =2009-03-01T00:00Z= | |
-
-*%GREEN% Example: %ENDCOLOR%*:
-
-Coordinator application definition:
-
-<verbatim>
-<coordinator-app name="app-coord" frequency="${coord:days(1)}"
- start='2009-08-06T00:00Z' end="2009-12-31T00:00" timezone="UTC"
- xmlns="uri:oozie:coordinator:0.4">
- <input-events>
- <data-in name="input" dataset="logs">
- <dataset name='a' frequency='1' initial-instance='2009-06-06T00:00Z'
- timezone='UTC' freq_timeunit='DAY' end_of_duration='NONE'>
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}
- </uri-template>
- </dataset>
- <start-instance>${coord:endOfMonths(-1)}</start-instance>
- <end-instance>${coord:current(0)}</end-instance>
- </data-in>
- </input-events>
- <action>
- <workflow>
- .............
- </workflow>
- </action>
-</coordinator-app>
-</verbatim>
-
-Then, the dataset instances for the input events for the coordinator action at first run will be:
-
-<verbatim>
- hdfs://bar:8020/app/logs/2009/08/01
- hdfs://bar:8020/app/logs/2009/08/02
- hdfs://bar:8020/app/logs/2009/08/02
- hdfs://bar:8020/app/logs/2009/08/03
- hdfs://bar:8020/app/logs/2009/08/04
- hdfs://bar:8020/app/logs/2009/08/05
- hdfs://bar:8020/app/logs/2009/08/06
-</verbatim>
-
-The dataset instances for the input events for the coordinator action at second run will be:
-
-<verbatim>
- hdfs://bar:8020/app/logs/2009/08/01
- hdfs://bar:8020/app/logs/2009/08/02
- hdfs://bar:8020/app/logs/2009/08/02
- hdfs://bar:8020/app/logs/2009/08/03
- hdfs://bar:8020/app/logs/2009/08/04
- hdfs://bar:8020/app/logs/2009/08/05
- hdfs://bar:8020/app/logs/2009/08/06
- hdfs://bar:8020/app/logs/2009/08/07
-</verbatim>
+ Datasets Definition:
----++++ 6.6.10. coord:endOfWeeks(int n) EL Function for Synchronous Datasets
+ ```
+ <datasets>
+ .
+ <dataset name="logs" frequency="${coord:days(1)}"
+ initial-instance="2009-01-01T00:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}</uri-template>
+ </dataset>
+ .
+ </datasets>
+ ```
+
+ For a coordinator action creation time: `2009-05-29T00:00Z` the `${coord:endOfMonths(int n)}` EL function
+ would resolve to the following datetime values for the 'logs' dataset:
+
+ | **${coord:endOfMonths(int offset)}** | **Dataset 'logs'** | **Comments** |
+| --- | --- | --- |
+ | `${coord:endOfMonths(-1)}` | `2009-05-01T00:00Z` | End of last month i.e. start of current month |
+ | `${coord:endOfMonths(0)}` | `2009-06-01T00:00Z` | End of current month i.e. start of next month |
+ | `${coord:endOfMonths(-2)}` | `2009-04-01T00:00Z` | |
+ | `${coord:endOfMonths(-3)}` | `2009-03-01T00:00Z` | |
+
+ **<font color="#008000"> Example: </font>**:
+
+ Coordinator application definition:
-=${coord:endOfWeeks(int n)}= represents dataset instance at start of n <sup>th</sup> week. The start of the week
+ ```
+ <coordinator-app name="app-coord" frequency="${coord:days(1)}"
+ start='2009-08-06T00:00Z' end="2009-12-31T00:00" timezone="UTC"
+ xmlns="uri:oozie:coordinator:0.4">
+ <input-events>
+ <data-in name="input" dataset="logs">
+ <dataset name='a' frequency='1' initial-instance='2009-06-06T00:00Z'
+ timezone='UTC' freq_timeunit='DAY' end_of_duration='NONE'>
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}
+ </uri-template>
+ </dataset>
+ <start-instance>${coord:endOfMonths(-1)}</start-instance>
+ <end-instance>${coord:current(0)}</end-instance>
+ </data-in>
+ </input-events>
+ <action>
+ <workflow>
+ .............
+ </workflow>
+ </action>
+ </coordinator-app>
+ ```
+
+ Then, the dataset instances for the input events for the coordinator action at first run will be:
+
+
+ ```
+ hdfs://bar:8020/app/logs/2009/08/01
+ hdfs://bar:8020/app/logs/2009/08/02
+ hdfs://bar:8020/app/logs/2009/08/02
+ hdfs://bar:8020/app/logs/2009/08/03
+ hdfs://bar:8020/app/logs/2009/08/04
+ hdfs://bar:8020/app/logs/2009/08/05
+ hdfs://bar:8020/app/logs/2009/08/06
+ ```
+
+ The dataset instances for the input events for the coordinator action at second run will be:
+
+
+ ```
+ hdfs://bar:8020/app/logs/2009/08/01
+ hdfs://bar:8020/app/logs/2009/08/02
+ hdfs://bar:8020/app/logs/2009/08/02
+ hdfs://bar:8020/app/logs/2009/08/03
+ hdfs://bar:8020/app/logs/2009/08/04
+ hdfs://bar:8020/app/logs/2009/08/05
+ hdfs://bar:8020/app/logs/2009/08/06
+ hdfs://bar:8020/app/logs/2009/08/07
+ ```
+
+
+#### 6.6.10. coord:endOfWeeks(int n) EL Function for Synchronous Datasets
+
+
+`${coord:endOfWeeks(int n)}` represents dataset instance at start of n <sup>th</sup> week. The start of the week
calculated similar to mentioned in
-[[CoordinatorFunctionalSpec#a4.4.3._The_coord:endOfWeeksint_n_EL_function][coord:endOfWeeks section above]].
+[coord:endOfWeeks section above](CoordinatorFunctionalSpec.html#a4.4.3._The_coord:endOfWeeksint_n_EL_function).
coord:endOfWeeks is only supported with range, where start-instance is coord:endOfWeeks and end-instance is
coord:current. Specifying start of a week is useful if you want to process all the dataset instances from
starting of a week to the current instance.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-1. *. =${coord:endOfWeeks(int n)}= datetime calculation:*
+1. **. `${coord:endOfWeeks(int n)}` datetime calculation:**
-Datasets Definition:
-
-<verbatim>
-<datasets>
-.
- <dataset name="logs" frequency="${coord:days(1)}"
- initial-instance="2009-01-01T00:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}</uri-template>
- </dataset>
-.
-</datasets>
-</verbatim>
-
-For a coordinator action creation time: =2009-05-29T00:00Z= the =${coord:endOfWeeks(int n)}= EL function
-would resolve to the following datetime values for the 'logs' dataset:
-
-| *${coord:endOfWeeks(int offset)}* | *Dataset 'logs'* | *Comments* |
-| =${coord:endOfWeeks(-1)}= | =2009-05-24T00:00Z= | End of last week i.e. start of current week |
-| =${coord:endOfWeeks(0)}= | =2009-05-31T00:00Z= | End of current week i.e. start of next week |
-| =${coord:endOfWeeks(-2)}= | =2009-05-17T00:00Z= | |
-| =${coord:endOfWeeks(-4)}= | =2009-05-03T00:00Z= | |
+ Datasets Definition:
-*%GREEN% Example: %ENDCOLOR%*:
+ ```
+ <datasets>
+ .
+ <dataset name="logs" frequency="${coord:days(1)}"
+ initial-instance="2009-01-01T00:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}</uri-template>
+ </dataset>
+ .
+ </datasets>
+ ```
-Coordinator application definition:
+ For a coordinator action creation time: `2009-05-29T00:00Z` the `${coord:endOfWeeks(int n)}` EL function
+ would resolve to the following datetime values for the 'logs' dataset:
-<verbatim>
-<coordinator-app name="app-coord" frequency="${coord:days(1)}"
- start='2009-08-06T00:00Z' end="2009-12-31T00:00" timezone="UTC"
- xmlns="uri:oozie:coordinator:0.4">
- <input-events>
- <data-in name="input" dataset="logs">
- <dataset name='a' frequency='1' initial-instance='2009-06-06T00:00Z'
- timezone='UTC' freq_timeunit='DAY' end_of_duration='NONE'>
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}
- </uri-template>
- </dataset>
- <start-instance>${coord:endOfWeeks(-1)}</start-instance>
- <end-instance>${coord:current(0)}</end-instance>
- </data-in>
- </input-events>
- <action>
- <workflow>
- .............
- </workflow>
- </action>
-</coordinator-app>
-</verbatim>
-
-Then, the dataset instances for the input events for the coordinator action at first run will be:
-
-<verbatim>
- hdfs://bar:8020/app/logs/2009/08/02
- hdfs://bar:8020/app/logs/2009/08/03
- hdfs://bar:8020/app/logs/2009/08/04
- hdfs://bar:8020/app/logs/2009/08/05
- hdfs://bar:8020/app/logs/2009/08/06
-</verbatim>
-
-The dataset instances for the input events for the coordinator action at second run will be:
-
-<verbatim>
- hdfs://bar:8020/app/logs/2009/08/02
- hdfs://bar:8020/app/logs/2009/08/03
- hdfs://bar:8020/app/logs/2009/08/04
- hdfs://bar:8020/app/logs/2009/08/05
- hdfs://bar:8020/app/logs/2009/08/06
- hdfs://bar:8020/app/logs/2009/08/07
-</verbatim>
+ | **${coord:endOfWeeks(int offset)}** | **Dataset 'logs'** | **Comments** |
+| --- | --- | --- |
+ | `${coord:endOfWeeks(-1)}` | `2009-05-24T00:00Z` | End of last week i.e. start of current week |
+ | `${coord:endOfWeeks(0)}` | `2009-05-31T00:00Z` | End of current week i.e. start of next week |
+ | `${coord:endOfWeeks(-2)}` | `2009-05-17T00:00Z` | |
+ | `${coord:endOfWeeks(-4)}` | `2009-05-03T00:00Z` | |
----++++ 6.6.11. coord:endOfDays(int n) EL Function for Synchronous Datasets
+ **<font color="#008000"> Example: </font>**:
+
+ Coordinator application definition:
-=${coord:endOfDays(int n)}= represents dataset instance at start of n <sup>th</sup> day. coord:endOfDays is only
+ ```
+ <coordinator-app name="app-coord" frequency="${coord:days(1)}"
+ start='2009-08-06T00:00Z' end="2009-12-31T00:00" timezone="UTC"
+ xmlns="uri:oozie:coordinator:0.4">
+ <input-events>
+ <data-in name="input" dataset="logs">
+ <dataset name='a' frequency='1' initial-instance='2009-06-06T00:00Z'
+ timezone='UTC' freq_timeunit='DAY' end_of_duration='NONE'>
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}
+ </uri-template>
+ </dataset>
+ <start-instance>${coord:endOfWeeks(-1)}</start-instance>
+ <end-instance>${coord:current(0)}</end-instance>
+ </data-in>
+ </input-events>
+ <action>
+ <workflow>
+ .............
+ </workflow>
+ </action>
+ </coordinator-app>
+ ```
+
+ Then, the dataset instances for the input events for the coordinator action at first run will be:
+
+
+ ```
+ hdfs://bar:8020/app/logs/2009/08/02
+ hdfs://bar:8020/app/logs/2009/08/03
+ hdfs://bar:8020/app/logs/2009/08/04
+ hdfs://bar:8020/app/logs/2009/08/05
+ hdfs://bar:8020/app/logs/2009/08/06
+ ```
+
+ The dataset instances for the input events for the coordinator action at second run will be:
+
+
+ ```
+ hdfs://bar:8020/app/logs/2009/08/02
+ hdfs://bar:8020/app/logs/2009/08/03
+ hdfs://bar:8020/app/logs/2009/08/04
+ hdfs://bar:8020/app/logs/2009/08/05
+ hdfs://bar:8020/app/logs/2009/08/06
+ hdfs://bar:8020/app/logs/2009/08/07
+ ```
+
+
+#### 6.6.11. coord:endOfDays(int n) EL Function for Synchronous Datasets
+
+
+`${coord:endOfDays(int n)}` represents dataset instance at start of n <sup>th</sup> day. coord:endOfDays is only
supported with range, where start-instance is coord:endOfDays and end-instance is coord:current. Specifying start
of a day is useful if you want to process all the dataset instances from starting of a day to the current instance.
-*%GREEN% Examples: %ENDCOLOR%*
+**<font color="#008000"> Examples: </font>**
-1. *. =${coord:endOfDays(int n)}= datetime calculation:*
+1. **. `${coord:endOfDays(int n)}` datetime calculation:**
-Datasets Definition:
-
-<verbatim>
-<datasets>
-.
- <dataset name="logs" frequency="${coord:days(1)}"
- initial-instance="2009-01-01T00:00Z" timezone="UTC">
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}/${HOUR}/${MINUTE}</uri-template>
- </dataset>
-.
-</datasets>
-</verbatim>
-
-For a coordinator action creation time: =2009-05-25T23:00Z= the =${coord:endOfDays(int n)}= EL function
-would resolve to the following datetime values for the 'logs' dataset:
-
-| *${coord:endOfDays(int offset)}* | *Dataset 'logs'* | *Comments* |
-| =${coord:endOfDays(-1)}= | =2009-05-25T00:00Z= | End of previous day i.e. start of the current day |
-| =${coord:endOfDays(0)}= | =2009-05-26T00:00Z= | End of current day i.e. start of the next day |
-| =${coord:endOfDays(-2)}= | =2009-05-24T00:00Z= | |
-| =${coord:endOfDays(-4)}= | =2009-05-22T00:00Z= | |
-
-*%GREEN% Example: %ENDCOLOR%*:
-
-Coordinator application definition:
-
-<verbatim>
-<coordinator-app name="app-coord" frequency='60' freq_timeunit='MINUTE'
- start='2009-08-06T00:00Z' end="2009-12-31T00:00" timezone="UTC"
- xmlns="uri:oozie:coordinator:0.4">
- <input-events>
- <data-in name="input" dataset="logs">
- <dataset name='a' frequency='60' initial-instance='2009-06-06T00:00Z'
- timezone='UTC' freq_timeunit='MINUTE' end_of_duration='NONE'>
- <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}
- </uri-template>
- </dataset>
- <start-instance>${coord:endOfDays(-1)}</start-instance>
- <end-instance>${coord:current(0)}</end-instance>
- </data-in>
- </input-events>
- <action>
- <workflow>
- .............
- </workflow>
- </action>
-</coordinator-app>
-</verbatim>
-
-Then, the dataset instances for the input events for the coordinator action at first run will be:
-
-<verbatim>
- hdfs://bar:8020/app/logs/2009/08/06/00/00
- hdfs://bar:8020/app/logs/2009/08/06/01/00
- hdfs://bar:8020/app/logs/2009/08/06/02/00
- hdfs://bar:8020/app/logs/2009/08/06/03/00
- hdfs://bar:8020/app/logs/2009/08/06/04/00
- hdfs://bar:8020/app/logs/2009/08/06/05/00
-</verbatim>
-
-The dataset instances for the input events for the coordinator action at second run will be:
-
-<verbatim>
- hdfs://bar:8020/app/logs/2009/08/06/00/00
- hdfs://bar:8020/app/logs/2009/08/06/01/00
- hdfs://bar:8020/app/logs/2009/08/06/02/00
- hdfs://bar:8020/app/logs/2009/08/06/03/00
- hdfs://bar:8020/app/logs/2009/08/06/04/00
- hdfs://bar:8020/app/logs/2009/08/06/05/00
- hdfs://bar:8020/app/logs/2009/08/06/06/00
-</verbatim>
+ Datasets Definition:
----++++ 6.6.12. coord:version(int n) EL Function for Asynchronous Datasets
+ ```
+ <datasets>
+ .
+ <dataset name="logs" frequency="${coord:days(1)}"
+ initial-instance="2009-01-01T00:00Z" timezone="UTC">
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}${MONTH}/${DAY}/${HOUR}/${MINUTE}</uri-template>
+ </dataset>
+ .
+ </datasets>
+ ```
+
+ For a coordinator action creation time: `2009-05-25T23:00Z` the `${coord:endOfDays(int n)}` EL function
+ would resolve to the following datetime values for the 'logs' dataset:
+
+ | **${coord:endOfDays(int offset)}** | **Dataset 'logs'** | **Comments** |
+| --- | --- | --- |
+ | `${coord:endOfDays(-1)}` | `2009-05-25T00:00Z` | End of previous day i.e. start of the current day |
+ | `${coord:endOfDays(0)}` | `2009-05-26T00:00Z` | End of current day i.e. start of the next day |
+ | `${coord:endOfDays(-2)}` | `2009-05-24T00:00Z` | |
+ | `${coord:endOfDays(-4)}` | `2009-05-22T00:00Z` | |
+
+ **<font color="#008000"> Example: </font>**:
+
+ Coordinator application definition:
+
+
+ ```
+ <coordinator-app name="app-coord" frequency='60' freq_timeunit='MINUTE'
+ start='2009-08-06T00:00Z' end="2009-12-31T00:00" timezone="UTC"
+ xmlns="uri:oozie:coordinator:0.4">
+ <input-events>
+ <data-in name="input" dataset="logs">
+ <dataset name='a' frequency='60' initial-instance='2009-06-06T00:00Z'
+ timezone='UTC' freq_timeunit='MINUTE' end_of_duration='NONE'>
+ <uri-template>hdfs://bar:8020/app/logs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}
+ </uri-template>
+ </dataset>
+ <start-instance>${coord:endOfDays(-1)}</start-instance>
+ <end-instance>${coord:current(0)}</end-instance>
+ </data-in>
+ </input-events>
+ <action>
+ <workflow>
+ .............
+ </workflow>
+ </action>
+ </coordinator-app>
+ ```
+
+ Then, the dataset instances for the input events for the coordinator action at first run will be:
+
+
+ ```
+ hdfs://bar:8020/app/logs/2009/08/06/00/00
+ hdfs://bar:8020/app/logs/2009/08/06/01/00
+ hdfs://bar:8020/app/logs/2009/08/06/02/00
+ hdfs://bar:8020/app/logs/2009/08/06/03/00
+ hdfs://bar:8020/app/logs/2009/08/06/04/00
+ hdfs://bar:8020/app/logs/2009/08/06/05/00
+ ```
+
+ The dataset instances for the input events for the coordinator action at second run will be:
+
+
+ ```
+ hdfs://bar:8020/app/logs/2009/08/06/00/00
+ hdfs://bar:8020/app/logs/2009/08/06/01/00
+ hdfs://bar:8020/app/logs/2009/08/06/02/00
+ hdfs://bar:8020/app/logs/2009/08/06/03/00
+ hdfs://bar:8020/app/logs/2009/08/06/04/00
+ hdfs://bar:8020/app/logs/2009/08/06/05/00
+ hdfs://bar:8020/app/logs/2009/08/06/06/00
+ ```
+
+
+#### 6.6.12. coord:version(int n) EL Function for Asynchronous Datasets
* TBD
----++++ 6.6.13. coord:latest(int n) EL Function for Asynchronous Datasets
+#### 6.6.13. coord:latest(int n) EL Function for Asynchronous Datasets
* TBD
----++++ 6.6.14. Dataset Instance Resolution for Instances Before the Initial Instance
+#### 6.6.14. Dataset Instance Resolution for Instances Before the Initial Instance
When defining input events that refer to dataset instances it may be possible that the resolution of instances is out of it lower bound. This is scenario is likely to happen when the instance resolution is very close to the initial-instance. This is useful for bootstrapping the application.
-To address this edge scenario, Oozie Coordinator<strong> silently ignores dataset instances out of bounds.</strong>
+To address this edge scenario, Oozie Coordinator **silently ignores dataset instances out of bounds.**
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:hours(1)}"
start="2009-01-01T01:00Z" end="2009-12-31T24:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -2570,27 +2673,28 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
-In the case of the synchronous 'logs' dataset, for the first action of this coordinator job, the instances referred in the input events will resolve to just 1 instance. For the second action it will resolve to 2 instances. And so on. Only after the 24th action, the input events will resolve constantly to 24 instances. In other words, while =${coord:current(-23)}= resolves to datetimes prior to the 'initial-instance' the required range will start from the 'initial-instance', '2009-01-01T00:00Z' in this example.
+In the case of the synchronous 'logs' dataset, for the first action of this coordinator job, the instances referred in the input events will resolve to just 1 instance. For the second action it will resolve to 2 instances. And so on. Only after the 24th action, the input events will resolve constantly to 24 instances. In other words, while `${coord:current(-23)}` resolves to datetimes prior to the 'initial-instance' the required range will start from the 'initial-instance', '2009-01-01T00:00Z' in this example.
----+++ 6.7. Parameterization of Coordinator Application Actions
+### 6.7. Parameterization of Coordinator Application Actions
Actions started by a coordinator application normally require access to the dataset instances resolved by the input and output events to be able to propagate them to the workflow job as parameters.
The following EL functions are the mechanism that enables this propagation.
----++++ 6.7.1. coord:dataIn(String name) EL Function
+#### 6.7.1. coord:dataIn(String name) EL Function
-The =${coord:dataIn(String name)}= EL function resolves to all the URIs for the dataset instances specified in an input event dataset section.
+The `${coord:dataIn(String name)}` EL function resolves to all the URIs for the dataset instances specified in an input event dataset section.
-The =${coord:dataIn(String name)}= is commonly used to pass the URIs of dataset instances that will be consumed by a workflow job triggered by a coordinator action.
+The `${coord:dataIn(String name)}` is commonly used to pass the URIs of dataset instances that will be consumed by a workflow job triggered by a coordinator action.
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -2620,35 +2724,37 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
In this example, each coordinator action will use as input events the last day hourly instances of the 'logs' dataset.
-The =${coord:dataIn(String name)}= function enables the coordinator application to pass the URIs of all the dataset instances for the last day to the workflow job triggered by the coordinator action. For the =2009-01-02T00:00Z" run, the =${coord:dataIn('inputLogs')}= function will resolve to:
+The `${coord:dataIn(String name)}` function enables the coordinator application to pass the URIs of all the dataset instances for the last day to the workflow job triggered by the coordinator action. For the "2009-01-02T00:00Z" run, the `${coord:dataIn('inputLogs')}` function will resolve to:
-<verbatim>
+
+```
hdfs://bar:8020/app/logs/2009/01/01/01,
hdfs://bar:8020/app/logs/2009/01/01/02,
...
hdfs://bar:8020/app/logs/2009/01/01/23,
hdfs://bar:8020/app/logs/2009/02/00/00
-</verbatim>
+```
-The =${coord:dataIn('inputLogs')}= is used for workflow job configuration property 'wfInput' for the workflow job that will be submitted by the coordinator action on January 2nd 2009. Thus, when the workflow job gets started, the 'wfInput' workflow job configuration property will contain all the above URIs.
+The `${coord:dataIn('inputLogs')}` is used for workflow job configuration property 'wfInput' for the workflow job that will be submitted by the coordinator action on January 2nd 2009. Thus, when the workflow job gets started, the 'wfInput' workflow job configuration property will contain all the above URIs.
Note that all the URIs form a single string value and the URIs are separated by commas. Multiple HDFS URIs separated by commas can be specified as input data to a Map/Reduce job.
----++++ 6.7.2. coord:dataOut(String name) EL Function
+#### 6.7.2. coord:dataOut(String name) EL Function
-The =${coord:dataOut(String name)}= EL function resolves to all the URIs for the dataset instance specified in an output event dataset section.
+The `${coord:dataOut(String name)}` EL function resolves to all the URIs for the dataset instance specified in an output event dataset section.
-The =${coord:dataOut(String name)}= is commonly used to pass the URIs of a dataset instance that will be produced by a workflow job triggered by a coordinator action.
+The `${coord:dataOut(String name)}` is commonly used to pass the URIs of a dataset instance that will be produced by a workflow job triggered by a coordinator action.
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
Datasets Definition file 'datasets.xml'
-<verbatim>
+
+```
<datasets>
.
<dataset name="hourlyLogs" frequency="${coord:hours(1)}"
@@ -2661,11 +2767,12 @@
<uri-template>hdfs://bar:8020/app/daily-logs/${YEAR}/${MONTH}/${DAY}</uri-template>
</dataset>
</datasets>
-</verbatim>
+```
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -2699,33 +2806,35 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
In this example, each coordinator action will use as input events the last 24 hourly instances of the 'hourlyLogs' dataset to create a 'dailyLogs' dataset instance.
-The =${coord:dataOut(String name)}= function enables the coordinator application to pass the URIs of the dataset instance that will be created by the workflow job triggered by the coordinator action. For the =2009-01-01T24:00Z" run, the =${coord:dataOut('dailyLogs')}= function will resolve to:
+The `${coord:dataOut(String name)}` function enables the coordinator application to pass the URIs of the dataset instance that will be created by the workflow job triggered by the coordinator action. For the "2009-01-01T24:00Z" run, the `${coord:dataOut('dailyLogs')}` function will resolve to:
-<verbatim>
+
+```
hdfs://bar:8020/app/logs/2009/01/02
-</verbatim>
+```
-NOTE: The use of =24:00= as hour is useful for human to denote end of the day, but internally Oozie handles it as the zero hour of the next day.
+NOTE: The use of `24:00` as hour is useful for human to denote end of the day, but internally Oozie handles it as the zero hour of the next day.
-The =${coord:dataOut('dailyLogs')}= is used for workflow job configuration property 'wfOutput' for the workflow job that will be submitted by the coordinator action on January 2nd 2009. Thus, when the workflow job gets started, the 'wfOutput' workflow job configuration property will contain the above URI.
+The `${coord:dataOut('dailyLogs')}` is used for workflow job configuration property 'wfOutput' for the workflow job that will be submitted by the coordinator action on January 2nd 2009. Thus, when the workflow job gets started, the 'wfOutput' workflow job configuration property will contain the above URI.
----++++ 6.7.3. coord:nominalTime() EL Function
+#### 6.7.3. coord:nominalTime() EL Function
-The =${coord:nominalTime()}= EL function resolves to the coordinator action creation (materialization) datetime.
+The `${coord:nominalTime()}` EL function resolves to the coordinator action creation (materialization) datetime.
The nominal times is always the coordinator job start datetime plus a multiple of the coordinator job frequency.
This is, when the coordinator action was created based on driver event. For synchronous coordinator applications this would be every tick of the frequency.
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -2755,33 +2864,35 @@
</configuration>
</action>
</coordinator-app>
-</verbatim>
+```
The nominal times for the coordinator actions of this coordinator application example are:
-<verbatim>
+
+```
2009-01-02T00:00Z
2009-01-03T00:00Z
2009-01-04T00:00Z
...
2010-01-01T00:00Z
-</verbatim>
+```
These are the times the action where created (materialized).
----++++ 6.7.4. coord:actualTime() EL Function
+#### 6.7.4. coord:actualTime() EL Function
-The =${coord:actualTime()}= EL function resolves to the coordinator action actual creation datetime.
+The `${coord:actualTime()}` EL function resolves to the coordinator action actual creation datetime.
When the coordinator action is created based on driver event, the current time is saved to action. An action's
actual time is less than the nominal time if coordinator job is in running in current mode. If job is running
as catch-up mode (job's start time is in the past), the actual time is greater than the nominal time.
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
Coordinator application definition:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="2011-05-01T24:00Z" end="2011-12-31T24:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -2807,31 +2918,32 @@
</configuration>
</action>
</coordinator-app>
-</verbatim>
+```
The actual time for the coordinator actions of this coordinator application example will look like:
If coordinator job was started at 2011-05-01, then actions' actualTime is
-<verbatim>
+
+```
2011-05-01
2011-05-02
2011-05-03
...
2011-12-31
-</verbatim>
+```
----++++ 6.7.5. coord:user() EL Function (since Oozie 2.3)
+#### 6.7.5. coord:user() EL Function (since Oozie 2.3)
-The =coord:user()= function returns the user that started the coordinator job.
+The `coord:user()` function returns the user that started the coordinator job.
----+++ 6.8 Using HCatalog data instances in Coordinator Applications (since Oozie 4.x)
+### 6.8 Using HCatalog data instances in Coordinator Applications (since Oozie 4.x)
This section describes the different EL functions that work with HCatalog data dependencies, in order to write
Coordinator applications that use HCatalog data dependencies.
----++++ 6.8.1 coord:databaseIn(String name), coord:databaseOut(String name) EL function
+#### 6.8.1 coord:databaseIn(String name), coord:databaseOut(String name) EL function
-The functions =${coord:databaseIn(String name)}= and =${coord:databaseOut(String name)}= are used to pass the database
+The functions `${coord:databaseIn(String name)}` and `${coord:databaseOut(String name)}` are used to pass the database
name of HCat dataset instances, input and output respectively, that will be consumed by a workflow job triggered
by a coordinator action.
@@ -2839,20 +2951,20 @@
Similarly for output database, pass the "data-out" name attribute of your 'output-events'.
To illustrate it better:
-If data belongs to 'input-events' and the name attribute of your =<data-in>= is "raw-logs",
-use =${coord:databaseIn('raw-logs')}=.
-Else if it belongs to 'output-events', and the name attribute of your =<data-out>= is "processed-logs",
-use =${coord:databaseOut('processed-logs')}=.
+If data belongs to 'input-events' and the name attribute of your `<data-in>` is "raw-logs",
+use `${coord:databaseIn('raw-logs')}`.
+Else if it belongs to 'output-events', and the name attribute of your `<data-out>` is "processed-logs",
+use `${coord:databaseOut('processed-logs')}`.
Taking this passed argument as input, the EL functions give as string the 'database' name corresponding to your input or output data events.
-Pitfall: Please note NOT to pass the =<dataset>= name itself (as defined under combined set =<datasets>=),
+Pitfall: Please note NOT to pass the `<dataset>` name itself (as defined under combined set `<datasets>`),
as this function works on the 'data-in' and 'data-out' names.
-Refer to the [[CoordinatorFunctionalSpec#HCatPigExampleOne][Example]] below for usage.
+Refer to the [Example](CoordinatorFunctionalSpec.html#HCatPigExampleOne) below for usage.
----++++ 6.8.2 coord:tableIn(String name), coord:tableOut(String name) EL function
+#### 6.8.2 coord:tableIn(String name), coord:tableOut(String name) EL function
-The functions =${coord:tableIn(String name)}= and =${coord:tableOut(String name)}= are used to pass the table
+The functions `${coord:tableIn(String name)}` and `${coord:tableOut(String name)}` are used to pass the table
name of HCat dataset instances, input and output respectively, that will be consumed by a workflow job triggered
by a coordinator action.
@@ -2860,52 +2972,52 @@
Similarly for output table, pass the "data-out" name attribute of your 'output-events'.
To illustrate it better:
-If data belongs to 'input-events' and the name attribute of your =<data-in>= is "raw-logs",
-use =${coord:tableIn('raw-logs')}=.
-Similarly, if it belongs to 'output-events', and the name attribute of your =<data-out>= is "processed-logs",
-use =${coord:tableOut('processed-logs')}=.
+If data belongs to 'input-events' and the name attribute of your `<data-in>` is "raw-logs",
+use `${coord:tableIn('raw-logs')}`.
+Similarly, if it belongs to 'output-events', and the name attribute of your `<data-out>` is "processed-logs",
+use `${coord:tableOut('processed-logs')}`.
Taking this passed argument as input, the EL functions give as string the 'table' name corresponding to your input or output data events.
-Pitfall: Please note NOT to pass the =<dataset>= name itself (as defined under combined set =<datasets>=),
+Pitfall: Please note NOT to pass the `<dataset>` name itself (as defined under combined set `<datasets>`),
as this function works on the 'data-in' and 'data-out' names.
-Refer to the [[CoordinatorFunctionalSpec#HCatPigExampleOne][Example]] below for usage.
+Refer to the [Example](CoordinatorFunctionalSpec.html#HCatPigExampleOne) below for usage.
----++++ 6.8.3 coord:dataInPartitionFilter(String name, String type) EL function
+#### 6.8.3 coord:dataInPartitionFilter(String name, String type) EL function
-The =${coord:dataInPartitionFilter(String name, String type)}= EL function resolves to a filter clause to filter
+The `${coord:dataInPartitionFilter(String name, String type)}` EL function resolves to a filter clause to filter
all the partitions corresponding to the dataset instances specified in an input event dataset section. This EL function
takes two arguments - the name of the input dataset, and the type of the workflow action which will be consuming this filter.
There are 3 types - 'pig', 'hive' and 'java'. This filter clause from the EL function is to be passed as a parameter in the
respective action in the workflow.
The evaluated value of the filter clause will vary based on the action type passed to the EL function. In case of pig,
-the filter will have "==" as the equality operator in the condition. In case of hive and java, the filter will have "="
+the filter will have "``" as the equality operator in the condition. In case of hive and java, the filter will have "="
as the equality operator in the condition. The type java is for java actions, which use HCatInputFormat directly and
launch jobs. The filter clause in that case can be used to construct the InputJobInfo in
-=HCatInputFormat.setInput(Job job, InputJobInfo inputJobInfo)=.
+`HCatInputFormat.setInput(Job job, InputJobInfo inputJobInfo)`.
-Refer to the [[CoordinatorFunctionalSpec#HCatPigExampleOne][Example]] below for usage.
+Refer to the [Example](CoordinatorFunctionalSpec.html#HCatPigExampleOne) below for usage.
----++++ 6.8.4 coord:dataOutPartitions(String name) EL function
+#### 6.8.4 coord:dataOutPartitions(String name) EL function
-The =${coord:dataOutPartitions(String name)}= EL function resolves to a comma-separated list of partition key-value
+The `${coord:dataOutPartitions(String name)}` EL function resolves to a comma-separated list of partition key-value
pairs for the output-event dataset. This can be passed as an argument to HCatStorer in Pig scripts or in case of
java actions that directly use HCatOutputFormat and launch jobs, the partitions list can be parsed to construct
-partition values map for OutputJobInfo in =HcatOutputFormat.setOutput(Job job, OutputJobInfo outputJobInfo)=.
+partition values map for OutputJobInfo in `HcatOutputFormat.setOutput(Job job, OutputJobInfo outputJobInfo)`.
The example below illustrates a pig job triggered by a coordinator, using the EL functions for HCat database, table,
input partitions filter and output partitions. The example takes as input previous day's hourly data to produce
aggregated daily output.
-*%GREEN% Example: %ENDCOLOR%*
+**<font color="#008000"> Example: </font>**
-#HCatPigExampleOne
+<a name="HCatPigExampleOne"></a>
-*Coordinator application definition:*
+**Coordinator application definition:**
-<blockquote>
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.3">
@@ -2966,24 +3078,24 @@
</workflow>
</action>
</coordinator-app>
-</blockquote>
+```
Parameterizing the input/output databases and tables using the corresponding EL function as shown will make them
available in the pig action of the workflow 'logsprocessor-wf'.
Each coordinator action will use as input events the last 24 hourly instances of the 'Click-data' dataset.
-The =${coord:dataInPartitionFilter(String name, String type)}= function enables the coordinator application to pass the
+The `${coord:dataInPartitionFilter(String name, String type)}` function enables the coordinator application to pass the
Partition Filter corresponding to all the dataset instances for the last 24 hours to the workflow job triggered
-by the coordinator action. The =${coord:dataOutPartitions(String name)}= function enables the coordinator application
-to pass the partition key-value string needed by the *HCatStorer* in Pig job when the workflow is triggered by the
+by the coordinator action. The `${coord:dataOutPartitions(String name)}` function enables the coordinator application
+to pass the partition key-value string needed by the **HCatStorer** in Pig job when the workflow is triggered by the
coordinator action.
-#HCatWorkflow
+<a name="HCatWorkflow"></a>
-*Workflow definition:*
+**Workflow definition:**
-<blockquote>
+```
<workflow-app xmlns="uri:oozie:workflow:0.3" name="logsprocessor-wf">
<credentials>
<credential name='hcatauth' type='hcat'>
@@ -3023,28 +3135,28 @@
</kill>
<end name="end"/>
</workflow-app>
-</blockquote>
+```
-*Important*: Ensure that the required hcatalog jars and hive-site.xml are in classpath, with versions corresponding to
-hcatalog installation. Refer [[DG_HCatalogIntegration#HCatalogLibraries][HCatalog Libraries]] for the different ways
+**Important**: Ensure that the required hcatalog jars and hive-site.xml are in classpath, with versions corresponding to
+hcatalog installation. Refer [HCatalog Libraries](DG_HCatalogIntegration.html#HCatalogLibraries) for the different ways
to place them in the hadoop job classpath.
-*Important*: See [[DG_ActionAuthentication][Action Authentication]] for more information about how to access a secure HCatalog from
+**Important**: See [Action Authentication](DG_ActionAuthentication.html) for more information about how to access a secure HCatalog from
any workflow action.
-*Example usage in Pig:*
+**Example usage in Pig:**
-<blockquote>
+```
A = load '$HCAT_IN_DB.$HCAT_IN_TABLE' using org.apache.hive.hcatalog.pig.HCatLoader();
B = FILTER A BY $PARTITION_FILTER;
C = foreach B generate foo, bar;
store C into '$HCAT_OUT_DB.$HCAT_OUT_TABLE' using org.apache.hive.hcatalog.pig.HCatStorer('$OUTPUT_PARTITIONS');
-</blockquote>
+```
-For the =2009-01-02T00:00Z= run with the given dataset instances, the above Pig script with resolved values would look
+For the `2009-01-02T00:00Z` run with the given dataset instances, the above Pig script with resolved values would look
like:
-<blockquote>
+```
A = load 'myInputDatabase.myInputTable' using org.apache.hive.hcatalog.pig.HCatLoader();
B = FILTER A BY ((datestamp==2009010101 AND region==USA) OR
(datestamp==2009010102 AND region==USA) OR
@@ -3053,44 +3165,44 @@
(datestamp==2009010200 AND region==USA));
C = foreach B generate foo, bar;
store C into 'myOutputDatabase.myOutputTable' using org.apache.hive.hcatalog.pig.HCatStorer('datestamp=20090102,region=EUR');
-</blockquote>
+```
----++++ 6.8.5 coord:dataInPartitionMin(String name, String partition) EL function
+#### 6.8.5 coord:dataInPartitionMin(String name, String partition) EL function
-The =${coord:dataInPartitionMin(String name, String partition)}= EL function resolves to the *minimum* value of the
+The `${coord:dataInPartitionMin(String name, String partition)}` EL function resolves to the **minimum** value of the
specified partition for all the dataset instances specified in an input event dataset section. It can be used to do
range based filtering of partitions in pig scripts together
-with [[CoordinatorFunctionalSpec#DataInPartitionMax][dataInPartitionMax]] EL function.
+with [dataInPartitionMax](CoordinatorFunctionalSpec.html#DataInPartitionMax) EL function.
-Refer to the [[CoordinatorFunctionalSpec#HCatPigExampleTwo][Example]] below for usage.
+Refer to the [Example](CoordinatorFunctionalSpec.html#HCatPigExampleTwo) below for usage.
-#DataInPartitionMax
----++++ 6.8.6 coord:dataInPartitionMax(String name, String partition) EL function
+<a name="DataInPartitionMax"></a>
+#### 6.8.6 coord:dataInPartitionMax(String name, String partition) EL function
-The =${coord:dataInPartitionMax(String name, String partition)}= EL function resolves to the *maximum* value of the
+The `${coord:dataInPartitionMax(String name, String partition)}` EL function resolves to the **maximum** value of the
specified partition for all the dataset instances specified in an input event dataset section. It is a better practice
-to use =dataInPartitionMin= and =dataInPartitionMax= to form a range filter wherever possible instead
-of =datainPartitionPigFilter= as it will be more efficient for filtering.
+to use `dataInPartitionMin` and `dataInPartitionMax` to form a range filter wherever possible instead
+of `datainPartitionPigFilter` as it will be more efficient for filtering.
-Refer to the [[CoordinatorFunctionalSpec#HCatPigExampleTwo][Example]] below for usage.
+Refer to the [Example](CoordinatorFunctionalSpec.html#HCatPigExampleTwo) below for usage.
----++++ 6.8.7 coord:dataOutPartitionValue(String name, String partition) EL function
+#### 6.8.7 coord:dataOutPartitionValue(String name, String partition) EL function
-The =${coord:dataOutPartitionValue(String name, String partition)}= EL function resolves to value of the specified
+The `${coord:dataOutPartitionValue(String name, String partition)}` EL function resolves to value of the specified
partition for the output-event dataset; that will be consumed by a workflow job, e.g Pig job triggered by a
coordinator action. This is another convenience function to use a single partition-key's value if required, in
-addition to =dataoutPartitionsPig()= and either one can be used.
+addition to `dataoutPartitionsPig()` and either one can be used.
The example below illustrates a pig job triggered by a coordinator, using the aforementioned EL functions for input
partition max/min values, output partition value, and database and table.
-*%GREEN% Example: %ENDCOLOR%*
+**<font color="#008000"> Example: </font>**
-#HCatPigExampleTwo
+<a name="HCatPigExampleTwo"></a>
-*Coordinator application definition:*
+**Coordinator application definition:**
-<blockquote>
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -3159,27 +3271,27 @@
</workflow>
</action>
</coordinator-app>
-</blockquote>
+```
In this example, each coordinator action will use as input events the last 24 hourly instances of the 'logs' dataset.
-For the =2009-01-02T00:00Z= run, the =${coord:dataInPartitionMin('raw-logs','datestamp')}= function will resolve to
+For the `2009-01-02T00:00Z` run, the `${coord:dataInPartitionMin('raw-logs','datestamp')}` function will resolve to
the minimum of the 5 dataset instances for partition 'datestamp'
i.e. among 2009010101, 2009010102, ...., 2009010123, 2009010200, the minimum would be "2009010101".
-Similarly, the =${coord:dataInPartitionMax('raw-logs','datestamp')}= function will resolve to the maximum of the 5
+Similarly, the `${coord:dataInPartitionMax('raw-logs','datestamp')}` function will resolve to the maximum of the 5
dataset instances for partition 'datestamp'
i.e. among 2009010120, 2009010121, ...., 2009010123, 2009010200, the maximum would be "2009010200".
-Finally, the =${coord:dataOutPartitionValue(String name, String partition)}= function enables the coordinator
+Finally, the `${coord:dataOutPartitionValue(String name, String partition)}` function enables the coordinator
application to pass a specified partition's value string needed by the HCatStorer in Pig job.
-The =${coord:dataOutPartitionValue('processed-logs','region')}= function will resolve to: "${region}"
-and =${coord:dataOutPartitionValue('processed-logs','datestamp')}= function will resolve to: "20090102".
+The `${coord:dataOutPartitionValue('processed-logs','region')}` function will resolve to: "${region}"
+and `${coord:dataOutPartitionValue('processed-logs','datestamp')}` function will resolve to: "20090102".
-For the workflow definition with <pig> action, refer to [[CoordinatorFunctionalSpec#HCatWorkflow][previous example]],
+For the workflow definition with \<pig\> action, refer to [previous example](CoordinatorFunctionalSpec.html#HCatWorkflow),
with the following change in pig params in addition to database and table.
-<blockquote>
+```
...
<param>PARTITION_DATE_MIN=${DATE_MIN}</param>
<param>PARTITION_DATE_MAX=${DATE_MAX}</param>
@@ -3187,32 +3299,32 @@
<param>OUT_PARTITION_VAL_REGION=${OUT_PARTITION_VAL_REGION}</param>
<param>OUT_PARTITION_VAL_DATE=${OUT_PARTITION_VAL_DATE}</param>
...
-</blockquote>
+```
-*Example usage in Pig:*
+**Example usage in Pig:**
This illustrates another pig script which filters partitions based on range, with range limits parameterized with the
EL functions
-<blockquote>
+```
A = load '$HCAT_IN_DB.$HCAT_IN_TABLE' using org.apache.hive.hcatalog.pig.HCatLoader();
B = FILTER A BY datestamp >= '$PARTITION_DATE_MIN' AND datestamp < '$PARTITION_DATE_MAX' AND region=='$REGION';
C = foreach B generate foo, bar;
store C into '$HCAT_OUT_DB.$HCAT_OUT_TABLE' using org.apache.hive.hcatalog.pig.HCatStorer('region=$OUT_PARTITION_VAL_REGION,datestamp=$OUT_PARTITION_VAL_DATE');
-</blockquote>
+```
For example,
-for the =2009-01-02T00:00Z= run with the given dataset instances, the above Pig script with resolved values would look like:
+for the `2009-01-02T00:00Z` run with the given dataset instances, the above Pig script with resolved values would look like:
-<blockquote>
+```
A = load 'myInputDatabase.myInputTable' using org.apache.hive.hcatalog.pig.HCatLoader();
B = FILTER A BY datestamp >= '2009010101' AND datestamp < '2009010200' AND region='APAC';
C = foreach B generate foo, bar;
store C into 'myOutputDatabase.myOutputTable' using org.apache.hive.hcatalog.pig.HCatStorer('region=APAC,datestamp=20090102');
-</blockquote>
+```
----++++ 6.8.8 coord:dataInPartitions(String name, String type) EL function
+#### 6.8.8 coord:dataInPartitions(String name, String type) EL function
-The =${coord:dataInPartitions(String name, String type)}= EL function resolves to a list of partition key-value
+The `${coord:dataInPartitions(String name, String type)}` EL function resolves to a list of partition key-value
pairs for the input-event dataset. Currently the only type supported is 'hive-export'. The 'hive-export' type
supports only one partition instance and it can be used to create the complete partition value string that can
be used in a hive query for partition export/import.
@@ -3220,13 +3332,13 @@
The example below illustrates a hive export-import job triggered by a coordinator, using the EL functions for HCat database,
table, input partitions. The example replicates the hourly processed data across hive tables.
-*%GREEN% Example: %ENDCOLOR%*
+**<font color="#008000"> Example: </font>**
-#HCatHiveExampleOne
+<a name="HCatHiveExampleOne"></a>
-*Coordinator application definition:*
+**Coordinator application definition:**
-<blockquote>
+```
<coordinator-app xmlns="uri:oozie:coordinator:0.3" name="app-coord"
frequency="${coord:hours(1)}" start="2014-03-28T08:00Z"
end="2030-01-01T00:00Z" timezone="UTC">
@@ -3285,23 +3397,23 @@
</workflow>
</action>
</coordinator-app>
-</blockquote>
+```
Parameterizing the input/output databases and tables using the corresponding EL function as shown will make them
available in the hive action of the workflow 'logsreplicator-wf'.
Each coordinator action will use as input events the hourly instances of the 'processed-logs-1' dataset. The
-=${coord:dataInPartitions(String name, String type)}= function enables the coordinator application to pass the
+`${coord:dataInPartitions(String name, String type)}` function enables the coordinator application to pass the
partition corresponding to hourly dataset instances to the workflow job triggered by the coordinator action.
The workflow passes this partition value to the hive export script that exports the hourly partition from source
-database to the staging location referred as =EXPORT_PATH=. The hive import script imports the hourly partition from
-=EXPORT_PATH= staging location into the target database.
+database to the staging location referred as `EXPORT_PATH`. The hive import script imports the hourly partition from
+`EXPORT_PATH` staging location into the target database.
-#HCatWorkflow
+<a name="HCatWorkflow"></a>
-*Workflow definition:*
+**Workflow definition:**
-<blockquote>
+```
<workflow-app xmlns="uri:oozie:workflow:0.3" name="logsreplicator-wf">
<start to="table-export"/>
<action name="table-export">
@@ -3359,64 +3471,65 @@
</kill>
<end name="end"/>
</workflow-app>
-</blockquote>
+```
Ensure that the following jars are in classpath, with versions corresponding to hcatalog installation:
hcatalog-core.jar, webhcat-java-client.jar, hive-common.jar, hive-exec.jar, hive-metastore.jar, hive-serde.jar,
libfb303.jar. The hive-site.xml needs to be present in classpath as well.
-*Example Hive Export script:*
+**Example Hive Export script:**
The following script exports a particular Hive table partition into staging location, where the partition value
- is computed through =${coord:dataInPartitions(String name, String type)}= EL function.
-<blockquote>
+ is computed through `${coord:dataInPartitions(String name, String type)}` EL function.
+```
export table ${sourceDatabase}.${sourceTable} partition (${sourcePartition}) to '${sourceStagingDir}';
-</blockquote>
+```
For example, for the 2014-03-28T08:00Z run with the given dataset instances and ${coord:dataInPartitions(
'processed-logs-1', 'hive-export'), the above Hive script with resolved values would look like:
-<blockquote>
+```
export table myInputDatabase1/myInputTable1 partition (year='2014',month='03',day='28',hour='08') to 'hdfs://bar:8020/staging/2014-03-28-08';
-</blockquote>
+```
-*Example Hive Import script:*
+**Example Hive Import script:**
The following script imports a particular Hive table partition from staging location, where the partition value is computed
- through =${coord:dataInPartitions(String name, String type)}= EL function.
-<blockquote>
+ through `${coord:dataInPartitions(String name, String type)}` EL function.
+```
use ${targetDatabase};
alter table ${targetTable} drop if exists partition ${targetPartition};
import table ${targetTable} partition (${targetPartition}) from '${sourceStagingDir}';
-</blockquote>
+```
For example, for the 2014-03-28T08:00Z run with the given dataset instances and ${coord:dataInPartitions(
'processed-logs-2', 'hive-export'), the above Hive script with resolved values would look like:
-<blockquote>
+```
use myInputDatabase2;
alter table myInputTable2 drop if exists partition (year='2014',month='03',day='28',hour='08');
import table myInputTable2 partition (year='2014',month='03',day='28',hour='08') from 'hdfs://bar:8020/staging/2014-03-28-08';
-</blockquote>
+```
----+++ 6.9. Parameterization of Coordinator Application
+### 6.9. Parameterization of Coordinator Application
This section describes the EL functions that could be used to parameterized both data-set and coordination application action.
----++++ 6.9.1. coord:dateOffset(String baseDate, int instance, String timeUnit) EL Function
+#### 6.9.1. coord:dateOffset(String baseDate, int instance, String timeUnit) EL Function
-The =${coord:dateOffset(String baseDate, int instance, String timeUnit)}= EL function calculates the date based on the following
-equation : =newDate = baseDate + (instance * timeUnit)=
-In other words, it offsets the =baseDate= by the amount specified by =instance= and =timeUnit=.
+The `${coord:dateOffset(String baseDate, int instance, String timeUnit)}` EL function calculates the date based on the following
+equation : `newDate = baseDate + (instance * timeUnit)`
+In other words, it offsets the `baseDate` by the amount specified by `instance` and `timeUnit`.
-The =timeUnit= argument accepts one of 'DAY', 'MONTH', 'HOUR', 'MINUTE', 'MONTH'
+The `timeUnit` argument accepts one of 'DAY', 'MONTH', 'HOUR', 'MINUTE', 'MONTH'
-For example, if =baseDate= is '2009-01-01T00:00Z', =instance= is '2' and =timeUnit= is 'MONTH', the return date will be
-'2009-03-01T00:00Z'. If =baseDate= is '2009-01-01T00:00Z', =instance= is '1' and =timeUnit= is 'YEAR', the return date will be
+For example, if `baseDate` is '2009-01-01T00:00Z', `instance` is '2' and `timeUnit` is 'MONTH', the return date will be
+'2009-03-01T00:00Z'. If `baseDate` is '2009-01-01T00:00Z', `instance` is '1' and `timeUnit` is 'YEAR', the return date will be
'2010-01-01T00:00Z'.
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="2009-01-01T23:00Z" end="2009-12-31T23:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -3437,30 +3550,31 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
In this example, the 'nextInstance' will be '2009-01-02T23:00Z' for the first action. And the value of 'previousInstance' will be
'2008-12-31T23:00Z' for the same instance.
----++++ 6.9.2. coord:dateTzOffset(String baseDate, String timezone) EL Function
+#### 6.9.2. coord:dateTzOffset(String baseDate, String timezone) EL Function
-The =${coord:dateTzOffset(String baseDate, String timezone)}= EL function calculates the date based on the following
-equation : =newDate = baseDate + (Oozie processing timezone - timezone)=
-In other words, it offsets the =baseDate= by the difference from Oozie processing timezone to the given =timezone=. It will
-account for daylight saving time based on the given =baseDate= and =timezone=.
+The `${coord:dateTzOffset(String baseDate, String timezone)}` EL function calculates the date based on the following
+equation : `newDate = baseDate + (Oozie processing timezone - timezone)`
+In other words, it offsets the `baseDate` by the difference from Oozie processing timezone to the given `timezone`. It will
+account for daylight saving time based on the given `baseDate` and `timezone`.
-The =timezone= argument accepts any timezone or GMT offset that is returned by the
-[[DG_CommandLineTool#Getting_a_list_of_time_zones]["info -timezones"]] command. For example, "America/Los_Angeles".
+The `timezone` argument accepts any timezone or GMT offset that is returned by the
+["info -timezones"](DG_CommandLineTool.html#Getting_a_list_of_time_zones) command. For example, "America/Los_Angeles".
-For example, if =baseDate= is '2012-06-13T00:00Z' and =timezone= is 'America/Los_Angeles', the return date will be
-'2012-06-12T17:00Z'. But if =baseDate= is '2012-12-13T00:00Z', then the return date will be '2012-12-12T16:00Z'. The difference
+For example, if `baseDate` is '2012-06-13T00:00Z' and `timezone` is 'America/Los_Angeles', the return date will be
+'2012-06-12T17:00Z'. But if `baseDate` is '2012-12-13T00:00Z', then the return date will be '2012-12-12T16:00Z'. The difference
in return dates occurs because the former occurs during Summer when DST is in effect (UTC-0700) and the latter occurs during Winter
when DST is not in effect (UTC-0800).
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="2009-01-01T24:00Z" end="2009-12-31T24:00Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -3477,21 +3591,21 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
In this example, the 'myDate' will be '2009-01-01T15:00Z' for the first action.
----++++ 6.9.3. coord:formatTime(String ts, String format) EL Function (since Oozie 2.3.2)
+#### 6.9.3. coord:formatTime(String ts, String format) EL Function (since Oozie 2.3.2)
-The =${coord:formatTime(String timeStamp, String format)}= function allows transformation of the standard ISO8601 timestamp strings into other desired formats.
+The `${coord:formatTime(String timeStamp, String format)}` function allows transformation of the standard ISO8601 timestamp strings into other desired formats.
-The format string should be in Java's [[http://download.oracle.com/javase/6/docs/api/java/text/SimpleDateFormat.html][SimpleDateFormat]] format.
+The format string should be in Java's [SimpleDateFormat](http://download.oracle.com/javase/6/docs/api/java/text/SimpleDateFormat.html) format.
For example, if timeStamp is '2009-01-01T00:00Z' and format is 'yyyy', the returned date string will be '2009'.
----++++ 6.9.4. coord:epochTime(String ts, String millis) EL Function (since Oozie 4.3)
+#### 6.9.4. coord:epochTime(String ts, String millis) EL Function (since Oozie 4.3)
-The =${coord:epochTime(String timeStamp, String millis)}= function allows transformation of the standard ISO8601 timestamp
+The `${coord:epochTime(String timeStamp, String millis)}` function allows transformation of the standard ISO8601 timestamp
strings into Unix epoch time (seconds or milliseconds since January 1, 1970).
If millis is 'false', the returned time string will be the number of seconds since the epoch. If 'true', the returned time string
@@ -3500,41 +3614,45 @@
For example, if timeStamp is '2009-01-01T00:00Z' and millis is 'false', the returned date string will be '1230768000'. If millis
is 'true', the returned date string will be '1230768000000'.
----+++ 6.10. Conditional coordinator input logic
+### 6.10. Conditional coordinator input logic
By default, all input dependencies are "AND", which means all dependencies has to be available before the action starts running.
With conditional input logic, one should able to specify conditional operations among multiple datasets.
Supported operators are OR, AND, COMBINE. OR and AND operators are nested, one can form multiple nested expressions using them.
+
* OR: Logical OR, where an expression will evaluate to true if one of the datasets is available.
* AND: Logical AND, where an expression will evaluate to true when all of the datasets are available.
* COMBINE : With combine, instances of A and B can be interleaved to get the final "combined" set of total instances. All datasets in combine should have the same range defined with the current EL function. Combine does not support latest and future EL functions. Combine cannot also be nested.
Additional options
- * *%BLUE% MIN: %ENDCOLOR%* Minimum number of input instances that should be available. This can be used in cases where inputs are optional and the processing can be done on a subset of input instances.
- * %BLUE% WAIT (in minutes): %ENDCOLOR% Wait is used to specify the amount of time to continue checking for availability of instances of a particular dataset before moving on to the next dataset defined in a OR condition. The wait time is calculated from the beginning of the nominal time of the action or the action creation time whichever is later. The main purpose of this is to give preference to the primary datasource before checking the secondary datasource.
+
+ * **<font color="#0000ff"> MIN: </font>** Minimum number of input instances that should be available. This can be used in cases where inputs are optional and the processing can be done on a subset of input instances.
+ * <font color="#0000ff"> WAIT (in minutes): </font> Wait is used to specify the amount of time to continue checking for availability of instances of a particular dataset before moving on to the next dataset defined in a OR condition. The wait time is calculated from the beginning of the nominal time of the action or the action creation time whichever is later. The main purpose of this is to give preference to the primary datasource before checking the secondary datasource.
Wait when used with min option has a totally different purpose. It is used to specify the additional amount of time to wait and check for more instances after the required minimum set of instances become available. Any additional instances that become available during the wait time are then included.
-The conditional logic can be specified using the <input-logic> tag in the coordinator.xml using the [[CoordinatorFunctionalSpec#Oozie_Coordinator_Schema_0.5][Oozie Coordinator Schema 0.5]] and above. If not specified, the default behavior of "AND" of all defined input dependencies is applied.
+The conditional logic can be specified using the \<input-logic\> tag in the coordinator.xml using the [Oozie Coordinator Schema 0.5](CoordinatorFunctionalSpec.html#Oozie_Coordinator_Schema_0.5) and above. If not specified, the default behavior of "AND" of all defined input dependencies is applied.
Order of definition of the dataset matters. Availability of inputs is checked in that order. Only if input instances of the first dataset is not available, then the input instances of the second dataset will be checked and so on. In the case of AND or OR, the second dataset is picked only if the first dataset does not meet all the input dependencies first. In the case of COMBINE, only the input instances missing on the first dataset are checked for availability on the other datasets in order and then included.
coord:dataIn() function can be used to get the comma separated list of evaluated hdfs paths given the name of the conditional operator.
-*%GREEN% Example: %ENDCOLOR%*:
-<verbatim>
+**<font color="#008000"> Example: </font>**:
+
+```
<input-logic>
<or name="AorB">
<data-in dataset="A"/>
<data-in dataset="B"/>
</or>
</input-logic>
-</verbatim>
+```
With above expression one can specify the dataset as AorB. Action will start running as soon dataset A or B is available. Dataset "A" has higher precedence over "B" because it is defined first. Oozie will first check for availability of dataset A and only if A is not available, availability of dataset B will be checked.
-*%GREEN% Example: %ENDCOLOR%*:
-<verbatim>
+**<font color="#008000"> Example: </font>**:
+
+```
<input-logic>
<or name="AorBorC">
<data-in dataset="A" wait = "60"/>
@@ -3542,11 +3660,12 @@
<data-in dataset="C"/>
</or>
</input-logic>
-</verbatim>
+```
With the above expression, it will wait for 60 mins from the nominal time of the action or the action creation time whichever is later for all the instances of dataset A to be available. If it is not available in 60 minutes, then it will start checking for instances of dataset B. If instances of B are not available in another 30 minutes, then it will start checking for dataset C.
-*%GREEN% Example: %ENDCOLOR%*:
-<verbatim>
+**<font color="#008000"> Example: </font>**:
+
+```
<datasets>
<dataset name="dataset_a" frequency="${coord:minutes(20)}" initial-instance="2010-01-01T00:00Z" timezone="UTC">
<uri-template>${nameNode}/user/${coord:user()}/${examplesRoot}/input-data/rawLogs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}</uri-template>
@@ -3575,62 +3694,68 @@
</and>
</or>
</input-logic>
-</verbatim>
+```
Action will start running as soon as dependency A and B or C and D are available.
-*%GREEN% Example: %ENDCOLOR%*:
-<verbatim>
+**<font color="#008000"> Example: </font>**:
+
+```
<input-logic>
<combine name="AorB">
<data-in dataset="A"/>
<data-in dataset="B"/>
</combine>
</input-logic>
-</verbatim>
+```
Combine function will first check instances from A and whatever is missing it will check from B.
-*%GREEN% Example: %ENDCOLOR%*:
-<verbatim>
+**<font color="#008000"> Example: </font>**:
+
+```
<input-logic>
<data-in dataset="A" min=2/>
</input-logic>
-</verbatim>
+```
Action will start running if available dependencies >= 2.
-*%GREEN% Example: %ENDCOLOR%*:
-<verbatim>
+**<font color="#008000"> Example: </font>**:
+
+```
<input-logic>
<or name="AorB" min=2>
<data-in dataset="A"/>
<data-in dataset="B"/>
</or>
</input-logic>
-</verbatim>
+```
Action will start running if A has available dependencies >= 2 or B has available dependencies >= 2
-*%GREEN% Example: %ENDCOLOR%*:
-<verbatim>
+**<font color="#008000"> Example: </font>**:
+
+```
<input-logic>
<or name="AorB" min="2">
<data-in dataset="A" wait="10"/>
<data-in dataset="B"/>
</or>
</input-logic>
-</verbatim>
+```
After the mininum two dependencies are available, processing will wait for additional 10 minutes to include any dependencies that become available during that period.
-<verbatim>
+
+```
<input-logic>
<or name="AorB" min="5" wait="10">
<data-in dataset="A"/>
<data-in dataset="B"/>
</or>
</input-logic>
-</verbatim>
+```
MIN and WAIT can be used at parent level, which will get propagated to child node. Above expression is equivalent to dataset A with min = 2 and wait = 10 minutes and dataset B with min = 2 and wait = 10 minutes.
-*%GREEN% Example: %ENDCOLOR%*:
-<verbatim>
+**<font color="#008000"> Example: </font>**:
+
+```
<input-logic>
<or>
<and name="AorB">
@@ -3658,43 +3783,45 @@
</configuration>
</workflow>
</action>
-</verbatim>
+```
Each nested operation can be named and passed into the workflow using coord:dataIn().
----++ 7. Handling Timezones and Daylight Saving Time
+## 7. Handling Timezones and Daylight Saving Time
As mentioned in section #4.1.1 'Timezones and Daylight-Saving', the coordinator engine works exclusively in UTC, and dataset and application definitions are always expressed in UTC.
-*%GREEN% Example of nominal times in case of DST change: %ENDCOLOR%*
+**<font color="#008000"> Example of nominal times in case of DST change: </font>**
-| *Frequency* | *Timezone* | *Nominal times in local time* | *Comments* |
-| =${coord:months(1)}= or =${10 23 1 1-12 *}= | America/Los_Angeles | 2016-03-01T15:10 <br/> 2016-04-01T15:10 <br/> 2016-05-01T15:10 <br/> ... <br/> 2016-11-01T15:10 <br/> 2016-12-01T15:10 | <br/>DST Start on March 13, 2:00 am <br/><br/><br/><br/> DST End on November 6, 2:00 am|
-| =${coord:month(3)} or =${10 13 1 */3 *}= | America/Los_Angeles |2016-01-01T05:10 <br/> 2016-04-01T05:10 <br/> 2016-07-01T05:10 <br/> 2016-10-01T05:10 <br/> 2017-01-01T05:10 <br/> 2017-04-01T05:10 <br/> 2017-07-01T05:10 | <br/> DST Start on 2016 March 13, 2:00 am <br/><br/><br/>DST End on 2016 November 6, 2:00 am <br/> DST Start on 2017 March 12, 2:00 am|
-| =${coord:days(20)}=| America/Los_Angeles | 2016-03-12T05:10 <br/> 2016-04-01T05:10 <br/> 2016-04-21T05:10 <br/> ... <br/> 2016-11-07T05:10 <br/> 2016-11-27T05:10 |<br/> DST Start on March 13, 2:00 am <br/><br/><br/> DST End on November 6, 2:00 am|
-| =${10 13 */20 * *}= | America/Los_Angeles | 2016-03-01T05:10 <br/> 2016-03-21T05:10 <br/> 2016-11-01T05:10 <br/> 2016-11-21T05:10 <br/> 2016-12-01T05:10 | <br/> DST Start on March 13, 2:00 am <br/><br/> DST End on November 6, 2:00|
-| =${coord:days(1)}= or =${10 23 * * *}= | America/Los_Angeles | 2016-03-11T15:10 <br/> 2016-03-12T15:10 <br/> 2016-03-13T15:10 <br/> 2016-03-14T15:10 | <br/> DST Start on March 13, 2:00 am|
-| =${coord:hours(24)}=| America/Los_Angeles | 2016-03-11T15:10 <br/> 2016-03-12T15:10 <br/> 2016-03-13T16:10 <br/> 2016-03-14T16:10 | <br/><br/> DST Start on March 13, 2:00 am, but since the time unit is in hours, there will be a shift in local time|
-| =${coord:hours(1)}= or =${10 * * * *}= | America/Los_Angeles | 2017-03-12T00:10 <br/> 2017-03-12T01:10 <br/> 2017-03-12T03:10 <br/> 2017-03-12T04:10 | <br/><br/> DST Start on March 12, 2:00 am, so hour 2 will be skipped|
-| =${coord:hours(1)}= or =${10 * * * *}= | America/Los_Angeles | 2017-11-05T00:10 <br/> 2017-11-05T01:10 <br/> 2017-11-05T01:10 <br/> 2017-11-05T02:10 <br/> 2017-11-05T03:10 | <br/><br/> DST End on November 5, 2:00 am, so hour 1 will be doubled|
-| =${10 */20 12-14 3 *}= | America/Los_Angeles | 2016-03-12T12:10 <br/> 2016-03-12T16:10 <br/> 2016-03-13T13:10 <br/> 2016-03-13T17:10 <br/> 2016-03-14T13:10 <br/> ... <br/> 2016-11-05T17:10 <br/> 2016-11-06T12:10 <br/> 2016-11-06T16:10 | <br/> <br/> DST Start on March 13, 2:00 am, so after this time the nominal times will be shifted <br/><br/><br/> <br/> DST End on November 6, 2:00 am|
-| =${coord:hours(20)}= | America/Los_Angeles |2016-03-12T05:10 <br/> 2016-03-13T01:10 <br/> 2016-03-13T22:10 <br/> 2016-03-14T18:10 <br/> 2016-03-15T14:10 <br/> ... <br/> 2016-11-05T12:10 <br/> 2016-11-06T07:10 <br/> 2016-11-07T03:10 <br/> 2016-11-07T23:10 | <br/><br/> DST Start on March 13, 2:00, so here will be 21 hours in local time between the two materialization times <br/><br/><br/><br/><br/> DST End on November 6, 2:00 am, so here will be a 19 hour difference in local time|
-| =${coord:minutes(30)}= or =${*/30 * * * *}= | America/Los_Angeles | 2016-03-13T01:00 <br/> 2016-03-13T01:30 <br/> 2016-03-13T02:00 <br/> 2016-03-13T02:30 <br/> 2016-03-13T04:00 <br/> 2016-03-13T04:30 | <br/><br/><br/> DST Start on March 13, 2:00 am|
+| **Frequency** | **Timezone** | **Nominal times in local time** | **Comments** |
+| --- | --- | --- | --- |
+| `${coord:months(1)}` or `${10 23 1 1-12 *}` | America/Los_Angeles | 2016-03-01T15:10 <br/> 2016-04-01T15:10 <br/> 2016-05-01T15:10 <br/> ... <br/> 2016-11-01T15:10 <br/> 2016-12-01T15:10 | <br/>DST Start on March 13, 2:00 am <br/><br/><br/><br/> DST End on November 6, 2:00 am|
+| `${coord:month(3)} or `${10 13 1 **/3 **}= | America/Los_Angeles |2016-01-01T05:10 <br/> 2016-04-01T05:10 <br/> 2016-07-01T05:10 <br/> 2016-10-01T05:10 <br/> 2017-01-01T05:10 <br/> 2017-04-01T05:10 <br/> 2017-07-01T05:10 | <br/> DST Start on 2016 March 13, 2:00 am <br/><br/><br/>DST End on 2016 November 6, 2:00 am <br/> DST Start on 2017 March 12, 2:00 am|
+| `${coord:days(20)}`| America/Los_Angeles | 2016-03-12T05:10 <br/> 2016-04-01T05:10 <br/> 2016-04-21T05:10 <br/> ... <br/> 2016-11-07T05:10 <br/> 2016-11-27T05:10 |<br/> DST Start on March 13, 2:00 am <br/><br/><br/> DST End on November 6, 2:00 am|
+| `${10 13 **/20 ** *}` | America/Los_Angeles | 2016-03-01T05:10 <br/> 2016-03-21T05:10 <br/> 2016-11-01T05:10 <br/> 2016-11-21T05:10 <br/> 2016-12-01T05:10 | <br/> DST Start on March 13, 2:00 am <br/><br/> DST End on November 6, 2:00|
+| `${coord:days(1)}` or `${10 23 ** ** *}` | America/Los_Angeles | 2016-03-11T15:10 <br/> 2016-03-12T15:10 <br/> 2016-03-13T15:10 <br/> 2016-03-14T15:10 | <br/> DST Start on March 13, 2:00 am|
+| `${coord:hours(24)}`| America/Los_Angeles | 2016-03-11T15:10 <br/> 2016-03-12T15:10 <br/> 2016-03-13T16:10 <br/> 2016-03-14T16:10 | <br/><br/> DST Start on March 13, 2:00 am, but since the time unit is in hours, there will be a shift in local time|
+| `${coord:hours(1)}` or `${10 ** ** ** **}` | America/Los_Angeles | 2017-03-12T00:10 <br/> 2017-03-12T01:10 <br/> 2017-03-12T03:10 <br/> 2017-03-12T04:10 | <br/><br/> DST Start on March 12, 2:00 am, so hour 2 will be skipped|
+| `${coord:hours(1)}` or `${10 ** ** ** **}` | America/Los_Angeles | 2017-11-05T00:10 <br/> 2017-11-05T01:10 <br/> 2017-11-05T01:10 <br/> 2017-11-05T02:10 <br/> 2017-11-05T03:10 | <br/><br/> DST End on November 5, 2:00 am, so hour 1 will be doubled|
+| `${10 **/20 12-14 3 **}` | America/Los_Angeles | 2016-03-12T12:10 <br/> 2016-03-12T16:10 <br/> 2016-03-13T13:10 <br/> 2016-03-13T17:10 <br/> 2016-03-14T13:10 <br/> ... <br/> 2016-11-05T17:10 <br/> 2016-11-06T12:10 <br/> 2016-11-06T16:10 | <br/> <br/> DST Start on March 13, 2:00 am, so after this time the nominal times will be shifted <br/><br/><br/> <br/> DST End on November 6, 2:00 am|
+| `${coord:hours(20)}` | America/Los_Angeles |2016-03-12T05:10 <br/> 2016-03-13T01:10 <br/> 2016-03-13T22:10 <br/> 2016-03-14T18:10 <br/> 2016-03-15T14:10 <br/> ... <br/> 2016-11-05T12:10 <br/> 2016-11-06T07:10 <br/> 2016-11-07T03:10 <br/> 2016-11-07T23:10 | <br/><br/> DST Start on March 13, 2:00, so here will be 21 hours in local time between the two materialization times <br/><br/><br/><br/><br/> DST End on November 6, 2:00 am, so here will be a 19 hour difference in local time|
+| `${coord:minutes(30)}` or `${**/30 ** ** ** *}` | America/Los_Angeles | 2016-03-13T01:00 <br/> 2016-03-13T01:30 <br/> 2016-03-13T02:00 <br/> 2016-03-13T02:30 <br/> 2016-03-13T04:00 <br/> 2016-03-13T04:30 | <br/><br/><br/> DST Start on March 13, 2:00 am|
-*IMPORTANT:* Please note, that in the actual implementation, DST corrections are not applied in case of higher frequencies than one day, so for this frequencies, some shifting in nominal times are expected.
+**IMPORTANT:** Please note, that in the actual implementation, DST corrections are not applied in case of higher frequencies than one day, so for this frequencies, some shifting in nominal times are expected.
----+++ 7.1. Handling Timezones with No Day Light Saving Time
+### 7.1. Handling Timezones with No Day Light Saving Time
For timezones that don't observe day light saving time, handling timezones offsets is trivial.
For these timezones, dataset and application definitions, it suffices to express datetimes taking into account the timezone offset.
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
Coordinator application definition: A daily coordinator job for India timezone (+05:30) that consumes 24 hourly dataset instances from the previous day starting at the beginning of 2009 for a full year.
-<verbatim>
+
+```
<coordinator-app name="app-coord" frequency="${coord:days(1)}"
start="2008-12-31T19:30Z" end="2009-12-30T19:30Z" timezone="UTC"
xmlns="uri:oozie:coordinator:0.1">
@@ -3714,9 +3841,9 @@
...
</action>
</coordinator-app>
-</verbatim>
+```
----+++ 7.2. Handling Timezones with Daylight Saving Time
+### 7.2. Handling Timezones with Daylight Saving Time
Oozie Coordinator provides all the necessary functionality to write coordinator applications that work properly when data and processing spans across multiple timezones and different daylight saving rules.
@@ -3725,9 +3852,10 @@
1 Process logs hourly data from the last day from US East-coast
1 Process logs hourly data from the last day from US East-coast and Continental Europe
-*1. Process logs hourly data from the last day from US East-coast:*
+**1. Process logs hourly data from the last day from US East-coast:**
-<verbatim>
+
+```
<coordinator-app name="eastcoast-processing" frequency="${coord:days(1)}"
start="2009-01-02T05:00Z" end="2010-01-02T05:00Z" timezone="America/New_York"
xmlns="uri:oozie:coordinator:0.1">
@@ -3757,19 +3885,20 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
-Because the =${coord:days(1)}= EL function is used to specify the job frequency, each coordinator action will be materialized (created) at 00:00 EST5EDT regardless of timezone daylight-saving adjustments (05:00 UTC in Winter and 04:00 UTC in Summer)
+Because the `${coord:days(1)}` EL function is used to specify the job frequency, each coordinator action will be materialized (created) at 00:00 EST5EDT regardless of timezone daylight-saving adjustments (05:00 UTC in Winter and 04:00 UTC in Summer)
-The =${coord:hoursInDay(-1)}= EL function will resolve to number of hours of the previous day taking into account daylight-saving changes if any. It will resolve to =24= (on regular days), =23= (on spring forward day) or =25= (on fall backward day).
+The `${coord:hoursInDay(-1)}` EL function will resolve to number of hours of the previous day taking into account daylight-saving changes if any. It will resolve to `24` (on regular days), `23` (on spring forward day) or `25` (on fall backward day).
-Because of the use of the =${coord:hoursInDay(-1)}= EL function, the dataset instances range resolves [-24 .. -1], [-23 .. -1] or [-25 .. -1]. Thus, they will resolve into the exact number of dataset instances for the day taking daylight-saving adjustments into account.
+Because of the use of the `${coord:hoursInDay(-1)}` EL function, the dataset instances range resolves [-24 .. -1], [-23 .. -1] or [-25 .. -1]. Thus, they will resolve into the exact number of dataset instances for the day taking daylight-saving adjustments into account.
Note that because the coordinator application and the dataset are in the same timezone, there is no need to do any hour offset corrections in the dataset instances being used as input for each coordinator action.
-*2. Process logs hourly data from the last day from US East-coast and the US West-coast:*
+**2. Process logs hourly data from the last day from US East-coast and the US West-coast:**
-<verbatim>
+
+```
<coordinator-app name="eastcoast-europe-processing" frequency="${coord:days(1)}"
start="2009-01-02T09:00Z" end="2010-01-02T09:00Z" timezone="America/Los_Angeles"
xmlns="uri:oozie:coordinator:0.1">
@@ -3809,7 +3938,7 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
The additional complexity of this use case over the first use case is because the job and the datasets are not all in the same timezone. The corresponding timezone offset has to accounted for.
@@ -3817,9 +3946,10 @@
The data input range for the East coast dataset must be adjusted (with -3) in order to take the data for the previous EST5EDT day.
-*3. Process logs hourly data from the last day from US East-coast and Continental Europe:*
+**3. Process logs hourly data from the last day from US East-coast and Continental Europe:**
-<verbatim>
+
+```
<coordinator-app name="eastcoast-europe-processing" frequency="${coord:days(1)}"
start="2009-01-02T05:00Z" end="2010-01-02T05:00Z" timezone="America/New_York"
xmlns="uri:oozie:coordinator:0.1">
@@ -3859,32 +3989,32 @@
</workflow>
</action>
</coordinator-app>
-</verbatim>
+```
The additional complexity of this use case over the second use case is because the timezones used for the job and the datasets do not follow the same daylight saving rules (Europe and the US apply the DST changes on different days).
-Because of this, the timezone offset between Europe and the US is not constant. To obtain the current timezone offset between the coordinator job and a dataset, the =${coord:tzOffset()}= EL function must be used.
+Because of this, the timezone offset between Europe and the US is not constant. To obtain the current timezone offset between the coordinator job and a dataset, the `${coord:tzOffset()}` EL function must be used.
As the use care requires to process all the daily data for the East coast and the continental Europe, the processing happens on East coast time (thus having daily data already available for both Europe and the East coast).
-The data input range for the Europe dataset must be adjusted with the =${coord:tzOffset()}= EL function in order to take the data for the previous EST5EDT day.
+The data input range for the Europe dataset must be adjusted with the `${coord:tzOffset()}` EL function in order to take the data for the previous EST5EDT day.
-IMPORTANT: The =${coord:tzOffset()}= function returns the offset in minutes, and the datasets in the example are hourly datasets. Because of this, the offset must be divided by =60= to compute the instance offset.
+IMPORTANT: The `${coord:tzOffset()}` function returns the offset in minutes, and the datasets in the example are hourly datasets. Because of this, the offset must be divided by `60` to compute the instance offset.
----+++ 7.3. Timezone and Daylight Saving Tools
+### 7.3. Timezone and Daylight Saving Tools
The Coordinator engine should provide tools to help developers convert and compute UTC datetimes to timezone datetimes and to daylight saving aware timezones.
----++ 8. Operational Considerations
+## 8. Operational Considerations
----+++ 8.1. Reprocessing
+### 8.1. Reprocessing
* TBD
----++ 9. User Propagation
+## 9. User Propagation
-When submitting a coordinator job, the configuration must contain a =user.name= property. If security is enabled, Oozie must ensure that the value of the =user.name= property in the configuration match the user credentials present in the protocol (web services) request.
+When submitting a coordinator job, the configuration must contain a `user.name` property. If security is enabled, Oozie must ensure that the value of the `user.name` property in the configuration match the user credentials present in the protocol (web services) request.
-When submitting a coordinator job, the configuration may contain the =oozie.job.acl= property (the =group.name= property
+When submitting a coordinator job, the configuration may contain the `oozie.job.acl` property (the `group.name` property
has been deprecated). If authorization is enabled, this property is treated as as the ACL for the job, it can contain
user and group IDs separated by commas.
@@ -3892,11 +4022,11 @@
Oozie must propagate the specified user and ACL to the system executing the actions (workflow jobs).
----++ 10. Coordinator Application Deployment
+## 10. Coordinator Application Deployment
Coordinator applications consist exclusively of dataset definitions and coordinator application definitions. They must be installed in an HDFS directory. To submit a job for a coordinator application, the full HDFS path to coordinator application definition must be specified.
----+++ 10.1. Organizing Coordinator Applications
+### 10.1. Organizing Coordinator Applications
The usage of Oozie Coordinator can be categorized in 3 different segments:
@@ -3904,21 +4034,21 @@
* *Medium:* consisting of a single shared dataset definitions and a few coordinator applications
* *Large:* consisting of a single or multiple shared dataset definitions and several coordinator applications
-Systems that fall in the *medium* and (specially) in the *large* categories are usually referred as data pipeline systems.
+Systems that fall in the **medium** and (specially) in the **large** categories are usually referred as data pipeline systems.
Oozie Coordinator definition XML schemas provide a convenient and flexible mechanism for all 3 systems categorization define above.
-For *small* systems: All dataset definitions and the coordinator application definition can be defined in a single XML file. The XML definition file is commonly in its own HDFS directory.
+For **small** systems: All dataset definitions and the coordinator application definition can be defined in a single XML file. The XML definition file is commonly in its own HDFS directory.
-For *medium* systems: A single datasets XML file defines all shared/public datasets. Each coordinator application has its own definition file, they may have embedded/private datasets and they may refer, via inclusion, to the shared datasets XML file. All the XML definition files are grouped in a single HDFS directory.
+For **medium** systems: A single datasets XML file defines all shared/public datasets. Each coordinator application has its own definition file, they may have embedded/private datasets and they may refer, via inclusion, to the shared datasets XML file. All the XML definition files are grouped in a single HDFS directory.
-For *large* systems: Multiple datasets XML file define all shared/public datasets. Each coordinator application has its own definition file, they may have embedded/private datasets and they may refer, via inclusion, to multiple shared datasets XML files. XML definition files are logically grouped in different HDFS directories.
+For **large** systems: Multiple datasets XML file define all shared/public datasets. Each coordinator application has its own definition file, they may have embedded/private datasets and they may refer, via inclusion, to multiple shared datasets XML files. XML definition files are logically grouped in different HDFS directories.
NOTE: Oozie Coordinator does not enforce any specific organization, grouping or naming for datasets and coordinator application definition files.
The fact that each coordinator application is in a separate XML definition file simplifies coordinator job submission, monitoring and managing of jobs. Tools to support groups of jobs can be built on of the basic, per job, commands provided by the Oozie coordinator engine.
----++++ 10.1.1. Dataset Names Collision Resolution
+#### 10.1.1. Dataset Names Collision Resolution
Embedded dataset definitions within a coordinator application cannot have the same name.
@@ -3930,7 +4060,7 @@
If a coordinator application includes one or more dataset definition XML files and it has embedded dataset definitions, in case of dataset name collision between the included and the embedded definition files, the embedded dataset takes precedence over the included dataset.
----++ 11. Coordinator Job Submission
+## 11. Coordinator Job Submission
When a coordinator job is submitted to Oozie Coordinator, the submitter must specified all the required job properties plus the HDFS path to the coordinator application definition for the job.
@@ -3939,9 +4069,10 @@
All the coordinator job properties, the HDFS path for the coordinator application, the 'user.name' and 'oozie.job.acl'
must be submitted to the Oozie coordinator engine using an XML configuration file (Hadoop XML configuration file).
-*%GREEN% Example: %ENDCOLOR%*:
+**<font color="#008000"> Example: </font>**:
-<verbatim>
+
+```
<?xml version="1.0" encoding="UTF-8"?>
<configuration>
<property>
@@ -3954,16 +4085,17 @@
</property>
...
</configuration>
-</verbatim>
+```
----++ 12. SLA Handling
+## 12. SLA Handling
Oozie 2.0 is integrated with GMS (Grid Monitoring System).
-If you add *sla* tags to the Coordinator or Workflow XML files, then the SLA information will be propagated to the GMS system.
+If you add **sla** tags to the Coordinator or Workflow XML files, then the SLA information will be propagated to the GMS system.
----+++ Coordinator SLA Example
-<verbatim>
+### Coordinator SLA Example
+
+```
<coordinator-app name="hello-coord" frequency="${coord:days(1)}"
start="2009-01-02T08:01Z" end="2010-01-01T08:01Z"
timezone="America/Los_Angeles"
@@ -4010,11 +4142,12 @@
</sla:info>
</action>
</coordinator-app>
-</verbatim>
+```
----+++ Workflow SLA Example
-<verbatim>
+### Workflow SLA Example
+
+```
<workflow-app name="hello-wf"
xmlns="uri:oozie:workflow:0.2"
xmlns:sla="uri:oozie:sla:0.1">
@@ -4056,66 +4189,68 @@
<end name="end"/>
</workflow-app>
-</verbatim>
+```
* TBD
----++ 13. Web Services API
+## 13. Web Services API
`
-See the [[WebServicesAPI][Web Services API]] page.
+See the [Web Services API](WebServicesAPI.html) page.
----++ 14. Coordinator Rerun
----+++ Rerunning a Coordinator Action or Multiple Actions
+## 14. Coordinator Rerun
+### Rerunning a Coordinator Action or Multiple Actions
Example:
-<verbatim>
+
+```
$oozie job -rerun <coord_Job_id> [-nocleanup] [-refresh] [-failed]
[-config <arg> (job configuration file '.xml' or '.properties', this file can used to supply properties, which can be used for workflow)]
[-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.)
-</verbatim>
+```
-The =rerun= option reruns a terminated (=TIMEDOUT=, =SUCCEEDED=, =KILLED=, =FAILED=) coordinator action when coordinator job
-is not in =FAILED= or =KILLED= state.
+The `rerun` option reruns a terminated (`TIMEDOUT`, `SUCCEEDED`, `KILLED`, `FAILED`) coordinator action when coordinator job
+is not in `FAILED` or `KILLED` state.
-After the command is executed the rerun coordinator action will be in =WAITING= status.
+After the command is executed the rerun coordinator action will be in `WAITING` status.
-Refer to the [[DG_CoordinatorRerun][Rerunning Coordinator Actions]] for details on rerun.
+Refer to the [Rerunning Coordinator Actions](DG_CoordinatorRerun.html) for details on rerun.
-#CoordinatorNotifications
----++ 15. Coordinator Notifications
+<a name="CoordinatorNotifications"></a>
+## 15. Coordinator Notifications
Coordinator jobs can be configured to make an HTTP GET notification upon whenever a coordinator action changes its status.
Oozie will make a best effort to deliver the notifications, in case of failure it will retry the notification a
pre-configured number of times at a pre-configured interval before giving up.
-See also [[WorkflowFunctionalSpec#WorkflowNotifications][Workflow Notifications]]
+See also [Workflow Notifications](WorkflowFunctionalSpec.html#WorkflowNotifications)
----+++ 15.1 Coordinator Action Status Notification
+### 15.1 Coordinator Action Status Notification
-If the =oozie.coord.action.notification.url= property is present in the coordinator job properties when submitting the job,
+If the `oozie.coord.action.notification.url` property is present in the coordinator job properties when submitting the job,
Oozie will make a notification to the provided URL when any of the coordinator's actions changes its status.
-=oozie.coord.action.notification.proxy= property can be used to configure either a http or socks proxy.
+`oozie.coord.action.notification.proxy` property can be used to configure either a http or socks proxy.
The format is proxyHostname:port or proxyType@proxyHostname:port. If proxy type is not specified, it defaults to http.
For eg: myhttpproxyhost.mydomain.com:80 or socks@mysockshost.mydomain.com:1080.
If the URL contains any of the following tokens, they will be replaced with the actual values by Oozie before making
the notification:
- * =$actionId= : The coordinator action ID
- * =$status= : The coordinator action's current status
+ * `$actionId` : The coordinator action ID
+ * `$status` : The coordinator action's current status
----++ Appendixes
+## Appendixes
----+++ Appendix A, Oozie Coordinator XML-Schema
+### Appendix A, Oozie Coordinator XML-Schema
----++++ Oozie Coordinator Schema 0.5
+#### Oozie Coordinator Schema 0.5
-<verbatim>
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:coordinator="uri:oozie:coordinator:0.5"
elementFormDefault="qualified" targetNamespace="uri:oozie:coordinator:0.5">
@@ -4293,11 +4428,12 @@
</xs:sequence>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie Coordinator Schema 0.4
+#### Oozie Coordinator Schema 0.4
-<verbatim>
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:coordinator="uri:oozie:coordinator:0.2"
elementFormDefault="qualified" targetNamespace="uri:oozie:coordinator:0.2">
@@ -4419,11 +4555,12 @@
</xs:sequence>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie Coordinator Schema 0.2
+#### Oozie Coordinator Schema 0.2
-<verbatim>
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:coordinator="uri:oozie:coordinator:0.2"
elementFormDefault="qualified" targetNamespace="uri:oozie:coordinator:0.2">
@@ -4530,10 +4667,11 @@
</xs:sequence>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie Coordinator Schema 0.1
-<verbatim>
+#### Oozie Coordinator Schema 0.1
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:coordinator="uri:oozie:coordinator:0.1"
elementFormDefault="qualified" targetNamespace="uri:oozie:coordinator:0.1">
@@ -4639,15 +4777,16 @@
</xs:sequence>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie SLA Schemas
+#### Oozie SLA Schemas
----+++++ Oozie SLA Version 0.2
+##### Oozie SLA Version 0.2
* Supported in Oozie coordinator schema version 0.4
-<verbatim>
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:sla="uri:oozie:sla:0.2" elementFormDefault="qualified"
targetNamespace="uri:oozie:sla:0.2">
@@ -4677,11 +4816,12 @@
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----+++++ Oozie SLA Version 0.1
+##### Oozie SLA Version 0.1
-<verbatim>
+
+```
<?xml version="1.0" encoding="UTF-8"?>
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:sla="uri:oozie:sla:0.1" elementFormDefault="qualified"
@@ -4742,9 +4882,9 @@
</xs:restriction>
</xs:simpleType>
</xs:schema>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_ActionAuthentication.twiki b/docs/src/site/twiki/DG_ActionAuthentication.twiki
index bbf2d57..d280baf 100644
--- a/docs/src/site/twiki/DG_ActionAuthentication.twiki
+++ b/docs/src/site/twiki/DG_ActionAuthentication.twiki
@@ -1,12 +1,12 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Action Authentication
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Action Authentication
----++ Background
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Background
A secure cluster requires that actions have been authenticated (typically via Kerberos). However, due to the way that Oozie runs
actions, Kerberos credentials are not easily made available to actions launched by Oozie. For many action types, this is not a
@@ -20,17 +20,18 @@
behalf of the user from the service in question. The details of what this means is beyond the scope of this documentation, but
basically, Oozie needs some extra configuration in the workflow so that it can obtain this delegation token.
----++ Oozie Server Configuration
+## Oozie Server Configuration
The code to obtain delegation tokens is pluggable so that it is easy to add support for different services by simply subclassing
org.apache.oozie.action.hadoop.Credentials to retrieve a delegation token from the service and add it to the Configuration.
Out of the box, Oozie already comes with support for some credential types
-(see [[DG_ActionAuthentication#Built-in_Credentials_Implementations][Built-in Credentials Implementations]]).
+(see [Built-in Credentials Implementations](DG_ActionAuthentication.html#Built-in_Credentials_Implementations)).
The credential classes that Oozie should load are specified by the following property in oozie-site.xml. The left hand side of the
equals sign is the type for the credential type, while the right hand side is the class.
-<verbatim>
+
+```
<property>
<name>oozie.credentials.credentialclasses</name>
<value>
@@ -39,19 +40,20 @@
hive2=org.apache.oozie.action.hadoop.Hive2Credentials
</value>
</property>
-</verbatim>
+```
----++ Workflow Changes
+## Workflow Changes
-The user should add a =credentials= section to the top of their workflow that contains 1 or more =credential= sections. Each of
-these =credential= sections contains a name for the credential, the type for the credential, and any configuration properties
-needed by that type of credential for obtaining a delegation token. The =credentials= section is available in workflow schema
+The user should add a `credentials` section to the top of their workflow that contains 1 or more `credential` sections. Each of
+these `credential` sections contains a name for the credential, the type for the credential, and any configuration properties
+needed by that type of credential for obtaining a delegation token. The `credentials` section is available in workflow schema
version 0.3 and later.
For example, the following workflow is configured to obtain an HCatalog delegation token, which is given to a Pig action so that the
Pig action can talk to a secure HCatalog:
-<verbatim>
+
+```
<workflow-app xmlns='uri:oozie:workflow:0.4' name='pig-wf'>
<credentials>
<credential name='my-hcat-creds' type='hcat'>
@@ -80,41 +82,41 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-The type of the =credential= is "hcat", which is the type name we gave for the HCatCredentials class in oozie-site.xml. We gave
-the =credential= a name, "my-hcat-creds", which can be whatever you want; we then specify cred='my-hcat-creds' in the Pig action,
+The type of the `credential` is "hcat", which is the type name we gave for the HCatCredentials class in oozie-site.xml. We gave
+the `credential` a name, "my-hcat-creds", which can be whatever you want; we then specify cred='my-hcat-creds' in the Pig action,
so that Oozie will include these credentials with the action. You can include multiple credentials with an action by specifying
-a comma-separated list of =credential= names. And finally, the HCatCredentials required two properties (the metastore URI and
+a comma-separated list of `credential` names. And finally, the HCatCredentials required two properties (the metastore URI and
principal), which we also specified.
-Adding the =credentials= section to a workflow and referencing it in an action will make Oozie always try to obtain that delegation
+Adding the `credentials` section to a workflow and referencing it in an action will make Oozie always try to obtain that delegation
token. Ordinarily, this would mean that you cannot re-use this workflow in a non-secure cluster without editing it because trying
-to obtain the delegation token will likely fail. However, you can tell Oozie to ignore the =credentials= for a workflow by setting
-the job-level property =oozie.credentials.skip= to =true=; this will allow you to use the same workflow.xml in a secure and
-non-secure cluster by simply changing the job-level property at runtime. If omitted or set to =false=, Oozie will handle
-the =credentials= section normally. In addition, you can also set this property at the action-level or server-level to skip getting
+to obtain the delegation token will likely fail. However, you can tell Oozie to ignore the `credentials` for a workflow by setting
+the job-level property `oozie.credentials.skip` to `true`; this will allow you to use the same workflow.xml in a secure and
+non-secure cluster by simply changing the job-level property at runtime. If omitted or set to `false`, Oozie will handle
+the `credentials` section normally. In addition, you can also set this property at the action-level or server-level to skip getting
credentials for just that action or for all workflows, respectively. The order of priority is this:
- 1. =oozie.credentials.skip= in the =configuration= section of an action, if set
- 1. =oozie.credentials.skip= in the job.properties for a workflow, if set
- 1. =oozie.credentials.skip= in oozie-site.xml for all workflows, if set
+ 1. `oozie.credentials.skip` in the `configuration` section of an action, if set
+ 1. `oozie.credentials.skip` in the job.properties for a workflow, if set
+ 1. `oozie.credentials.skip` in oozie-site.xml for all workflows, if set
1. (don't skip)
----++ Built-in Credentials Implementations
+## Built-in Credentials Implementations
Oozie currently comes with the following Credentials implementations:
- 1. HCatalog and Hive Metastore: =org.apache.oozie.action.hadoop.HCatCredentials=
- 1. HBase: =org.apache.oozie.action.hadoop.HBaseCredentials=
- 1. Hive Server 2: =org.apache.oozie.action.hadoop.Hive2Credentials=
+ 1. HCatalog and Hive Metastore: `org.apache.oozie.action.hadoop.HCatCredentials`
+ 1. HBase: `org.apache.oozie.action.hadoop.HBaseCredentials`
+ 1. Hive Server 2: `org.apache.oozie.action.hadoop.Hive2Credentials`
HCatCredentials requires these two properties:
- 1. =hcat.metastore.principal= or hive.metastore.kerberos.principal
- 1. =hcat.metastore.uri= or hive.metastore.uris
+ 1. `hcat.metastore.principal` or hive.metastore.kerberos.principal
+ 1. `hcat.metastore.uri` or hive.metastore.uris
-*Note:* The HCatalog Metastore and Hive Metastore are one and the same and so the "hcat" type credential can also be used to talk
+**Note:** The HCatalog Metastore and Hive Metastore are one and the same and so the "hcat" type credential can also be used to talk
to a secure Hive Metastore, though the property names would still start with "hcat.".
HBase does not require any additional properties since the hbase-site.xml on the Oozie server provides necessary information
@@ -122,9 +124,9 @@
Hive2Credentials requires these two properties:
- 1. =hive2.server.principal=
- 1. =hive2.jdbc.url=
+ 1. `hive2.server.principal`
+ 1. `hive2.jdbc.url`
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_CommandLineTool.twiki b/docs/src/site/twiki/DG_CommandLineTool.twiki
index 8d33c50..51382c7 100644
--- a/docs/src/site/twiki/DG_CommandLineTool.twiki
+++ b/docs/src/site/twiki/DG_CommandLineTool.twiki
@@ -1,39 +1,42 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Command Line Interface Utilities
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Command Line Interface Utilities
----++ Introduction
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
-Oozie provides a command line utility, =oozie=, to perform job and admin tasks. All operations are done via
-sub-commands of the =oozie= CLI.
+## Introduction
-The =oozie= CLI interacts with Oozie via its WS API.
+Oozie provides a command line utility, `oozie`, to perform job and admin tasks. All operations are done via
+sub-commands of the `oozie` CLI.
----++ Oozie Command Line Usage
+The `oozie` CLI interacts with Oozie via its WS API.
-<verbatim>
+## Oozie Command Line Usage
+
+
+```
usage:
the env variable 'OOZIE_URL' is used as default value for the '-oozie' option
the env variable 'OOZIE_TIMEZONE' is used as default value for the '-timezone' option
the env variable 'OOZIE_AUTH' is used as default value for the '-auth' option
custom headers for Oozie web services can be specified using '-Dheader:NAME=VALUE'
-</verbatim>
+```
----+++ Oozie basic commands
-<verbatim>
+### Oozie basic commands
+
+```
oozie help : display usage
oozie version : show client version
-</verbatim>
+```
----+++ Oozie job operation commands
-<verbatim>
+### Oozie job operation commands
+
+```
oozie job <OPTIONS> : job operations
-action <arg> coordinator rerun/kill on action ids (requires -rerun/-kill);
@@ -117,10 +120,11 @@
prefix defined in
=oozie-site.xml#oozie.client.jobs.application.generated.path= is used. Output is the workflow ID
-</verbatim>
+```
----+++ Oozie jobs operation commands
-<verbatim>
+### Oozie jobs operation commands
+
+```
oozie jobs <OPTIONS> : jobs status
-auth <arg> select authentication type [SIMPLE|KERBEROS]
-doas <arg> doAs user, impersonates as the specified user.
@@ -142,10 +146,11 @@
other options, it will resume all the first 50 workflow jobs. Command will fail if one or more
of the jobs is in wrong state.
-verbose verbose mode
-</verbatim>
+```
----+++ Oozie admin operation commands
-<verbatim>
+### Oozie admin operation commands
+
+```
oozie admin <OPTIONS> : admin operations
-auth <arg> select authentication type [SIMPLE|KERBEROS]
-configuration show Oozie system configuration
@@ -165,27 +170,30 @@
-version show Oozie server build version
-purge <arg> purge old oozie workflow, coordinator and bundle records from DB (parameter unit: day)
wf=<N>\;coord=<N>\;bundle=<N>\;limit=<N>\;oldcoordaction=<true/false>
-</verbatim>
+```
----+++ Oozie validate command
-<verbatim>
+### Oozie validate command
+
+```
oozie validate <OPTIONS> <ARGS> : validate a workflow, coordinator, bundle XML file
-auth <arg> select authentication type [SIMPLE|KERBEROS]
-oozie <arg> Oozie URL
-</verbatim>
+```
----+++ Oozie SLA operation commands
-<verbatim>
+### Oozie SLA operation commands
+
+```
oozie sla <OPTIONS> : sla operations (Deprecated as of Oozie 4.0)
-auth <arg> select authentication type [SIMPLE|KERBEROS]
-len <arg> number of results (default '100', max limited by oozie server setting which defaults to '1000')
-offset <arg> start offset (default '0')
-oozie <arg> Oozie URL
-filter <arg> jobid=<JobID/ActionID>\;appname=<Application Name>
-</verbatim>
+```
----+++ Oozie Pig submit command
-<verbatim>
+### Oozie Pig submit command
+
+```
oozie pig <OPTIONS> -X <ARGS> : submit a pig job, everything after '-X' are pass-through parameters to pig, any '-D' arguments
after '-X' are put in <configuration>
-auth <arg> select authentication type [SIMPLE|KERBEROS]
@@ -195,10 +203,11 @@
-file <arg> Pig script
-oozie <arg> Oozie URL
-P <property=value> set parameters for script
-</verbatim>
+```
----+++ Oozie Hive submit command
-<verbatim>
+### Oozie Hive submit command
+
+```
oozie hive <OPTIONS> -X<ARGS> : submit a hive job, everything after '-X' are pass-through parameters to hive, any '-D' arguments
after '-X' are put in <configuration>
-auth <arg> select authentication type [SIMPLE|KERBEROS]
@@ -208,10 +217,11 @@
-file <arg> hive script
-oozie <arg> Oozie URL
-P <property=value> set parameters for script
-</verbatim>
+```
----+++ Oozie Sqoop submit command
-<verbatim>
+### Oozie Sqoop submit command
+
+```
oozie sqoop <OPTIONS> -X<ARGS> : submit a sqoop job, any '-D' arguments after '-X' are put in <configuration>
-auth <arg> select authentication type [SIMPLE|KERBEROS]
-config <arg> job configuration file '.properties'
@@ -219,246 +229,259 @@
-doas <arg> doAs user, impersonates as the specified user
-command <arg> sqoop command
-oozie <arg> Oozie URL
-</verbatim>
+```
----+++ Oozie info command
-<verbatim>
+### Oozie info command
+
+```
oozie info <OPTIONS> : get more detailed info about specific topics
-timezones display a list of available time zones
-</verbatim>
+```
----+++ Oozie MapReduce job command
-<verbatim>
+### Oozie MapReduce job command
+
+```
oozie mapreduce <OPTIONS> : submit a mapreduce job
-auth <arg> select authentication type [SIMPLE|KERBEROS]
-config <arg> job configuration file '.properties'
-D <property=value> set/override value for given property
-doas <arg> doAs user, impersonates as the specified user
-oozie <arg> Oozie URL
-</verbatim>
+```
----++ Common CLI Options
+## Common CLI Options
----+++ Authentication
+### Authentication
-The =oozie= CLI automatically perform authentication if the Oozie server requests it. By default it supports both
+The `oozie` CLI automatically perform authentication if the Oozie server requests it. By default it supports both
pseudo/simple authentication and Kerberos HTTP SPNEGO authentication.
-To perform a specific authentication, the =auth= option with authentication type requests Oozie client to run the
-specified authentication mechanism only. Oozie client provides two types =simple= and =kerberos= which supports =pseudo/simple= and =Kerberos=.
+To perform a specific authentication, the `auth` option with authentication type requests Oozie client to run the
+specified authentication mechanism only. Oozie client provides two types `simple` and `kerberos` which supports `pseudo/simple` and `Kerberos`.
-For pseudo/simple authentication the =oozie= CLI uses the user name of the current OS user.
+For pseudo/simple authentication the `oozie` CLI uses the user name of the current OS user.
-For Kerberos HTTP SPNEGO authentication the =oozie= CLI uses the default principal for the OS Kerberos cache
-(normally the principal that did =kinit=).
+For Kerberos HTTP SPNEGO authentication the `oozie` CLI uses the default principal for the OS Kerberos cache
+(normally the principal that did `kinit`).
Oozie uses Apache Hadoop-Auth (Java HTTP SPNEGO) library for authentication.
This library can be extended to support other authentication mechanisms.
Once authentication is performed successfully the received authentication token is cached in the user home directory
-in the =.oozie-auth-token= file with owner-only permissions. Subsequent requests reuse the cached token while valid.
+in the `.oozie-auth-token` file with owner-only permissions. Subsequent requests reuse the cached token while valid.
-The use of the cache file can be disabled by invoking the =oozie= CLI with the =-Doozie.auth.token.cache=false=
+The use of the cache file can be disabled by invoking the `oozie` CLI with the `-Doozie.auth.token.cache`false=
option.
-To use an custom authentication mechanism, a Hadoop-Auth =Authenticator= implementation must be specified with the
- =-Dauthenticator.class= = =CLASS= option.
+To use an custom authentication mechanism, a Hadoop-Auth `Authenticator` implementation must be specified with the
+ `-Dauthenticator.class=CLASS` option.
----+++ Impersonation, doAs
+### Impersonation, doAs
-The <code>-doas</code> option allows the current user to impersonate other users when interacting with the Oozie
+The `-doas` option allows the current user to impersonate other users when interacting with the Oozie
system. The current user must be configured as a proxyuser in the Oozie system. The proxyuser configuration may
restrict from which hosts a user may impersonate users, as well as users of which groups can be impersonated.
----+++ Oozie URL
+### Oozie URL
-All =oozie= CLI sub-commands expect the <code>-oozie OOZIE_URL</code> option indicating the URL of the Oozie system
-to run the command against. If the OOZIE_URL environment variable has not been set, =oozie= will use the default
-URL specified in oozie-client-env.sh (equivalent to =!http://$(hostname -f):11000/oozie=).
+All `oozie` CLI sub-commands expect the `-oozie OOZIE_URL` option indicating the URL of the Oozie system
+to run the command against. If the OOZIE_URL environment variable has not been set, `oozie` will use the default
+URL specified in oozie-client-env.sh (equivalent to `!http://$(hostname -f):11000/oozie`).
-If the <code>-oozie</code> option is not specified, the =oozie= CLI will look for the =OOZIE_URL= environment variable
+If the `-oozie` option is not specified, the `oozie` CLI will look for the `OOZIE_URL` environment variable
and uses it if set.
-If the option is not provided and the environment variable is not set, the =oozie= CLI will fail.
+If the option is not provided and the environment variable is not set, the `oozie` CLI will fail.
----+++ Time zone
+### Time zone
-The <code>-timezone TIME_ZONE_ID</code> option in the =job= and =jobs= sub-commands allows you to specify the time zone to use in
-the output of those sub-commands. The <code>TIME_ZONE_ID</code> should be one of the standard Java Time Zone IDs. You can get a
-list of the available time zones with the command =oozie info -timezones=.
+The `-timezone TIME_ZONE_ID` option in the `job` and `jobs` sub-commands allows you to specify the time zone to use in
+the output of those sub-commands. The `TIME_ZONE_ID` should be one of the standard Java Time Zone IDs. You can get a
+list of the available time zones with the command `oozie info -timezones`.
-If the <code>-localtime</code> option is used, it will cause Oozie to use whatever the time zone is of the machine. If
-both <code>-localtime</code> and <code>-timezone TIME_ZONE_ID</code> are used, the <code>-localtime</code> option will override
-the <code>-timezone TIME_ZONE_ID</code> option. If neither option is given, Oozie will look for the =OOZIE_TIMEZONE= environment
+If the `-localtime` option is used, it will cause Oozie to use whatever the time zone is of the machine. If
+both `-localtime` and `-timezone TIME_ZONE_ID` are used, the `-localtime` option will override
+the `-timezone TIME_ZONE_ID` option. If neither option is given, Oozie will look for the `OOZIE_TIMEZONE` environment
variable and uses it if set. If neither option is given and the environment variable is not set, or if Oozie is given an invalid
time zone, it will use GMT.
----+++ Debug Mode
+### Debug Mode
-If you export <code>OOZIE_DEBUG=1</code> then the Oozie CLI will output the Web Services API details used by any commands you
+If you export `OOZIE_DEBUG=1` then the Oozie CLI will output the Web Services API details used by any commands you
execute. This is useful for debugging purposes to or see how the Oozie CLI works with the WS API.
----+++ CLI retry
+### CLI retry
Oozie CLI retries connection to Oozie servers for transparent high availability failover when one of the Oozie servers go down.
-=Oozie= CLI command will retry for all commands in case of ConnectException.
-In case of SocketException, all commands except =PUT= and =POST= will have retry logic.
-All job submit are POST call, examples of PUT and POST commands can be find out from [[WebServicesAPI][WebServicesAPI]].
-Retry count can be configured with system property =oozie.connection.retry.count=. Default count is 4.
+`Oozie` CLI command will retry for all commands in case of ConnectException.
+In case of SocketException, all commands except `PUT` and `POST` will have retry logic.
+All job submit are POST call, examples of PUT and POST commands can be find out from [WebServicesAPI](WebServicesAPI.html).
+Retry count can be configured with system property `oozie.connection.retry.count`. Default count is 4.
----++ Job Operations
+## Job Operations
----+++ Submitting a Workflow, Coordinator or Bundle Job
+### Submitting a Workflow, Coordinator or Bundle Job
* Submitting bundle feature is only supported in Oozie 3.0 or later. Similarly, all bundle operation features below are only
supported in Oozie 3.0 or later.
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -config job.properties -submit
.
job: 14-20090525161321-oozie-joe
-</verbatim>
+```
The parameters for the job must be provided in a file, either a Java Properties file (.properties) or a Hadoop XML
-Configuration file (.xml). This file must be specified with the <code>-config</code> option.
+Configuration file (.xml). This file must be specified with the `-config` option.
-The workflow application path must be specified in the file with the =oozie.wf.application.path= property. The
-coordinator application path must be specified in the file with the =oozie.coord.application.path= property.The
-bundle application path must be specified in the file with the =oozie.bundle.application.path= property.
+The workflow application path must be specified in the file with the `oozie.wf.application.path` property. The
+coordinator application path must be specified in the file with the `oozie.coord.application.path` property.The
+bundle application path must be specified in the file with the `oozie.bundle.application.path` property.
Specified path must be an HDFS path.
-The job will be created, but it will not be started, it will be in =PREP= status.
+The job will be created, but it will not be started, it will be in `PREP` status.
----+++ Starting a Workflow, Coordinator or Bundle Job
+### Starting a Workflow, Coordinator or Bundle Job
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -start 14-20090525161321-oozie-joe
-</verbatim>
+```
-The =start= option starts a previously submitted workflow job or bundle job that is in =PREP= status.
+The `start` option starts a previously submitted workflow job or bundle job that is in `PREP` status.
-After the command is executed the workflow job will be in =RUNNING= status and bundle job will be in =RUNNING=status.
+After the command is executed the workflow job will be in `RUNNING` status and bundle job will be in `RUNNING`status.
-A coordinator job does not support the =start= action. It will show the following error message when trying to start it
+A coordinator job does not support the `start` action. It will show the following error message when trying to start it
via the CLI:
-<verbatim>
-Error: E0303 : E0303: Invalid parameter value, [action] = [start]
-</verbatim>
----+++ Running a Workflow, Coordinator or Bundle Job
+```
+Error: E0303 : E0303: Invalid parameter value, [action] = [start]
+```
+
+### Running a Workflow, Coordinator or Bundle Job
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -config job.properties -run
.
job: 15-20090525161321-oozie-joe
-</verbatim>
+```
-The =run= option creates and starts a workflow job, coordinator job or bundle job.
+The `run` option creates and starts a workflow job, coordinator job or bundle job.
The parameters for the job must be provided in a file, either a Java Properties file (.properties) or a Hadoop XML
-Configuration file (.xml). This file must be specified with the <code>-config</code> option.
+Configuration file (.xml). This file must be specified with the `-config` option.
-The workflow application path must be specified in the file with the =oozie.wf.application.path= property. The
-coordinator application path must be specified in the file with the =oozie.coord.application.path= property. The
-bundle application path must be specified in the file with the =oozie.bundle.application.path= property.The
+The workflow application path must be specified in the file with the `oozie.wf.application.path` property. The
+coordinator application path must be specified in the file with the `oozie.coord.application.path` property. The
+bundle application path must be specified in the file with the `oozie.bundle.application.path` property.The
specified path must be an HDFS path.
-The job will be created and it will started, the job will be in =RUNNING= status.
+The job will be created and it will started, the job will be in `RUNNING` status.
----+++ Suspending a Workflow, Coordinator or Bundle Job
+### Suspending a Workflow, Coordinator or Bundle Job
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -suspend 14-20090525161321-oozie-joe
-</verbatim>
+```
-The =suspend= option suspends a workflow job in =RUNNING= status.
-After the command is executed the workflow job will be in =SUSPENDED= status.
+The `suspend` option suspends a workflow job in `RUNNING` status.
+After the command is executed the workflow job will be in `SUSPENDED` status.
-The =suspend= option suspends a coordinator/bundle job in =RUNNING=, =RUNNINGWITHERROR= or =PREP= status.
-When the coordinator job is suspended, running coordinator actions will stay in running and the workflows will be suspended. If the coordinator job is in =RUNNING=status, it will transit to =SUSPENDED=status; if it is in =RUNNINGWITHERROR=status, it will transit to =SUSPENDEDWITHERROR=; if it is in =PREP=status, it will transit to =PREPSUSPENDED=status.
+The `suspend` option suspends a coordinator/bundle job in `RUNNING`, `RUNNINGWITHERROR` or `PREP` status.
+When the coordinator job is suspended, running coordinator actions will stay in running and the workflows will be suspended. If the coordinator job is in `RUNNING`status, it will transit to `SUSPENDED`status; if it is in `RUNNINGWITHERROR`status, it will transit to `SUSPENDEDWITHERROR`; if it is in `PREP`status, it will transit to `PREPSUSPENDED`status.
-When the bundle job is suspended, running coordinators will be suspended. If the bundle job is in =RUNNING=status, it will transit to =SUSPENDED=status; if it is in =RUNNINGWITHERROR=status, it will transit to =SUSPENDEDWITHERROR=; if it is in =PREP=status, it will transit to =PREPSUSPENDED=status.
+When the bundle job is suspended, running coordinators will be suspended. If the bundle job is in `RUNNING`status, it will transit to `SUSPENDED`status; if it is in `RUNNINGWITHERROR`status, it will transit to `SUSPENDEDWITHERROR`; if it is in `PREP`status, it will transit to `PREPSUSPENDED`status.
----+++ Resuming a Workflow, Coordinator or Bundle Job
+### Resuming a Workflow, Coordinator or Bundle Job
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -resume 14-20090525161321-oozie-joe
-</verbatim>
+```
-The =resume= option resumes a workflow job in =SUSPENDED= status.
+The `resume` option resumes a workflow job in `SUSPENDED` status.
-After the command is executed the workflow job will be in =RUNNING= status.
+After the command is executed the workflow job will be in `RUNNING` status.
-The =suspend= option suspends a coordinator/bundle job in =SUSPENDED=, =SUSPENDEDWITHERROR= or =PREPSUSPENDED= status.
-If the coordinator job is in =SUSPENDED=status, it will transit to =RUNNING=status; if it is in =SUSPENDEDWITHERROR=status, it will transit to =RUNNINGWITHERROR=; if it is in =PREPSUSPENDED=status, it will transit to =PREP=status.
+The `suspend` option suspends a coordinator/bundle job in `SUSPENDED`, `SUSPENDEDWITHERROR` or `PREPSUSPENDED` status.
+If the coordinator job is in `SUSPENDED`status, it will transit to `RUNNING`status; if it is in `SUSPENDEDWITHERROR`status, it will transit to `RUNNINGWITHERROR`; if it is in `PREPSUSPENDED`status, it will transit to `PREP`status.
When the coordinator job is resumed it will create all the coordinator actions that should have been created during the time
it was suspended, actions will not be lost, they will delayed.
-When the bundle job is resumed, suspended coordinators will resume running. If the bundle job is in =SUSPENDED=status, it will transit to =RUNNING=status; if it is in =SUSPENDEDWITHERROR=status, it will transit to =RUNNINGWITHERROR=; if it is in =PREPSUSPENDED=status, it will transit to =PREP=status.
+When the bundle job is resumed, suspended coordinators will resume running. If the bundle job is in `SUSPENDED`status, it will transit to `RUNNING`status; if it is in `SUSPENDEDWITHERROR`status, it will transit to `RUNNINGWITHERROR`; if it is in `PREPSUSPENDED`status, it will transit to `PREP`status.
----+++ Killing a Workflow, Coordinator or Bundle Job
+### Killing a Workflow, Coordinator or Bundle Job
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -kill 14-20090525161321-oozie-joe
-</verbatim>
+```
-The =kill= option kills a workflow job in =PREP=, =SUSPENDED= or =RUNNING= status and a coordinator/bundle job in
-=PREP=, =RUNNING=, =PREPSUSPENDED=, =SUSPENDED=, =PREPPAUSED=, or =PAUSED= status.
+The `kill` option kills a workflow job in `PREP`, `SUSPENDED` or `RUNNING` status and a coordinator/bundle job in
+`PREP`, `RUNNING`, `PREPSUSPENDED`, `SUSPENDED`, `PREPPAUSED`, or `PAUSED` status.
-After the command is executed the job will be in =KILLED= status.
+After the command is executed the job will be in `KILLED` status.
----+++ Killing a Coordinator Action or Multiple Actions
+### Killing a Coordinator Action or Multiple Actions
Example:
-<verbatim>
-$oozie job -kill <coord_Job_id> [-action 1, 3-4, 7-40] [-date 2009-01-01T01:00Z::2009-05-31T23:59Z, 2009-11-10T01:00Z, 2009-12-31T22:00Z]
-</verbatim>
- * The =kill= option here for a range of coordinator actions kills a non-terminal (=RUNNING=, =WAITING=, =READY=, =SUSPENDED=) coordinator action when coordinator job is not in =FAILED= or =KILLED= state.
+```
+$oozie job -kill <coord_Job_id> [-action 1, 3-4, 7-40] [-date 2009-01-01T01:00Z::2009-05-31T23:59Z, 2009-11-10T01:00Z, 2009-12-31T22:00Z]
+```
+
+ * The `kill` option here for a range of coordinator actions kills a non-terminal (`RUNNING`, `WAITING`, `READY`, `SUSPENDED`) coordinator action when coordinator job is not in `FAILED` or `KILLED` state.
* Either -action or -date should be given.
* If neither -action nor -date is given, the exception will be thrown. Also if BOTH -action and -date are given, an error will be thrown.
* Multiple ranges can be used in -action or -date. See the above example.
* If one of the actions in the given list of -action is already in terminal state, the output of this command will only include the other actions.
* 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.
- * After the command is executed the killed coordinator action will have =KILLED= status.
+ * After the command is executed the killed coordinator action will have `KILLED` status.
----+++ Changing endtime/concurrency/pausetime/status of a Coordinator Job
+### Changing endtime/concurrency/pausetime/status of a Coordinator Job
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -change 14-20090525161321-oozie-joe -value endtime=2011-12-01T05:00Z\;concurrency=100\;2011-10-01T05:00Z
$ oozie job -oozie http://localhost:11000/oozie -change 0000001-140321155112907-oozie-puru-C -value status=RUNNING
-</verbatim>
+```
-The =endtime/concurrency/pausetime= option changes a coordinator job that is not in =KILLED= status.
+The `endtime/concurrency/pausetime` option changes a coordinator job that is not in `KILLED` status.
Valid value names are:
+
* endtime: the end time of the coordinator job.
* concurrency: the concurrency of the coordinator job.
* pausetime: the pause time of the coordinator job.
* status: new status for coordinator job.
Conditions and usage:
+
* Repeated value names are not allowed.
* New end time should not be before job's start time and last action time.
* If end time is before job start time and if the job has not materialized any actions, then job status is changed to SUCCEEDED.
@@ -470,85 +493,91 @@
After the command is executed the job's end time, concurrency or pause time should be changed. If an already-succeeded job changes its end time, its status will become running.
----+++ Changing endtime/pausetime of a Bundle Job
+### Changing endtime/pausetime of a Bundle Job
Example:
-<verbatim>
-$ oozie job -oozie http://localhost:11000/oozie -change 14-20090525161321-oozie-joe -value pausetime=2011-12-01T05:00Z
-</verbatim>
-The =change= option changes a bundle job that is not in =KILLED= status.
+```
+$ oozie job -oozie http://localhost:11000/oozie -change 14-20090525161321-oozie-joe -value pausetime=2011-12-01T05:00Z
+```
+
+The `change` option changes a bundle job that is not in `KILLED` status.
Valid value names are:
+
* pausetime: the pause time of the bundle job.
* endtime: the end time of the bundle job.
Repeated value names are not allowed. An empty string "" can be used to reset pause time to none. New end time should not be before job's kickoff time.
-Bundle will execute pause/end date change command on each coordinator job. Refer conditions and usage section of coordinator change command for more details [[DG_CommandLineTool#Changing_endtimeconcurrencypausetimestatus_of_a_Coordinator_Job][Coordinator job change command]].
+Bundle will execute pause/end date change command on each coordinator job. Refer conditions and usage section of coordinator change command for more details [Coordinator job change command](DG_CommandLineTool.html#Changing_endtimeconcurrencypausetimestatus_of_a_Coordinator_Job).
----+++ Rerunning a Workflow Job
+### Rerunning a Workflow Job
Example:
-<verbatim>
-$ oozie job -oozie http://localhost:11000/oozie -config job.properties -rerun 14-20090525161321-oozie-joe
-</verbatim>
-The =rerun= option reruns a completed ( =SUCCEEDED=, =FAILED= or =KILLED= ) job skipping the specified nodes.
+```
+$ oozie job -oozie http://localhost:11000/oozie -config job.properties -rerun 14-20090525161321-oozie-joe
+```
+
+The `rerun` option reruns a completed ( `SUCCEEDED`, `FAILED` or `KILLED` ) job skipping the specified nodes.
The parameters for the job must be provided in a file, either a Java Properties file (.properties) or a Hadoop XML
-Configuration file (.xml). This file must be specified with the <code>-config</code> option.
+Configuration file (.xml). This file must be specified with the `-config` option.
-The workflow application path must be specified in the file with the =oozie.wf.application.path= property. The
+The workflow application path must be specified in the file with the `oozie.wf.application.path` property. The
specified path must be an HDFS path.
-The list of nodes to skipped must be provided in the =oozie.wf.rerun.skip.nodes= property separated by commas.
+The list of nodes to skipped must be provided in the `oozie.wf.rerun.skip.nodes` property separated by commas.
-After the command is executed the job will be in =RUNNING= status.
+After the command is executed the job will be in `RUNNING` status.
-Refer to the [[DG_WorkflowReRun][Rerunning Workflow Jobs]] for details on rerun.
+Refer to the [Rerunning Workflow Jobs](DG_WorkflowReRun.html) for details on rerun.
----+++ Rerunning a Coordinator Action or Multiple Actions
+### Rerunning a Coordinator Action or Multiple Actions
Example:
-<verbatim>
+
+```
$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.)
-</verbatim>
+```
-The =rerun= option reruns a terminated (=TIMEDOUT=, =SUCCEEDED=, =KILLED=, =FAILED=, =IGNORED=) coordinator action when coordinator job
-is not in =FAILED= or =KILLED= state.
+The `rerun` option reruns a terminated (`TIMEDOUT`, `SUCCEEDED`, `KILLED`, `FAILED`, `IGNORED`) coordinator action when coordinator job
+is not in `FAILED` or `KILLED` state.
-After the command is executed the rerun coordinator action will be in =WAITING= status.
+After the command is executed the rerun coordinator action will be in `WAITING` status.
-Refer to the [[DG_CoordinatorRerun][Rerunning Coordinator Actions]] for details on rerun.
+Refer to the [Rerunning Coordinator Actions](DG_CoordinatorRerun.html) for details on rerun.
----+++ Rerunning a Bundle Job
+### Rerunning a Bundle Job
Example:
-<verbatim>
+
+```
$oozie job -rerun <bundle_Job_id> [-nocleanup] [-refresh]
[-coordinator c1, c3, c4] (-coordinator 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 -coordinator nor -date is given, the exception will be thrown.)
-</verbatim>
+```
-The =rerun= option reruns coordinator actions belonging to specified coordinators within the specified date range.
+The `rerun` option reruns coordinator actions belonging to specified coordinators within the specified date range.
-After the command is executed the rerun coordinator action will be in =WAITING= status.
+After the command is executed the rerun coordinator action will be in `WAITING` status.
----+++ Checking the Information and Status of a Workflow, Coordinator or Bundle Job or a Coordinator Action
+### Checking the Information and Status of a Workflow, Coordinator or Bundle Job or a Coordinator Action
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -info 14-20090525161321-oozie-joe
.
.----------------------------------------------------------------------------------------------------------------------------------------------------------------
@@ -567,36 +596,37 @@
.----------------------------------------------------------------------------------------------------------------------------------------------------------------
hadoop1 map-reduce OK end job_200904281535_0254 SUCCEEDED - 2009-05-26 05:01 +0000 2009-05-26 05:01 +0000
.----------------------------------------------------------------------------------------------------------------------------------------------------------------
-</verbatim>
+```
-The =info= option can display information about a workflow job or coordinator job or coordinator action.
-The =info= option for a Coordinator job will retrieve the Coordinator actions ordered by nominal time. However, the =info= command may timeout if the number of Coordinator actions are very high. In that case, =info= should be used with =offset= and =len= option.
+The `info` option can display information about a workflow job or coordinator job or coordinator action.
+The `info` option for a Coordinator job will retrieve the Coordinator actions ordered by nominal time. However, the `info` command may timeout if the number of Coordinator actions are very high. In that case, `info` should be used with `offset` and `len` option.
-The =offset= and =len= option should be used for pagination. offset determines the start offset of the action
+The `offset` and `len` option should be used for pagination. offset determines the start offset of the action
returned among all the actions that matched the filter criteria. len determines number of actions to be returned.
-The =localtime= option displays times in local time, if not specified times are displayed in GMT.
+The `localtime` option displays times in local time, if not specified times are displayed in GMT.
-The =filter= option can be used to filter coordinator actions based on some criteria.
-The filter option syntax is: <code><key><comparator><value>[;<key><comparator><value>]*</code>.
-(Note escape <code>\</code> needed before semicolon to specify multiple names for filter in shell)
+The `filter` option can be used to filter coordinator actions based on some criteria.
+The filter option syntax is: `<key><comparator><value>[;<key><comparator><value>]*`.
+(Note escape `\` needed before semicolon to specify multiple names for filter in shell)
key: status or nominalTime
-comparator: =, !=, <, <=, >, >=
-value: valid status like SUCCEEDED, KILLED, RUNNING etc. Only = and != apply for status
+comparator: `, !`, <, <`, >, >`
+value: valid status like SUCCEEDED, KILLED, RUNNING etc. Only ` and !` apply for status
value for nominalTime is valid date of the format yyyy-MM-dd'T'HH:mm'Z' (like 2014-06-01T00:00Z)
Multiple values must be specified as different name value pairs. The query is formed by doing AND of all conditions,
with the exception of = which uses OR if there are multiple values for the same key. For example,
-filter 'status=RUNNING;status=WAITING;nominalTime>=2014-06-01T00:00Z' maps to query (status = RUNNING OR status =
-WAITING) AND nominalTime >= 2014-06-01T00:00Z which returns all waiting or running actions with nominalTime >=
+filter 'status`RUNNING;status`WAITING;nominalTime>`2014-06-01T00:00Z' maps to query (status ` RUNNING OR status =
+WAITING) AND nominalTime >` 2014-06-01T00:00Z which returns all waiting or running actions with nominalTime >`
2014-06-01T00:00Z.
-Currently, the filter option can be used only with an =info= option on Coordinator job.
+Currently, the filter option can be used only with an `info` option on Coordinator job.
-The =verbose= option gives more detailed information for all the actions, if checking for workflow job or coordinator job.
-An example below shows how the =verbose= option can be used to gather action statistics information for a job:
+The `verbose` option gives more detailed information for all the actions, if checking for workflow job or coordinator job.
+An example below shows how the `verbose` option can be used to gather action statistics information for a job:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -info 0000001-111219170928042-oozie-para-W@mr-node -verbose
ID : 0000001-111219170928042-oozie-para-W@mr-node
------------------------------------------------------------------------------------------------------------------------------------
@@ -615,23 +645,24 @@
External Stats : {"org.apache.hadoop.mapred.JobInProgress$Counter":{"TOTAL_LAUNCHED_REDUCES":1,"TOTAL_LAUNCHED_MAPS":1,"DATA_LOCAL_MAPS":1},"ACTION_TYPE":"MAP_REDUCE","FileSystemCounters":{"FILE_BYTES_READ":1746,"HDFS_BYTES_READ":1409,"FILE_BYTES_WRITTEN":3524,"HDFS_BYTES_WRITTEN":1547},"org.apache.hadoop.mapred.Task$Counter":{"REDUCE_INPUT_GROUPS":33,"COMBINE_OUTPUT_RECORDS":0,"MAP_INPUT_RECORDS":33,"REDUCE_SHUFFLE_BYTES":0,"REDUCE_OUTPUT_RECORDS":33,"SPILLED_RECORDS":66,"MAP_OUTPUT_BYTES":1674,"MAP_INPUT_BYTES":1409,"MAP_OUTPUT_RECORDS":33,"COMBINE_INPUT_RECORDS":0,"REDUCE_INPUT_RECORDS":33}}
External ChildIDs : null
------------------------------------------------------------------------------------------------------------------------------------
-</verbatim>
+```
The two fields External Stats and External ChildIDs display the action statistics information (that includes counter information in case of MR action and PigStats information in case of a pig action) and child ids of the given job.
Note that the user can turn on/off External Stats by specifying the property _oozie.action.external.stats.write_ as _true_ or _false_ in workflow.xml. By default, it is set to false (not to collect External Stats). External ChildIDs will always be stored.
----+++ Listing all the Workflows for a Coordinator Action
+### Listing all the Workflows for a Coordinator Action
A coordinator action kicks off different workflows for its original run and all subsequent reruns.
Getting a list of those workflow ids is a useful tool to keep track of your actions' runs and
to go debug the workflow job logs if required. Along with ids, it also lists their statuses,
and start and end times for quick reference.
-This is achieved by using the Coordinator Action info command and specifying a flag *=allruns=*
-along with the =info= command.
+This is achieved by using the Coordinator Action info command and specifying a flag **`allruns`**
+along with the `info` command.
-<verbatim>
+
+```
$ oozie job -info 0000001-111219170928042-oozie-joe-C@1 -allruns -oozie http://localhost:11000/oozie
.
Job ID Status Started Ended
@@ -642,13 +673,14 @@
.----------------------------------------------------------------------------------------------------
0000001-140324164318985-oozie-joe-W SUCCEEDED 2014-03-24 23:44 GMT 2014-03-24 23:44 GMT
.----------------------------------------------------------------------------------------------------
-</verbatim>
+```
----+++ Listing all retry attempts of a workflow action
+### Listing all retry attempts of a workflow action
-When retry-max is specified for an action in the workflow definition, and there is a failure, it will be retried till it succeeds or retry-max attempts are exhausted. To get information on all the retry attempts, =-retries= command can be used.
+When retry-max is specified for an action in the workflow definition, and there is a failure, it will be retried till it succeeds or retry-max attempts are exhausted. To get information on all the retry attempts, `-retries` command can be used.
-<verbatim>
+
+```
$ oozie job -retries 0000000-161212175234862-oozie-puru-W@pig-node -oozie http://localhost:11000/oozie
ID : 0000000-161212175234862-oozie-puru-W@pig-node
@@ -669,14 +701,15 @@
Console URL : http://localhost:50030/jobdetails.jsp?jobid=job_201612051339_2650
------------------------------------------------------------------------------------------------------------------------------------
$
-</verbatim>
+```
----+++ Checking the xml definition of a Workflow, Coordinator or Bundle Job
+### Checking the xml definition of a Workflow, Coordinator or Bundle Job
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -definition 14-20090525161321-oozie-joe
.
<workflow-app xmlns="uri:oozie:workflow:0.2" name="sm3-segment-2413">
@@ -686,23 +719,25 @@
<end name="end"/>
</workflow-app>
-</verbatim>
+```
----+++ Checking the server logs of a Workflow, Coordinator or Bundle Job
+### Checking the server logs of a Workflow, Coordinator or Bundle Job
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -log 14-20090525161321-oozie-joe
-</verbatim>
+```
----+++ Checking the server error logs of a Workflow, Coordinator or Bundle Job
+### Checking the server error logs of a Workflow, Coordinator or Bundle Job
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -errorlog 0000000-150121110331712-oozie-puru-B
2015-01-21 11:33:29,090 WARN CoordSubmitXCommand:523 - SERVER[-] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0000000-150121110331712-oozie-puru-B] ACTION[] SAXException :
org.xml.sax.SAXParseException; lineNumber: 20; columnNumber: 22; cvc-complex-type.2.4.a: Invalid content was found starting with element 'concurrency'. One of '{"uri:oozie:coordinator:0.2":controls, "uri:oozie:coordinator:0.2":datasets, "uri:oozie:coordinator:0.2":input-events, "uri:oozie:coordinator:0.2":output-events, "uri:oozie:coordinator:0.2":action}' is expected.
@@ -722,39 +757,42 @@
at org.apache.xerces.parsers.XML11Configuration.parse(Unknown Source)
at org.apache.xerces.jaxp.validation.StreamValidatorHelper.validate(Unknown Source)
at org.apache.xerces.jaxp.validation.ValidatorImpl.validate(Unknown Source)
-</verbatim>
+```
----+++ Checking the audit logs of a Workflow, Coordinator or Bundle Job
+### Checking the audit logs of a Workflow, Coordinator or Bundle Job
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -auditlog 0000000-150322000230582-oozie-puru-C
2015-03-22 00:04:35,494 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [start], PARAMETER [null], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]
2015-03-22 00:05:13,823 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [suspend], PARAMETER [0000000-150322000230582-oozie-puru-C], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]
2015-03-22 00:06:59,561 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [suspend], PARAMETER [0000000-150322000230582-oozie-puru-C], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]
2015-03-22 23:22:20,012 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [suspend], PARAMETER [0000000-150322000230582-oozie-puru-C], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]
2015-03-22 23:28:48,218 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [resume], PARAMETER [0000000-150322000230582-oozie-puru-C], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]
-</verbatim>
+```
----+++ Checking the server logs for particular actions of a Coordinator Job
+### Checking the server logs for particular actions of a Coordinator Job
Example:
-<verbatim>
+
+```
$ oozie job -log <coord_job_id> [-action 1, 3-4, 7-40] (-action is optional.)
-</verbatim>
+```
----+++ Filtering the server logs with logfilter options
+### Filtering the server logs with logfilter options
User can provide multiple option to filter logs using -logfilter opt1=val1;opt2=val1;opt3=val1. This can be used to fetch only just logs of interest faster as fetching Oozie server logs is slow due to the overhead of pattern matching.
Available options are:
- * recent: Specify recent hours/min of logs to scan. The recent offset specified is taken relative to the =end= time specified, job end time or the current system time if the job is still running in that order of precedence. For eg: recent=3h or recent=30m will fetch logs starting 3 hours/30 minutes before the end time and up to the end time. H/h is used to denote hour and M/m is used to denote minutes. If no suffix is specified default is hours.
- * start: Start time for scanning logs. Default is start time of the job. User can provide a valid date or offset similar to =recent= option. Valid date formats are "yyyy-MM-dd'T'HH:mm'Z'" and "yyyy-MM-dd HH:mm:ss,SSS". When an offset is specified, it is calculated relative to the start time of the job. For eg: start=2h will fetch logs starting 2 hours after the job was started.
- * end: End time for scanning logs. Default is end time of the job or current system time if the job is still running. User can provide a valid date or offset similar to =start= option. When an offset is specified, it is calculated relative to start time i.e job start time . For eg: end=2h will fetch logs from start time and start time plus 2 hours.
+
+ * recent: Specify recent hours/min of logs to scan. The recent offset specified is taken relative to the `end` time specified, job end time or the current system time if the job is still running in that order of precedence. For eg: recent=3h or recent=30m will fetch logs starting 3 hours/30 minutes before the end time and up to the end time. H/h is used to denote hour and M/m is used to denote minutes. If no suffix is specified default is hours.
+ * start: Start time for scanning logs. Default is start time of the job. User can provide a valid date or offset similar to `recent` option. Valid date formats are "yyyy-MM-dd'T'HH:mm'Z'" and "yyyy-MM-dd HH:mm:ss,SSS". When an offset is specified, it is calculated relative to the start time of the job. For eg: start=2h will fetch logs starting 2 hours after the job was started.
+ * end: End time for scanning logs. Default is end time of the job or current system time if the job is still running. User can provide a valid date or offset similar to `start` option. When an offset is specified, it is calculated relative to start time i.e job start time . For eg: end=2h will fetch logs from start time and start time plus 2 hours.
* loglevel : Multiple log levels separated by "|" can be specified. Supported log levels are ALL, DEBUG, ERROR, INFO, TRACE, WARN, FATAL.
* text: String to search in logs.
* limit : Limit number of line to be searched. Log search will end when when n lines(excluding stack-trace) have been matched.
@@ -763,7 +801,8 @@
Examples.
Searching log with log level ERROR or WARN will only give log with Error and Warning (with stack-trace) only.
This will be useful if job has failed and user want to find error logs with exception.
-<verbatim>
+
+```
$ ./oozie job -log 0000006-140319184715726-oozie-puru-W -logfilter loglevel=WARN\;limit=3 -oozie http://localhost:11000/oozie/
2014-03-20 10:01:52,977 WARN ActionStartXCommand:542 - SERVER[ ] USER[-] GROUP[-] TOKEN[] APP[map-reduce-wf] JOB[0000006-140319184715726-oozie-puru-W] ACTION[0000006-140319184715726-oozie-puru-W@:start:] [***0000006-140319184715726-oozie-puru-W@:start:***]Action status=DONE
@@ -787,20 +826,22 @@
at org.apache.hadoop.mapred.JobTracker.submitJob(JobTracker.java:3613)
... 12 more
$
-</verbatim>
+```
Search with specific text and recent option.
-<verbatim>
+
+```
$ ./oozie job -log 0000003-140319184715726-oozie-puru-C -logfilter text=Missing\;limit=4\;recent=1h -oozie http://localhost:11000/oozie/
2014-03-20 09:59:50,329 INFO CoordActionInputCheckXCommand:539 - SERVER[ ] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0000003-140319184715726-oozie-puru-C] ACTION[0000003-140319184715726-oozie-puru-C@1] [0000003-140319184715726-oozie-puru-C@1]::CoordActionInputCheck:: Missing deps:hdfs://localhost:9000/user/purushah/examples/input-data/rawLogs/
2014-03-20 09:59:50,330 INFO CoordActionInputCheckXCommand:539 - SERVER[ ] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0000003-140319184715726-oozie-puru-C] ACTION[0000003-140319184715726-oozie-puru-C@1] [0000003-140319184715726-oozie-puru-C@1]::ActionInputCheck:: In checkListOfPaths: hdfs://localhost:9000/user/purushah/examples/input-data/rawLogs/ is Missing.
2014-03-20 10:02:19,087 INFO CoordActionInputCheckXCommand:539 - SERVER[ ] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0000003-140319184715726-oozie-puru-C] ACTION[0000003-140319184715726-oozie-puru-C@2] [0000003-140319184715726-oozie-puru-C@2]::CoordActionInputCheck:: Missing deps:hdfs://localhost:9000/user/purushah/examples/input-data/rawLogs/
2014-03-20 10:02:19,088 INFO CoordActionInputCheckXCommand:539 - SERVER[ ] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0000003-140319184715726-oozie-puru-C] ACTION[0000003-140319184715726-oozie-puru-C@2] [0000003-140319184715726-oozie-puru-C@2]::ActionInputCheck:: In checkListOfPaths: hdfs://localhost:9000/user/purushah/examples/input-data/rawLogs/ is Missing.
$
-</verbatim>
+```
Search example with specific date range.
-<verbatim>
+
+```
$ ./oozie job -log 0000003-140319184715726-oozie-puru-C -logfilter "start=2014-03-20 10:00:57,063;end=2014-03-20 10:10:57,063" -oozie http://localhost:11000/oozie/
2014-03-20 10:00:57,063 INFO CoordActionUpdateXCommand:539 - SERVER[ ] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0000003-140319184715726-oozie-puru-C] ACTION[0000003-140319184715726-oozie-puru-C@1] Updating Coordinator action id :0000003-140319184715726-oozie-puru-C@1 status to KILLED, pending = 0
2014-03-20 10:02:18,967 INFO CoordMaterializeTransitionXCommand:539 - SERVER[ ] USER[-] GROUP[-] TOKEN[] APP[aggregator-coord] JOB[0000003-140319184715726-oozie-puru-C] ACTION[-] materialize actions for tz=Coordinated Universal Time,
@@ -811,15 +852,16 @@
2014-03-20 10:02:18,971 WARN CoordELFunctions:542 - SERVER[ ] USER[-] GROUP[-] TOKEN[] APP[aggregator-coord] JOB[0000003-140319184715726-oozie-puru-C] ACTION[-] If the initial instance of the dataset is later than the current-instance specified, such as coord:current(-200) in this case, an empty string is returned. This means that no data is available at the current-instance specified by the user and the user could try modifying his initial-instance to an earlier time.
2014-03-20 10:02:18,975 INFO CoordMaterializeTransitionXCommand:539 - SERVER[ ] USER[-] GROUP[-] TOKEN[] APP[aggregator-coord] JOB[0000003-140319184715726-oozie-puru-C] ACTION[-] [0000003-140319184715726-oozie-puru-C]: all actions have been materialized, set pending to true
2014-03-20 10:02:18,975 INFO CoordMaterializeTransitionXCommand:539 - SERVER[ ] USER[-] GROUP[-] TOKEN[] APP[aggregator-coord] JOB[0000003-140319184715726-oozie-puru-C] ACTION[-] Coord Job status updated to = RUNNING
-</verbatim>
+```
----+++ Dryrun of Coordinator Job
+### Dryrun of Coordinator Job
* This feature is only supported in Oozie 2.0 or later.
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -dryrun -config job.properties
***coordJob after parsing: ***
@@ -856,47 +898,49 @@
</coordinator-app>
------------------------------------------------------------------------------------------------------------------------------------
-</verbatim>
+```
-The =dryrun= option tests running a coordinator job with given job properties and does not create the job.
+The `dryrun` option tests running a coordinator job with given job properties and does not create the job.
The parameters for the job must be provided in a file, either a Java Properties file (.properties) or a Hadoop XML
-Configuration file (.xml). This file must be specified with the <code>-config</code> option.
+Configuration file (.xml). This file must be specified with the `-config` option.
-The coordinator application path must be specified in the file with the =oozie.coord.application.path= property. The
+The coordinator application path must be specified in the file with the `oozie.coord.application.path` property. The
specified path must be an HDFS path.
----+++ Dryrun of Workflow Job
+### Dryrun of Workflow Job
* This feature is only supported in Oozie 3.3.2 or later.
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -dryrun -config job.properties
OK
-</verbatim>
+```
-The =dryrun= option tests running a workflow job with given job properties and does not create the job.
+The `dryrun` option tests running a workflow job with given job properties and does not create the job.
The parameters for the job must be provided in a file, either a Java Properties file (.properties) or a Hadoop XML
-Configuration file (.xml). This file must be specified with the <code>-config</code> option.
+Configuration file (.xml). This file must be specified with the `-config` option.
-The workflow application path must be specified in the file with the =oozie.wf.application.path= property. The
+The workflow application path must be specified in the file with the `oozie.wf.application.path` property. The
specified path must be an HDFS path.
-If the workflow is accepted (i.e. Oozie is able to successfully read and parse it), it will return ="OK"=; otherwise, it will return
+If the workflow is accepted (i.e. Oozie is able to successfully read and parse it), it will return `"OK"`; otherwise, it will return
an error message describing why it was rejected.
----+++ Dryrun of Bundle Job
+### Dryrun of Bundle Job
* This feature is only supported in Oozie 5.1 or later.
Example:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -dryrun -config job.properties
***Bundle job after parsing: ***
@@ -929,17 +973,17 @@
</coordinator>
</bundle-app>
-</verbatim>
+```
-The =dryrun= option tests running a bundle job with given job properties and does not create the job.
+The `dryrun` option tests running a bundle job with given job properties and does not create the job.
The parameters for the job must be provided in a file, either a Java Properties file (.properties) or a Hadoop XML
-Configuration file (.xml). This file must be specified with the <code>-config</code> option.
+Configuration file (.xml). This file must be specified with the `-config` option.
If the bundle job is accepted (i.e. Oozie is able to successfully read and parse it), it will return the parsed bundle job in xml;
otherwise, it will return an error message describing why it was rejected.
----+++ Updating coordinator definition and properties
+### Updating coordinator definition and properties
Existing coordinator definition will be replaced by new definition. The refreshed coordinator would keep the same coordinator ID, state, and coordinator actions.
All created coord action(including in WAITING) will use old configuration.
One can rerun actions with -refresh option, -refresh option will use new configuration to rerun coord action
@@ -948,13 +992,14 @@
Update command with -dryrun will show coordinator definition and properties differences.
Config option is optional, if not specified existing coordinator property is used to find coordinator path.
-Update command doesn't allow update of coordinator name, frequency, start time, end time and timezone and will fail on an attempt to change any of them. To change end time of coordinator use the =-change= command.
+Update command doesn't allow update of coordinator name, frequency, start time, end time and timezone and will fail on an attempt to change any of them. To change end time of coordinator use the `-change` command.
To change the entire XML for a running coordinator, hdfs path for the new XML can be specified
-as =oozie.coord.application.path= in job.properties. Then, use <code>-config job.properties</code> in the update command.
+as `oozie.coord.application.path` in job.properties. Then, use `-config job.properties` in the update command.
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -config job.properties -update 0000005-140402104721140-oozie-puru-C -dryrun
.
**********Job definition changes**********
@@ -994,62 +1039,66 @@
<value>localhost:9001</value>
</property>
**********************************
-</verbatim>
+```
----+++ Ignore a Coordinator Job
+### Ignore a Coordinator Job
Example:
-<verbatim>
+
+```
$oozie job -ignore <coord_Job_id>
-</verbatim>
+```
-The =ignore= option changes a coordinator job in =KILLED=, =FAILED= to =IGNORED= state.
-When a coordinator job in a bundle is in =IGNORED= state, the coordinator job doesn't impact the state of the bundle job.
-For example, when a coordinator job in a bundle failed and afterwards ignored, the bundle job becomes =SUCCEEDED= instead of =DONEWITHERROR= as long as other coordinator jobs in the bundle succeeded.
- A ignored coordinator job can be changed to =RUNNING= using -change command.
- Refer to the [[DG_CommandLineTool#Changing_endtimeconcurrencypausetimestatus_of_a_Coordinator_Job][Coordinator job change command]] for details.
+The `ignore` option changes a coordinator job in `KILLED`, `FAILED` to `IGNORED` state.
+When a coordinator job in a bundle is in `IGNORED` state, the coordinator job doesn't impact the state of the bundle job.
+For example, when a coordinator job in a bundle failed and afterwards ignored, the bundle job becomes `SUCCEEDED` instead of `DONEWITHERROR` as long as other coordinator jobs in the bundle succeeded.
+ A ignored coordinator job can be changed to `RUNNING` using -change command.
+ Refer to the [Coordinator job change command](DG_CommandLineTool.html#Changing_endtimeconcurrencypausetimestatus_of_a_Coordinator_Job) for details.
----+++ Ignore a Coordinator Action or Multiple Coordinator Actions
+### Ignore a Coordinator Action or Multiple Coordinator Actions
Example:
-<verbatim>
+
+```
$oozie job -ignore <coord_Job_id> -action 1,3-4,7-40
-</verbatim>
-The =ignore= option changes a coordinator action(s) in terminal state (=KILLED=, =FAILED=, =TIMEDOUT=) to =IGNORED= state, while not changing the state of the coordinator job.
-When a coordinator action is in =IGNORED= state, the action doesn't impact the state of a coordinator job.
-For example, when a coordinator action failed and afterwards ignored, a coordinator job becomes =SUCCEEDED= instead of =DONEWITHERROR= as long
+```
+The `ignore` option changes a coordinator action(s) in terminal state (`KILLED`, `FAILED`, `TIMEDOUT`) to `IGNORED` state, while not changing the state of the coordinator job.
+When a coordinator action is in `IGNORED` state, the action doesn't impact the state of a coordinator job.
+For example, when a coordinator action failed and afterwards ignored, a coordinator job becomes `SUCCEEDED` instead of `DONEWITHERROR` as long
as other coordinator actions succeeded.
A ignored coordinator action can be rerun using -rerun command.
-Refer to the [[DG_CoordinatorRerun][Rerunning Coordinator Actions]] for details.
-When a workflow job of a ignored coordinator action is rerun, the coordinator action becomes =RUNNING= state.
+Refer to the [Rerunning Coordinator Actions](DG_CoordinatorRerun.html) for details.
+When a workflow job of a ignored coordinator action is rerun, the coordinator action becomes `RUNNING` state.
----+++ Polling an Oozie job
+### Polling an Oozie job
-This command allows polling the Oozie server for an Oozie job until it reaches a completed status (e.g. =SUCCEEDED=, =KILLED=, etc).
+This command allows polling the Oozie server for an Oozie job until it reaches a completed status (e.g. `SUCCEEDED`, `KILLED`, etc).
Example:
-<verbatim>
+
+```
$ oozie job -poll <job_id> -interval 10 -timeout 60 -verbose
.
RUNNING
RUNNING
RUNNING
SUCCEEDED
-</verbatim>
+```
-The =-poll= argument takes a valid Workflow Job ID, Coordinator Job ID, Coordinator Action ID, or Bundle Job ID.
+The `-poll` argument takes a valid Workflow Job ID, Coordinator Job ID, Coordinator Action ID, or Bundle Job ID.
All other arguments are optional:
- * =verbose= will cause the job status to be printed at each poll; otherwise, there will be no output
- * =interval= allows specifying the polling interval in minutes (default is 5)
- * =timeout= allows specifying the timeout in minutes (default is 30 minutes); negative values indicate no timeout
----+++ Changing job SLA definition and alerting
+ * `verbose` will cause the job status to be printed at each poll; otherwise, there will be no output
+ * `interval` allows specifying the polling interval in minutes (default is 5)
+ * `timeout` allows specifying the timeout in minutes (default is 30 minutes); negative values indicate no timeout
+
+### Changing job SLA definition and alerting
* slaenable command can be used to enable job sla alerts.
* sladisable command can be used to disable job sla alerts.
* slachange command can be used to change sla job definition.
@@ -1059,20 +1108,22 @@
* Sla commands with -action or -date parameter will be applied to only non terminated actions.
Eg.
- <verbatim>
+
+```
$oozie job -slaenable <coord_Job_id> [-action 1,3-4,7-40] [-date 2009-01-01T01:00Z::2009-05-31T23:59Z,2009-11-10T01:00Z::2009-12-31T22:00Z]
$oozie job -sladisable <coord_Job_id> [-action 1,3-4,7-40] [-date 2009-01-01T01:00Z::2009-05-31T23:59Z,2009-11-10T01:00Z::2009-12-31T22:00Z]
- $oozie job -slachange <coord_Job_id> [-action 1,3-4,7-40] [-date 2009-01-01T01:00Z::2009-05-31T23:59Z,2009-11-10T01:00Z::2009-12-31T22:00Z] -value 'sla-max-duration=${10 * MINUTES};sla-should-end=${30 * MINUTES};sla-max-duration=${30 * MINUTES}'
+ $oozie job -slachange <coord_Job_id> [-action 1,3-4,7-40] [-date 2009-01-01T01:00Z::2009-05-31T23:59Z,2009-11-10T01:00Z::2009-12-31T22:00Z] -value 'sla-max-duration=${10 ** MINUTES};sla-should-end=${30 ** MINUTES};sla-max-duration=${30 * MINUTES}'
$oozie job -slaenable <bundle_job_id> [-action 1,3-4,7-40] [-date 2009-01-01T01:00Z::2009-05-31T23:59Z,2009-11-10T01:00Z::2009-12-31T22:00Z] [-coordinator <List_of_coord_names/ids]
- </verbatim>
+```
----+++ Getting missing dependencies of coordinator action(s)
+### Getting missing dependencies of coordinator action(s)
* Coordination action id can be specified directly for getting missing dependencies of a single action.
* To get information on multiple actions, either -action or -date option can be specified with the coordinator job id.
* missingdeps command doesn't recompute dependencies. It list missing dependencies which were last computed.
- * Oozie checks missing dependencies sequentially, and it will stop on first missing dependency. =Blocked On= is the first missing dependency for action. So, there could be a chance that Oozie will report some missing dependencies, but it might be present. To resolve the waiting issue, one should fix the blockedOn missing dependency.
+ * Oozie checks missing dependencies sequentially, and it will stop on first missing dependency. `Blocked On` is the first missing dependency for action. So, there could be a chance that Oozie will report some missing dependencies, but it might be present. To resolve the waiting issue, one should fix the blockedOn missing dependency.
* For input logic, missingdeps command doesn't compute input-logic expression. It will report everything which is missing or not computed.
-<verbatim>
+
+```
$oozie job -oozie http://localhost:11000/oozie -missingdeps 0000000-170104141851590-oozie-puru-C -action 1
$oozie job -oozie http://localhost:11000/oozie -missingdeps 0000000-170104141851590-oozie-puru-C@1
.
@@ -1090,34 +1141,37 @@
hdfs://localhost:9000/user/purushah/examples/input-data/rawLogs/2010/01/01/00/20/_SUCCESS
hdfs://localhost:9000/user/purushah/examples/input-data/rawLogs/2010/01/01/00/00/_SUCCESS
$
-</verbatim>
+```
----+++ Checking a workflow definition generated by a Fluent Job API jar file
+### Checking a workflow definition generated by a Fluent Job API jar file
Since Oozie 5.1.0.
Generate a workflow definition given the Fluent Job API jar file supplied at command line, and check for its correctness.
-*Preconditions:*
+**Preconditions:**
+
* the Fluent Job API jar file has to be present and readable by the current user at the local path provided
* the folder containing the Fluent Job API jar file provided has to be writable by the current user, since the generated workflow
definition is written there
-If the =-verbose= option is provided, a lot more debugging output, including the generated workflow definition, is given.
+If the `-verbose` option is provided, a lot more debugging output, including the generated workflow definition, is given.
For more information what an Fluent Job API jar file is, how to build it etc.,
-refer to [[DG_FluentJobAPI#AE.A_Appendix_A_API_JAR_format][Fluent Job API - API JAR format]].
+refer to [Fluent Job API - API JAR format](DG_FluentJobAPI.html#AE.A_Appendix_A_API_JAR_format).
-*Example:*
+**Example:**
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -validatejar /tmp/workflow-api-jar.jar
Valid workflow-app
-</verbatim>
+```
-*Example (verbose):*
+**Example (verbose):**
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -validatejar /tmp/workflow-api-jar.jar -verbose
Checking API jar:/tmp/workflow-api-jar.jar
Loading API jar /tmp/workflow-api-jar.jar
@@ -1131,39 +1185,42 @@
Servlet response is:
Valid workflow-app
API jar is valid.
-</verbatim>
+```
----+++ Submitting a workflow definition generated by a Fluent Job API jar file
+### Submitting a workflow definition generated by a Fluent Job API jar file
Since Oozie 5.1.0.
Generate a workflow definition given the Fluent Job API jar file supplied at command line, write it to a provided or generated HDFS
location, and submit.
-*Preconditions:*
+**Preconditions:**
+
* all the parameters that are present in the workflow definition have to be supplied either as command line parameters or via
- =job.properties= passed along with the =-config= option
+ `job.properties` passed along with the `-config` option
* the Fluent Job API jar file has to be present and readable by the current user at the local path provided
* the folder containing the Fluent Job API jar file provided has to be writable by the current user, since the generated workflow
definition is written there
- * the HDFS folder either given by =-Doozie.wf.application.path= command line parameter, or in its absence contained by
- =oozie-site.xml#oozie.client.jobs.application.generated.path= has to be writable by the current user
+ * the HDFS folder either given by `-Doozie.wf.application.path` command line parameter, or in its absence contained by
+ `oozie-site.xml#oozie.client.jobs.application.generated.path` has to be writable by the current user
-If the =-verbose= option is provided, a lot more debugging output, including the generated workflow definition, is given.
+If the `-verbose` option is provided, a lot more debugging output, including the generated workflow definition, is given.
For more information what an Fluent Job API jar file is, how to build it etc., refer to
-[[DG_FluentJobAPI#AE.A_Appendix_A_API_JAR_format][Fluent Job API - API JAR format]].
+[Fluent Job API - API JAR format](DG_FluentJobAPI.html#AE.A_Appendix_A_API_JAR_format).
-*Example:*
+**Example:**
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -submitjar /tmp/workflow-api-jar.jar -config /tmp/job.properties
job: 0000009-180107110323219-oozie-oozi-W
-</verbatim>
+```
-*Example (verbose):*
+**Example (verbose):**
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -submitjar /tmp/workflow-api-jar.jar -config /tmp/job.properties -verbose
Submitting a job based on API jar: /tmp/workflow-api-jar.jar
Loading API jar /tmp/workflow-api-jar.jar
@@ -1175,39 +1232,42 @@
.
job: 0000010-180107110323219-oozie-oozi-W
Job based on API jar submitted successfully.
-</verbatim>
+```
----+++ Running a workflow definition generated by a Fluent Job API jar file
+### Running a workflow definition generated by a Fluent Job API jar file
Since Oozie 5.1.0.
Generate a workflow definition given the Fluent Job API jar file supplied at command line, write it to a provided or generated HDFS
location, submit and run.
-*Preconditions:*
+**Preconditions:**
+
* all the parameters that are present in the workflow definition have to be supplied either as command line parameters or via
- =job.properties= passed along with the =-config= option
+ `job.properties` passed along with the `-config` option
* the Fluent Job API jar file has to be present and readable by the current user at the local path provided
* the folder containing the Fluent Job API jar file provided has to be writable by the current user, since the generated workflow
definition is written there
- * the HDFS folder either given by =-Doozie.wf.application.path= command line parameter, or in its absence contained by
- =oozie-site.xml#oozie.client.jobs.application.generated.path= has to be writable by the current user
+ * the HDFS folder either given by `-Doozie.wf.application.path` command line parameter, or in its absence contained by
+ `oozie-site.xml#oozie.client.jobs.application.generated.path` has to be writable by the current user
-If the =-verbose= option is provided, a lot more debugging output, including the generated workflow definition, is given.
+If the `-verbose` option is provided, a lot more debugging output, including the generated workflow definition, is given.
For more information what an Fluent Job API jar file is, how to build it etc., refer to
-[[DG_FluentJobAPI#AE.A_Appendix_A_API_JAR_format][Fluent Job API - API JAR format]].
+[Fluent Job API - API JAR format](DG_FluentJobAPI.html#AE.A_Appendix_A_API_JAR_format).
-*Example:*
+**Example:**
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -runjar /tmp/workflow-api-jar.jar -config /tmp/job.properties
job: 0000011-180107110323219-oozie-oozi-W
-</verbatim>
+```
-*Example (verbose):*
+**Example (verbose):**
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -runjar /tmp/workflow-api-jar.jar -config /tmp/job.properties -verbose
Submitting a job based on API jar: /tmp/workflow-api-jar.jar
Loading API jar /tmp/workflow-api-jar.jar
@@ -1219,15 +1279,16 @@
.
job: 0000010-180107110323219-oozie-oozi-W
Job based on API jar run successfully.
-</verbatim>
+```
----++ Jobs Operations
+## Jobs Operations
----+++ Checking the Status of multiple Workflow Jobs
+### Checking the Status of multiple Workflow Jobs
Example:
-<verbatim>
+
+```
$ oozie jobs -oozie http://localhost:11000/oozie -localtime -len 2 -filter status=RUNNING
.
Job Id Workflow Name Status Run User Group Created Started Ended
@@ -1235,20 +1296,20 @@
4-20090527151008-oozie-joe hadoopel-wf RUNNING 0 joe other 2009-05-27 15:34 +0530 2009-05-27 15:34 +0530 -
0-20090527151008-oozie-joe hadoopel-wf RUNNING 0 joe other 2009-05-27 15:15 +0530 2009-05-27 15:15 +0530 -
.----------------------------------------------------------------------------------------------------------------------------------------------------------------
-</verbatim>
+```
-The =jobs= sub-command will display information about multiple jobs.
+The `jobs` sub-command will display information about multiple jobs.
-The =offset= and =len= option specified the offset and number of jobs to display, default values are =1= and =100=
+The `offset` and `len` option specified the offset and number of jobs to display, default values are `1` and `100`
respectively.
-The =localtime= option displays times in local time, if not specified times are displayed in GMT.
+The `localtime` option displays times in local time, if not specified times are displayed in GMT.
-The =verbose= option gives more detailed information for each job.
+The `verbose` option gives more detailed information for each job.
A filter can be specified after all options.
-The =filter=option syntax is: <code>[NAME=VALUE][;NAME=VALUE]*</code>.
+The `filter`option syntax is: `[NAME=VALUE][;NAME=VALUE]*`.
Valid filter names are:
@@ -1259,20 +1320,21 @@
* status: the status of the job.
* startcreatedtime: the start of time window in specifying createdtime range filter.
* endcreatedtime: the end of time window in specifying createdtime range filter
- * sortby: order the results. Supported values for =sortby= are: =createdTime= and =lastModifiedTime=
+ * sortby: order the results. Supported values for `sortby` are: `createdTime` and `lastModifiedTime`
The query will do an AND among all the filter names. The query will do an OR among all the filter values for the same
name. Multiple values must be specified as different name value pairs.
-startCreatedTime and endCreatedTime should be specified either in *ISO8601 (UTC)* format (*yyyy-MM-dd'T'HH:mm'Z'*) or a offset value in days or hours from the current time. For example, -2d means the current time - 2 days. -3h means the current time - 3 hours, -5m means the current time - 5 minutes
+startCreatedTime and endCreatedTime should be specified either in **ISO8601 (UTC)** format (**yyyy-MM-dd'T'HH:mm'Z'**) or a offset value in days or hours from the current time. For example, -2d means the current time - 2 days. -3h means the current time - 3 hours, -5m means the current time - 5 minutes
----+++ Checking the Status of multiple Coordinator Jobs
+### Checking the Status of multiple Coordinator Jobs
* This feature is only supported in Oozie 2.0 or later.
Example:
-<verbatim>
+
+```
$ oozie jobs -oozie http://localhost:11000/oozie -jobtype coordinator
.
Job ID App Name Status Freq Unit Started Next Materialized
@@ -1281,9 +1343,9 @@
.----------------------------------------------------------------------------------------------------------------------------------------------------------------
0003823-100531045722929-oozie-wrkf-C coordcal2880minutescurrent SUCCEEDED 2880 MINUTE 2010-02-01 16:30 2010-02-05 16:30
.----------------------------------------------------------------------------------------------------------------------------------------------------------------
-</verbatim>
+```
-The =jobtype= option specified the job type to display, default value is 'wf'. To see the coordinator jobs, value is 'coordinator'.
+The `jobtype` option specified the job type to display, default value is 'wf'. To see the coordinator jobs, value is 'coordinator'.
Valid filter names are:
@@ -1293,15 +1355,16 @@
* status: the status of the job.
* frequency: the frequency of the Coordinator job.
* unit: the time unit. It can take one of the following four values: months, days, hours or minutes. Time unit should be added only when frequency is specified.
- * sortby: order the results. Supported values for =sortby= are: =createdTime= and =lastModifiedTime=
+ * sortby: order the results. Supported values for `sortby` are: `createdTime` and `lastModifiedTime`
----+++ Checking the Status of multiple Bundle Jobs
+### Checking the Status of multiple Bundle Jobs
* This feature is only supported in Oozie 3.0 or later.
Example:
-<verbatim>
+
+```
$ oozie jobs -oozie http://localhost:11000/oozie -jobtype bundle
Job ID Bundle Name Status Kickoff Created User Group
.------------------------------------------------------------------------------------------------------------------------------------
@@ -1311,15 +1374,16 @@
.------------------------------------------------------------------------------------------------------------------------------------
0000000-110322105610515-oozie-chao-B BUNDLE-TEST DONEWITHERROR2012-01-15 00:24 2011-03-22 17:58 joe users
.------------------------------------------------------------------------------------------------------------------------------------
-</verbatim>
+```
-The =jobtype= option specified the job type to display, default value is 'wf'. To see the bundle jobs, value is 'bundle'.
+The `jobtype` option specified the job type to display, default value is 'wf'. To see the bundle jobs, value is 'bundle'.
----+++ Bulk kill, suspend or resume multiple jobs
+### Bulk kill, suspend or resume multiple jobs
Example:
-<verbatim>
+
+```
$ oozie jobs -oozie http://localhost:11000/oozie -kill|-suspend|-resume -filter name=cron-coord -jobtype coordinator
The following jobs have been killed|suspended|resumed
Job ID App Name Status Freq Unit Started Next Materialized
@@ -1330,21 +1394,21 @@
.------------------------------------------------------------------------------------------------------------------------------------
0000000-150224141553231-oozie-bzha-C cron-coord KILLED 10 MINUTE 2015-02-25 22:00 GMT -
.------------------------------------------------------------------------------------------------------------------------------------
-</verbatim>
+```
The above command will kill, suspend, or resume all the coordinator jobs with name of "cron-coord" starting with offset 1
to 50.
-The =jobs= sub-command will bulk modify all the jobs that satisfy the filter, len, offset, and jobtype options when adding
+The `jobs` sub-command will bulk modify all the jobs that satisfy the filter, len, offset, and jobtype options when adding
a -kill|-suspend|-resume option. Another way to think about is the subcommand works to modify all the jobs that will be
displayed if the write option(kill|suspend|resume) is not there.
-The =offset= and =len= option specified the offset and number of jobs to be modified, default values are =1= and =50=
+The `offset` and `len` option specified the offset and number of jobs to be modified, default values are `1` and `50`
respectively.
-The =jobtype= option specifies the type of jobs to be modified, be it "wf", "coordinator" or "bundle". default value is "wf".
+The `jobtype` option specifies the type of jobs to be modified, be it "wf", "coordinator" or "bundle". default value is "wf".
A filter can be specified after all options.
-The =filter=option syntax is: <code>[NAME=VALUE][;NAME=VALUE]*</code>.
+The `filter`option syntax is: `[NAME=VALUE][;NAME=VALUE]*`.
Valid filter names are:
@@ -1354,21 +1418,22 @@
* status: the status of the job.
* frequency: the frequency of the Coordinator job.
* unit: the time unit. It can take one of the following four values: months, days, hours or minutes. Time unit should be added only when frequency is specified.
- * sortby: order the results. Supported values for =sortby= are: =createdTime= and =lastModifiedTime=
+ * sortby: order the results. Supported values for `sortby` are: `createdTime` and `lastModifiedTime`
The query will do an AND among all the filter names. The query will do an OR among all the filter values for the same
name. Multiple values must be specified as different name value pairs.
The following example shows how to suspend the first 20 bundle jobs whose name is "bundle-app":
-<verbatim>
-$ oozie jobs -oozie http://localhost:11000/oozie -suspend -filter name=bundle-app -jobtype bundle -len 20
-</verbatim>
----+++ Bulk monitoring for jobs launched via Bundles
+```
+$ oozie jobs -oozie http://localhost:11000/oozie -suspend -filter name=bundle-app -jobtype bundle -len 20
+```
+
+### Bulk monitoring for jobs launched via Bundles
* This command-line query helps to directly query for a bulk of jobs using a set of rich filters.
-The jobs need to have a parent *Bundle*, and this performs a deep query to provide results about all the workflows that satisfy your filters.
+The jobs need to have a parent **Bundle**, and this performs a deep query to provide results about all the workflows that satisfy your filters.
These results display relevant job-ids, app-names, and error message (if any) and are most helpful when you need to monitor a bulk of jobs and get a gist,
while avoiding typing multiple queries.
@@ -1376,20 +1441,22 @@
Example 1:
-<verbatim>
+
+```
$ oozie jobs -oozie http://localhost:11000/oozie -bulk bundle=bundle-app-1
.
Bundle Name Bundle ID Coord Name Coord Action ID External ID Status Created Time Error Message
.-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
bundle-app-1 0000000-130408151805946-oozie-chit-B coord-1 0000001-130408151805946-oozie-chit-C@1 0000002-130408151805946-oozie-chit-W KILLED 2013-04-08 22:20 GMT null
.-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-</verbatim>
+```
Example 2: This example illustrates giving multiple arguments and -verbose option.
_NOTE: The filter string after -bulk should be enclosed within quotes_
-<verbatim>
+
+```
.
$ oozie jobs -oozie http://localhost:11000/oozie -bulk 'bundle=bundle-app-2;actionstatus=SUCCEEDED' -verbose
.
@@ -1404,7 +1471,7 @@
User : user_xyz
Error Message : -
.------------------------------------------------------------------------------------------------------------------------------------
-</verbatim>
+```
You can type 'help' to view usage for the CLI options and filters available. Namely:
@@ -1414,7 +1481,8 @@
* startcreatedtime/endcreatedtime: specify boundaries for the created-time. Only jobs created within this window are included.
* startscheduledtime/endscheduledtime: specify boundaries for the nominal-time. Only jobs scheduled to run within this window are included.
-<verbatim>
+
+```
$ oozie jobs <OPTIONS> : jobs status
-bulk <arg> key-value pairs to filter bulk jobs response. e.g.
bundle=<B>\;coordinators=<C>\;actionstatus=<S>\;
@@ -1423,57 +1491,61 @@
coordinators and actionstatus can be multiple comma
separated values. "bundle" and "coordinators" are 'names' of those jobs.
Bundle name is mandatory, other params are optional
-</verbatim>
+```
Similar to the usual jobs filter, different filter arguments here should be separated by semicolon (;).
----++ Admin Operations
+## Admin Operations
----+++ Checking the Status of the Oozie System
+### Checking the Status of the Oozie System
Example:
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -status
.
Safemode: OFF
-</verbatim>
+```
It returns the current status of the Oozie system.
----+++ Changing the Status of the Oozie System
+### Changing the Status of the Oozie System
* This feature is only supported in Oozie 2.0 or later.
Example:
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -systemmode [NORMAL|NOWEBSERVICE|SAFEMODE]
.
Safemode: ON
-</verbatim>
+```
It returns the current status of the Oozie system.
----+++ Displaying the Build Version of the Oozie System
+### Displaying the Build Version of the Oozie System
Example:
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -version
.
Oozie server build version: 2.0.2.1-0.20.1.3092118008--
-</verbatim>
+```
It returns the Oozie server build version.
----+++ Displaying the queue dump of the Oozie System
+### Displaying the queue dump of the Oozie System
* This feature is for administrator debugging purpose
Example:
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -queuedump
.
[Server Queue Dump]:
@@ -1481,29 +1553,31 @@
(coord_action_ready,1)
(action.check,2)
-</verbatim>
+```
It returns the Oozie server current queued commands.
----+++ Displaying the list of available Oozie Servers
+### Displaying the list of available Oozie Servers
Example:
-<verbatim>
+
+```
$ oozie admin -oozie http://hostA:11000/oozie -servers
hostA : http://hostA:11000/oozie
hostB : http://hostB:11000/oozie
hostC : http://hostC:11000/oozie
-</verbatim>
+```
-It returns a list of available Oozie Servers. This is useful when Oozie is configured for [[AG_Install#HA][High Availability]]; if
+It returns a list of available Oozie Servers. This is useful when Oozie is configured for [High Availability](AG_Install.html#HA); if
not, it will simply return the one Oozie Server.
----+++ Displaying the Oozie server configuration
+### Displaying the Oozie server configuration
Example:
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -configuration
local.realm : LOCALHOST
oozie.JobCommand.job.console.url : http://localhost:11000/oozie?job=
@@ -1515,19 +1589,20 @@
oozie.action.max.output.data : 2048
oozie.action.retries.max : 3
...
-</verbatim>
+```
It returns a list of the configuration properties and values from oozie-site.xml and oozie-default.xml being used by the Oozie
server.
----+++ Displaying the Oozie server OS environment
+### Displaying the Oozie server OS environment
Example:
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -osenv
...
-JETTY_OPTS : -Doozie.home.dir=/Users/asasvari/dev/oozie -Doozie.config.dir=/Users/asasvari/dev/oozie/conf -Doozie.log.dir=/Users/asasvari/dev/oozie/logs -Doozie.data.dir=/Users/asasvari/dev/oozie/data -Doozie.config.file=oozie-site.xml -Doozie.log4j.file=oozie-log4j.properties -Doozie.log4j.reload=10 -Djava.library.path= -cp /Users/asasvari/dev/oozie/embedded-oozie-server/*:/Users/asasvari/dev/oozie/embedded-oozie-server/dependency/*:/Users/asasvari/dev/oozie/lib/*:/Users/asasvari/dev/oozie/libtools/*:/Users/asasvari/dev/oozie/embedded-oozie-server
+JETTY_OPTS : -Doozie.home.dir=/Users/asasvari/dev/oozie -Doozie.config.dir=/Users/asasvari/dev/oozie/conf -Doozie.log.dir=/Users/asasvari/dev/oozie/logs -Doozie.data.dir=/Users/asasvari/dev/oozie/data -Doozie.config.file=oozie-site.xml -Doozie.log4j.file=oozie-log4j.properties -Doozie.log4j.reload=10 -Djava.library.path= -cp /Users/asasvari/dev/oozie/embedded-oozie-server/**:/Users/asasvari/dev/oozie/embedded-oozie-server/dependency/**:/Users/asasvari/dev/oozie/lib/**:/Users/asasvari/dev/oozie/libtools/**:/Users/asasvari/dev/oozie/embedded-oozie-server
JETTY_OUT : /Users/asasvari/dev/oozie/logs/jetty.out
JETTY_PID_FILE : /Users/asasvari/dev/oozie/embedded-oozie-server/oozie.pid
OOZIE_CONFIG : /Users/asasvari/dev/oozie/conf
@@ -1538,15 +1613,16 @@
OOZIE_LOG4J_FILE : oozie-log4j.properties
OOZIE_LOG4J_RELOAD : 10
...
-</verbatim>
+```
It returns a list of OS environment variables in the Oozie server.
----+++ Displaying the Oozie server Java system properties
+### Displaying the Oozie server Java system properties
Example:
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -javasysprops
...
oozie.config.dir : /Users/asasvari/dev/oozie/conf
@@ -1557,17 +1633,18 @@
oozie.log4j.file : oozie-log4j.properties
oozie.log4j.reload : 10
...
-</verbatim>
+```
It returns a list of java system properties in the Oozie server.
----+++ Displaying the Oozie server Instrumentation
+### Displaying the Oozie server Instrumentation
Deprecated and by default disabled since 5.0.0.
Example:
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -instrumentation
COUNTERS
--------
@@ -1613,20 +1690,21 @@
total min time : 222
ticks : 1
...
-</verbatim>
+```
It returns the instrumentation from the Oozie server. Keep in mind that timers and counters that the Oozie server
hasn't incremented yet will not show up.
-*Note:* If Instrumentation is enabled, then Metrics is unavailable.
+**Note:** If Instrumentation is enabled, then Metrics is unavailable.
----+++ Displaying the Oozie server Metrics
+### Displaying the Oozie server Metrics
By default enabled since 5.0.0.
Example:
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -metrics
COUNTERS
--------
@@ -1709,69 +1787,74 @@
count : 13
standard deviation : 2.764240517940803
...
-</verbatim>
+```
It returns the metrics from the Oozie server. Keep in mind that timers and counters that the Oozie server
hasn't incremented yet will not show up.
-*Note:* If Metrics is enabled, then Instrumentation is unavailable.
+**Note:** If Metrics is enabled, then Instrumentation is unavailable.
----+++ Running purge command
+### Running purge command
Oozie admin purge command cleans up the Oozie Workflow/Coordinator/Bundle records based on the parameters.
The unit for parameters is day.
Purge command will delete the workflow records (wf=30) older than 30 days, coordinator records (coord=7) older than 7 days and
bundle records (bundle=7) older than 7 days. The limit (limit=10) defines, number of records to be fetch at a time. Turn
-(oldCoordAction=true/false) =on/off= coordinator action record purging for long running coordinators. If any of the parameter is
-not provided, then it will be taken from the =oozie-default/oozie-site= configuration.
+(oldCoordAction=true/false) `on/off` coordinator action record purging for long running coordinators. If any of the parameter is
+not provided, then it will be taken from the `oozie-default/oozie-site` configuration.
Example:
-<verbatim>
+
+```
$ oozie admin -purge wf=30\;coord=7\;bundle=7\;limit=10\;oldCoordAction=true
Purge command executed successfully
-</verbatim>
+```
----++ Validate Operations
+## Validate Operations
----+++ Validating a Workflow XML
+### Validating a Workflow XML
Example:
-<verbatim>
+
+```
$ oozie validate myApp/workflow.xml
.
Error: E0701: XML schema error, workflow.xml, org.xml.sax.SAXParseException: cvc-complex-type.2.4.a:
Invalid content was found starting with element 'xend'. One of '{"uri:oozie:workflow:0.1":decision,
"uri:oozie:workflow:0.1":fork, "uri:oozie:workflow:0.1":join, "uri:oozie:workflow:0.1":kill,
"uri:oozie:workflow:0.1":action, "uri:oozie:workflow:0.1":end}' is expected.
-</verbatim>
+```
-<verbatim>
+
+```
$ oozie validate /home/test/myApp/coordinator.xml
.
Error: E0701: XML schema error, coordinator.xml, org.xml.sax.SAXParseException; lineNumber: 4; columnNumber: 52; cvc-elt.1.a:
Cannot find the declaration of element 'coordinator-app-invalid'.
-</verbatim>
+```
-<verbatim>
+
+```
$ oozie validate hdfs://localhost:8020/user/test/myApp/bundle.xml
.
Error: E0701: XML schema error, bundle.xml, org.xml.sax.SAXParseException: cvc-complex-type.2.4.b:
The content of element 'bundle-app' is not complete. One of '{"uri:oozie:bundle:0.1":coordinator}' is expected.
-</verbatim>
+```
It performs an XML Schema validation on the specified workflow, coordinator, bundle XML file.
The XML file can be a local file or in HDFS.
----+++ Getting list of available sharelib
+### Getting list of available sharelib
This command is used to get list of available sharelib.
If the name of the sharelib is passed as an argument (regex supported) then all corresponding files are also listed.
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -shareliblist
[Available ShareLib]
oozie
@@ -1787,14 +1870,15 @@
pig
hdfs://localhost:9000/user/purushah/share/lib/lib_20131114095729/pig/pig.jar
hdfs://localhost:9000/user/purushah/share/lib/lib_20131114095729/pig/piggybank.jar
-</verbatim>
+```
----+++ Update system sharelib
+### Update system sharelib
This command makes the oozie server(s) to pick up the latest version of sharelib present
under oozie.service.WorkflowAppService.system.libpath directory based on the sharelib directory timestamp or reloads
the sharelib metafile if one is configured. The main purpose is to update the sharelib on the oozie server without restarting.
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -sharelibupdate
[ShareLib update status]
ShareLib update status]
@@ -1810,10 +1894,11 @@
host = host3:8080
status = Server not found
-</verbatim>
+```
Sharelib update for metafile configuration.
-<verbatim>
+
+```
$ oozie admin -oozie http://localhost:11000/oozie -sharelibupdate
[ShareLib update status]
host = host1
@@ -1827,18 +1912,19 @@
sharelibMetaFile = hdfs://localhost:9000/user/purushah/sharelib_metafile.property
sharelibMetaFileOldTimeStamp = Thu, 21 Nov 2013 00:40:04 GMT
sharelibMetaFileNewTimeStamp = Thu, 21 Nov 2013 01:01:25 GMT
-</verbatim>
+```
-#SLAOperations
----++ SLA Operations
+<a name="SLAOperations"></a>
+## SLA Operations
----+++ Getting a list of SLA events
+### Getting a list of SLA events
This set of sla commands are deprecated as of Oozie 4.0 with a newer SLA monitoring system.
Example:
-<verbatim>
+
+```
$ oozie sla -oozie http://localhost:11000/oozie -len 3
.
<sla-message>
@@ -1887,24 +1973,25 @@
<last-sequence-id>3</last-sequence-id>
</sla-message>
-</verbatim>
+```
-The =offset= and =len= option specified the offset and number of sla events to display, default values are =1= and =100= respectively.
+The `offset` and `len` option specified the offset and number of sla events to display, default values are `1` and `100` respectively.
-The =offset= corresponds to sequence ID of an event.
+The `offset` corresponds to sequence ID of an event.
-The max value of =len= limited by oozie server setting which defaults to '1000'. To get more than =1000= events, it is necessary to iterate based on the number of records you want.
+The max value of `len` limited by oozie server setting which defaults to '1000'. To get more than `1000` events, it is necessary to iterate based on the number of records you want.
The return message is XML format that can be easily consumed by SLA users.
----+++ Getting the SLA event with particular sequenceID
+### Getting the SLA event with particular sequenceID
* This feature is only supported in Oozie 2.0 or later.
Example: Get the SLA event with sequenceID = 3 (Note that offset corresponds to sequence ID)
-<verbatim>
+
+```
$ oozie sla -oozie http://localhost:11000/oozie -offset 2 -len 1
.
<sla-message>
@@ -1920,16 +2007,17 @@
<last-sequence-id>3</last-sequence-id>
</sla-message>
-</verbatim>
+```
----+++ Getting information about SLA events using filter
+### Getting information about SLA events using filter
* This feature is only supported in Oozie 2.0 or later.
Example:
-<verbatim>
+
+```
$ oozie sla -filter jobid=0000000-130130150445097-oozie-joe-C@1\;appname=aggregator-sla-app -len 1 -oozie http://localhost:11000/oozie
@@ -1960,11 +2048,11 @@
</event>
</sla-message>
-</verbatim>
+```
A filter can be specified after all options.
-The =filter=option syntax is: <code>[NAME=VALUE][\;NAME=VALUE]*</code>. Note <code>\</code> before semi-colon is for escape.
+The `filter`option syntax is: `[NAME=VALUE][\;NAME=VALUE]*`. Note `\` before semi-colon is for escape.
Valid filter names are:
@@ -1975,19 +2063,21 @@
name. Multiple values must be specified as different name value pairs.
----++ Pig Operations
+## Pig Operations
----+++ Submitting a pig job through HTTP
+### Submitting a pig job through HTTP
Syntax:
-<verbatim>
+
+```
$ oozie pig -file PIG-SCRIPT -config OOZIE-CONFIG [-Dkey=value] [-Pkey=value] [-X [-Dkey=value opts for Launcher/Job configuration] [Other opts to pass to Pig]]
-</verbatim>
+```
Example:
-<verbatim>
+
+```
$ oozie pig -file pigScriptFile -config job.properties -Dfs.default.name=hdfs://localhost:8020 -PINPUT=/user/me/in -POUTPUT=/user/me/out -X -Dmapred.job.queue.name=UserQueue -param_file params
.
job: 14-20090525161321-oozie-joe-W
@@ -1997,32 +2087,34 @@
mapreduce.jobtracker.kerberos.principal=ccc
dfs.namenode.kerberos.principal=ddd
oozie.libpath=hdfs://localhost:8020/user/oozie/pig/lib/
-</verbatim>
+```
The parameters for the job must be provided in a Java Properties file (.properties). jobtracker, namenode, libpath must be
specified in this file. pigScriptFile is a local file. All jar files (including pig jar file) and all other files needed by the pig
job (e.g., parameter file in above example) need to be uploaded onto HDFS under libpath beforehand. In addition to a parameter file,
specifying script parameters can be done via -Pkey=value. The workflow.xml will be created in Oozie server internally. Users can get
the workflow.xml from console or command line(-definition). The -D options passed after the -X will be placed into the generated
-workflow's <code><configuration> elements</code> (and make it to the configuration used by Pig); any other opts after -X will be
+workflow's `<configuration> elements` (and make it to the configuration used by Pig); any other opts after -X will be
passed as-is to the invoked Pig program.
Multiple -D and -P arguments can be specified.
The job will be created and run right away.
----++ Hive Operations
+## Hive Operations
----+++ Submitting a hive job through HTTP
+### Submitting a hive job through HTTP
Syntax:
-<verbatim>
+
+```
$ oozie hive -file HIVE-SCRIPT -config OOZIE-CONFIG [-Dkey=value] [-Pkey=value] [-X [-Dkey=value opts for Launcher/Job configuration] [Other opts to pass to Hive]]
-</verbatim>
+```
Example:
-<verbatim>
+
+```
$ oozie hive -file hiveScriptFile -config job.properties -Dfs.default.name=hdfs://localhost:8020 -PINPUT=/user/me/in -POUTPUT=/user/me/out -X -Dmapred.job.queue.name=UserQueue -v
.
job: 14-20090525161321-oozie-joe-W
@@ -2032,39 +2124,42 @@
mapreduce.jobtracker.kerberos.principal=ccc
dfs.namenode.kerberos.principal=ddd
oozie.libpath=hdfs://localhost:8020/user/oozie/hive/lib/
-</verbatim>
+```
The parameters for the job must be provided in a Java Properties file (.properties). jobtracker, namenode, libpath must be
specified in this file. hiveScriptFile is a local file. All jar files (including hive jar file) and all other files needed by the
hive job need to be uploaded onto HDFS under libpath beforehand. Specifying script parameters can be done via -Pkey=value. The
workflow.xml will be created in Oozie server internally. Users can get the workflow.xml from console or command line(-definition).
-The -D options passed after the -X will be placed into the generated workflow's <code><configuration> elements</code> (and make it
+The -D options passed after the -X will be placed into the generated workflow's `<configuration> elements` (and make it
to the configuration used by Hive); any other opts after -X will be passed as-is to the invoked Hive program.
Multiple -D and -P arguments can be specified.
The job will be created and run right away.
----++ Sqoop Operations
+## Sqoop Operations
----+++ Submitting a sqoop job through HTTP
+### Submitting a sqoop job through HTTP
Syntax:
-<verbatim>
+
+```
$ oozie sqoop [-Dkey=value] -command completeSqoopCommand -config OOZIE-CONFIG [-X [-Dkey=value opts for Launcher/Job configuration]]
-</verbatim>
+```
Example:
-<verbatim>
+
+```
$ oozie sqoop -oozie http://localhost:11000/oozie -Dfs.default.name=hdfs://localhost:8020 -command import --connect jdbc:mysql://localhost:3306/oozie --username oozie --password oozie --table WF_JOBS --target-dir '/user/${wf:user()}/${examplesRoot}/output-data/sqoop' -m 1 -config job.properties -X -Dmapred.job.queue.name=default
.
job: 14-20090525161322-oozie-joe-W
.
-</verbatim>
+```
Sqoop Freeform Example:
-<verbatim>
+
+```
$ oozie sqoop -oozie http://localhost:11000/oozie -command import --connect jdbc:mysql://localhost:3306/oozie --username oozie --password oozie --query "SELECT a.id FROM WF_JOBS a WHERE \$CONDITIONS" --target-dir '/user/${wf:user()}/${examplesRoot}/output-data/sqoop' -m 1 -config job.properties -X -Dmapred.job.queue.name=default
.
job: 14-20090525161321-oozie-joe-W
@@ -2074,13 +2169,13 @@
mapreduce.jobtracker.kerberos.principal=ccc
dfs.namenode.kerberos.principal=ddd
oozie.libpath=hdfs://localhost:8020/user/oozie/sqoop/lib/
-</verbatim>
+```
The parameters for the job must be provided in a Java Properties file (.properties). jobtracker, namenode,
libpath must be specified in this file. All jar files (including sqoop jar file) and all other files needed by the
sqoop job need to be uploaded onto HDFS under libpath beforehand. The workflow.xml will be created in Oozie server
internally. Users can get the workflow.xml from console or command line(-definition).
-The -D options passed after the -X will be placed into the generated workflow's <code><configuration> elements</code>
+The -D options passed after the -X will be placed into the generated workflow's `<configuration> elements`
(and make it to the configuration used by Sqoop); Multiple -D arguments can be specified.
The job will be created and run right away.
@@ -2090,15 +2185,16 @@
variable itself like the value of "--target-dir". All the "-D" arguments before "-X" that are overriding given property
must be placed before the "-command" argument.
----++ Info Operations
+## Info Operations
The Info sub-command provides a convenient place for Oozie to display misc information.
----+++ Getting a list of time zones
+### Getting a list of time zones
Example:
-<verbatim>
+
+```
$ oozie info -timezones
.
The format is "SHORT_NAME (ID)"
@@ -2114,30 +2210,31 @@
HAST (America/Atka)
HST (HST)
...
-</verbatim>
+```
-The <code>-timezones</code> option will print out a (long) list of all available time zones.
+The `-timezones` option will print out a (long) list of all available time zones.
-These IDs (the text in the parentheses) are what should be used for the <code>-timezone TIME_ZONE_ID</code> option in the =job=
-and =jobs= sub-commands
+These IDs (the text in the parentheses) are what should be used for the `-timezone TIME_ZONE_ID` option in the `job`
+and `jobs` sub-commands
----++ Map-reduce Operations
+## Map-reduce Operations
----+++ Submitting a map-reduce job
+### Submitting a map-reduce job
Example:
-<verbatim>
+
+```
$ oozie mapreduce -oozie http://localhost:11000/oozie -config job.properties
-</verbatim>
+```
The parameters must be in the Java Properties file (.properties). This file must be specified for a map-reduce job.
-The properties file must specify the =mapred.mapper.class=, =mapred.reducer.class=, =mapred.input.dir=, =mapred.output.dir=,
-=oozie.libpath=, =mapred.job.tracker=, and =fs.default.name= properties.
+The properties file must specify the `mapred.mapper.class`, `mapred.reducer.class`, `mapred.input.dir`, `mapred.output.dir`,
+`oozie.libpath`, `mapred.job.tracker`, and `fs.default.name` properties.
The map-reduce job will be created and submitted. All jar files and all other files needed by the mapreduce job need to be uploaded onto HDFS under libpath beforehand. The workflow.xml will be created in Oozie server internally. Users can get the workflow.xml from console or command line(-definition).
----++ Getting Oozie diagnostics bundles
+## Getting Oozie diagnostics bundles
A tool that collects a diagnostic bundle of information from Oozie. Collected information includes available Oozie ShareLibs;
effective configuration, system properties, environment variables, thread dump of the Oozie server; instrumentation logs;
@@ -2146,30 +2243,32 @@
Syntax:
-<verbatim>
+
+```
$ oozie-diag-bundle-collector.sh [-jobs <id ...>] [-maxchildactions <n>]
[-numbundles <n>] [-numcoordinators <n>] [-numworkflows <n>] -oozie
<url> -output <dir>
-</verbatim>
+```
where
- -jobs <id ...> Detailed information on the given job IDs will be
+ -jobs \<id ...\> Detailed information on the given job IDs will be
collected (default: none)
- -maxchildactions <n> Maximum number of Workflow or Coordinator actions
+ -maxchildactions \<n\> Maximum number of Workflow or Coordinator actions
that will be collected (default: 10)
- -numbundles <n> Detailed information on the last n Bundles will be
+ -numbundles \<n\> Detailed information on the last n Bundles will be
collected (default: 0)
- -numcoordinators <n> Detailed information on the last n Coordinators
+ -numcoordinators \<n\> Detailed information on the last n Coordinators
will be collected (default: 0)
- -numworkflows <n> Detailed information on the last n workflows will
+ -numworkflows \<n\> Detailed information on the last n workflows will
be collected (default: 0)
- -oozie <url> Required: Oozie URL (or specify with OOZIE_URL env
+ -oozie \<url\> Required: Oozie URL (or specify with OOZIE_URL env
var)
- -output <dir> Required: Directory to output the zip file
+ -output \<dir\> Required: Directory to output the zip file
Example:
-<verbatim>
+
+```
$ oozie-diag-bundle-collector.sh -jobs 0000001-170918144116149-oozie-test-W -oozie http://localhost:11000/oozie -output diag
...
@@ -2185,12 +2284,12 @@
Getting Metrics...Done
Getting Details for 0000001-170918144116149-oozie-test-W...Done
Creating Zip File: /var/lib/oozie/diag/oozie-diag-bundle-1505748797206.zip...Done
-</verbatim>
+```
Before executing the command, make sure OOZIE_HOME environment variable is set correctly. If Oozie authorization is enabled, then
the user must be an admin user in order to perform admin operations (for example getting a Thread dump of the Oozie server). If the
output directory does not exist, the tool will create it and store generated bundle there.
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_CoordinatorRerun.twiki b/docs/src/site/twiki/DG_CoordinatorRerun.twiki
index fbb1376..f535d16 100644
--- a/docs/src/site/twiki/DG_CoordinatorRerun.twiki
+++ b/docs/src/site/twiki/DG_CoordinatorRerun.twiki
@@ -1,12 +1,12 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Coordinator Rerun
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Coordinator Rerun
----++ Pre-Conditions
+<!-- 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.
@@ -16,14 +16,15 @@
* Coordinator Rerun will only use the original configs from first run.
* Coordinator Rerun will not re-read the coordinator.xml in hdfs.
----++ Rerun Arguments
+## Rerun Arguments
- <verbatim>
+
+```
$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.)
-</verbatim>
+```
* Either -action or -date should be given.
* If -action and -date both are given, an error will be thrown.
@@ -37,7 +38,7 @@
* 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 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.
@@ -47,6 +48,6 @@
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.
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_CustomActionExecutor.twiki b/docs/src/site/twiki/DG_CustomActionExecutor.twiki
index 7831484..5768b27 100644
--- a/docs/src/site/twiki/DG_CustomActionExecutor.twiki
+++ b/docs/src/site/twiki/DG_CustomActionExecutor.twiki
@@ -1,78 +1,83 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Custom Action Nodes
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Custom Action Nodes
----++ Introduction
-Oozie can be extended to support additional action types by writing a custom [[WorkflowFunctionalSpec#ActionNodes][Action Node]]. Action Nodes can be synchronous or asynchronous.
+<!-- 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. [[DG_CustomActionExecutor#Deploying_a_custom_Action_Executor][Deploying a Custom Action Executor]].
+## 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.
+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
+### 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
+**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()
+### 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)
+### 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)
-<code>check(...)</code> 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).
+### 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)
-<code>kill(...)</code> is called when there is an attempt to kill the running job or action. No workflow transition is made after this.
+### 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)
-<code>end(...)</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
+### 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
+## 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.
-<verbatim> <property>
+
+``` <property>
<name>oozie.service.ActionService.executor.ext.classes</name>
<value>
org.apache.oozie.wf.action.decision.CustomActionExecutor,
Custom_Action_Executr_2.class
</value>
- </property></verbatim>
+ </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
-[[DG_FluentJobAPI#AE.C_Appendix_C_How_To_Extend][Fluent Job API :: How To Extend]] for details.
+[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.
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_DistCpActionExtension.twiki b/docs/src/site/twiki/DG_DistCpActionExtension.twiki
index 8bab3da..13c2a0a 100644
--- a/docs/src/site/twiki/DG_DistCpActionExtension.twiki
+++ b/docs/src/site/twiki/DG_DistCpActionExtension.twiki
@@ -1,27 +1,28 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie DistCp Action Extension
+# Oozie DistCp Action Extension
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
----++ DistCp Action
+## DistCp Action
-The =DistCp= action uses Hadoop distributed copy to copy files from one cluster to another or within the same cluster.
+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
+**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
-[[DG_QuickStart#HadoopProxyUser][here]] on the Quick Start page.
+[here](DG_QuickStart.html#HadoopProxyUser) on the Quick Start page.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="distcp-example">
@@ -36,29 +37,31 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-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=.
+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
+**IMPORTANT:** If using the DistCp action between 2 secure clusters, the following property must be added to the `configuration` of
the action:
-<verbatim>
+
+```
<property>
<name>oozie.launcher.mapreduce.job.hdfs-servers</name>
<value>${nameNode1},${nameNode2}</value>
</property>
-</verbatim>
+```
-The =DistCp= action is also commonly used to copy files within the same cluster. Cases where copying files within
+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=
+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:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="copy-example">
@@ -73,14 +76,15 @@
</action>
...
</workflow-app>
-</verbatim>
+```
----++ Appendix, DistCp XML-Schema
+## Appendix, DistCp XML-Schema
----+++ AE.A Appendix A, DistCp XML-Schema
+### AE.A Appendix A, DistCp XML-Schema
----++++ DistCp Action Schema Version 1.0
-<verbatim>
+#### 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">
@@ -105,10 +109,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ DistCp Action Schema Version 0.2
-<verbatim>
+#### 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">
@@ -156,10 +161,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ DistCp Action Schema Version 0.1
-<verbatim>
+#### 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">
@@ -207,8 +213,8 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_EmailActionExtension.twiki b/docs/src/site/twiki/DG_EmailActionExtension.twiki
index 4de290c..1afcbb4 100644
--- a/docs/src/site/twiki/DG_EmailActionExtension.twiki
+++ b/docs/src/site/twiki/DG_EmailActionExtension.twiki
@@ -1,28 +1,29 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie Email Action Extension
+# Oozie Email Action Extension
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
-#EmailAction
----++++ 3.2.4 Email action
+<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=.
+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.
+All values specified in the `email` action can be parameterized (templatized) using EL expressions.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:0.1">
...
<action name="[NODE-NAME]">
@@ -40,37 +41,39 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-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
+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.
+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.
+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*
+**Configuration**
-The =email= action requires some SMTP server configuration to be present (in oozie-site.xml). The following are the values
+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.
+
+ * `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).
+ * `oozie.email.smtp.socket.timeout.ms` - The timeout to apply over all SMTP server socket operations (10000ms by default).
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:0.1">
...
<action name="an-email">
@@ -86,14 +89,15 @@
</action>
...
</workflow-app>
-</verbatim>
+```
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
+## AE.A Appendix A, Email XML-Schema
-<verbatim>
+
+```
<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">
@@ -112,11 +116,12 @@
</xs:sequence>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
-*GMail example to oozie-site.xml*
+**GMail example to oozie-site.xml**
-<verbatim>
+
+```
oozie.email.smtp.host=smtp.gmail.com
oozie.email.smtp.port=587
oozie.email.from.address=<some email address>
@@ -124,8 +129,8 @@
oozie.email.smtp.starttls.enable=true
oozie.email.smtp.username=<Gmail Id>
oozie.email.smtp.password=<Gmail Pass>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_Examples.twiki b/docs/src/site/twiki/DG_Examples.twiki
index 5323a17..ff33506 100644
--- a/docs/src/site/twiki/DG_Examples.twiki
+++ b/docs/src/site/twiki/DG_Examples.twiki
@@ -1,58 +1,61 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Oozie Examples
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Oozie Examples
----++ Command Line Examples
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
----+++ Setting Up the Examples
+## Command Line Examples
-Oozie examples are bundled within the Oozie distribution in the =oozie-examples.tar.gz= file.
+### Setting Up the Examples
-Expanding this file will create an =examples/= directory in the local file system.
+Oozie examples are bundled within the Oozie distribution in the `oozie-examples.tar.gz` file.
-The =examples/= directory must be copied to the user HOME directory in HDFS:
+Expanding this file will create an `examples/` directory in the local file system.
-<verbatim>
+The `examples/` directory must be copied to the user HOME directory in HDFS:
+
+
+```
$ hadoop fs -put examples examples
-</verbatim>
+```
-*NOTE:* If an examples directory already exists in HDFS, it must be deleted before copying it again. Otherwise files may not be
+**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
+### Running the Examples
-For the Streaming and Pig example, the [[DG_QuickStart#OozieShareLib][Oozie Share Library]] must be installed in HDFS.
+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.
+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
+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
+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 inputs for all examples are in the `examples/input-data/` directory.
-The examples create output under the =examples/output-data/${EXAMPLE_NAME}= 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.
+**Note**: The `job.properties` file needs to be a local file during submissions, and not a HDFS path.
-*How to run an example application:*
+**How to run an example application:**
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -config examples/apps/map-reduce/job.properties -run
.
job: 14-20090525161321-oozie-tucu
-</verbatim>
+```
Check the workflow job status:
-<verbatim>
+
+```
$ oozie job -oozie http://localhost:11000/oozie -info 14-20090525161321-oozie-tucu
.
.----------------------------------------------------------------------------------------------------------------------------------------------------------------
@@ -71,28 +74,30 @@
.----------------------------------------------------------------------------------------------------------------------------------------------------------------
mr-node map-reduce OK end job_200904281535_0254 SUCCEEDED - 2009-05-26 05:01 +0000 2009-05-26 05:01 +0000
.----------------------------------------------------------------------------------------------------------------------------------------------------------------
-</verbatim>
+```
-To check the workflow job status via the Oozie web console, with a browser go to =http://localhost:11000/oozie=.
+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
+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:
-<verbatim>
+
+```
$ export OOZIE_URL="http://localhost:11000/oozie"
$
$ oozie job -info 14-20090525161321-oozie-tucu
-</verbatim>
+```
----++ Java API Example
+## Java API Example
-Oozie provides a =[[./apidocs/org/org/apache/oozie/client/package-summary.html][Java Client API]] that simplifies
+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.
-<verbatim>
+
+```
import org.apache.oozie.client.OozieClient;
import org.apache.oozie.client.WorkflowJob;
.
@@ -127,20 +132,21 @@
System.out.println("Workflow job completed ...");
System.out.println(wf.getJobInfo(jobId));
...
-</verbatim>
+```
----++ Local Oozie Example
+## Local Oozie Example
-Oozie provides an embedded Oozie implementation, =[[./apidocs/org/apache/oozie/local/LocalOozie.html][LocalOozie]]=,
+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 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
+The examples bundled with Oozie include the complete and running class, `LocalOozieExample` from where this snippet was
taken.
-<verbatim>
+
+```
import org.apache.oozie.local.LocalOozie;
import org.apache.oozie.client.OozieClient;
import org.apache.oozie.client.WorkflowJob;
@@ -181,18 +187,18 @@
// stop local Oozie
LocalOozie.stop();
...
-</verbatim>
+```
-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.
+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
+## Fluent Job API Examples
-There are some elaborate examples how to use the [[DG_FluentJobAPI][Fluent Job API]], under =examples/fluentjob/=. There are two
-simple examples covered under [[DG_FluentJobAPI#A_Simple_Example][Fluent Job API :: A Simple Example]] and
-[[DG_FluentJobAPI#A_More_Verbose_Example][Fluent Job API :: A More Verbose Example]].
+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).
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_FluentJobAPI.twiki b/docs/src/site/twiki/DG_FluentJobAPI.twiki
index c8b764b..bd36517 100644
--- a/docs/src/site/twiki/DG_FluentJobAPI.twiki
+++ b/docs/src/site/twiki/DG_FluentJobAPI.twiki
@@ -1,27 +1,27 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Fluent Job API
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Fluent Job API
----++ Introduction
+<!-- 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
+### 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
+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
+ `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
@@ -40,7 +40,7 @@
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
+### 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
@@ -54,7 +54,7 @@
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
+### 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
@@ -71,7 +71,7 @@
Most of the non-goals may be targeted as enhancements of the Fluent Job API for future Oozie releases.
----+++ Approach
+### 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).
@@ -82,32 +82,33 @@
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
+## How To Use
----+++ A Simple Example
+### 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=.
+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=.
+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=.
+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, [[DG_CommandLineTool#Checking_a_workflow_definition_generated_by_a_Fluent_Job_API_jar_file][check via command line]] that
+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,
-[[DG_CommandLineTool#Running_a_workflow_definition_generated_by_a_Fluent_Job_API_jar_file][run via command line]] the Fluent Job API
+[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=:*
-<verbatim>
+**For reference, a simplistic API JAR example consisting of a `Workflow` having only one `ShellAction`:**
+
+```
public class MyFirstWorkflowFactory implements WorkflowFactory {
.
@Override
@@ -129,10 +130,11 @@
return shellWorkflow;
}
}
-</verbatim>
+```
-*After check, the generated workflow XML looks like this:*
-<verbatim>
+**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">
.
@@ -163,21 +165,22 @@
<workflow:end name="end"/>
.
</workflow:workflow-app>
-</verbatim>
+```
----+++ A More Verbose Example
+### A More Verbose Example
-*Error handling*
+**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
+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:*
-<verbatim>
+**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
@@ -218,10 +221,11 @@
return workflow;
}
}
-</verbatim>
+```
-*After check, the generated workflow XML looks like this:*
-<verbatim>
+**After check, the generated workflow XML looks like this:**
+
+```
<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
<workflow:workflow-app ... name="shell-example">
.
@@ -305,72 +309,76 @@
<workflow:end name="end"/>
.
</workflow:workflow-app>
-</verbatim>
+```
----+++ Runtime Limitations
+### 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 [[DG_FluentJobAPI#Non-goals][non-goals section]]. All such limitations are based on the current
+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 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=.
+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
+## Appendixes
----+++ AE.A Appendix A, API JAR format
+### 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=:
-<verbatim>
+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
-</verbatim>
+```
-*An example of the command line assembly of such an API JAR:*
-<verbatim>
+**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
-</verbatim>
+```
----+++ AE.B Appendix B, Some Useful Builder classes
+### 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
+For a complete list of `Builder` classes, please have a look at `oozie-fluent-job-api` artifact's following packages:
-On examples how to use these please see =oozie-examples= artifact's =org.apache.oozie.example.fluentjob= package.
+ * `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
----+++ AE.C Appendix C, How To Extend
+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=
+
+ * 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
+### 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
+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).
+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).
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_HCatalogIntegration.twiki b/docs/src/site/twiki/DG_HCatalogIntegration.twiki
index d3107b4..5c592e8 100644
--- a/docs/src/site/twiki/DG_HCatalogIntegration.twiki
+++ b/docs/src/site/twiki/DG_HCatalogIntegration.twiki
@@ -1,32 +1,32 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! HCatalog Integration (Since Oozie 4.x)
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# HCatalog Integration (Since Oozie 4.x)
----++ HCatalog Overview
- HCatalog is a table and storage management layer for Hadoop that enables users with different data processing
+<!-- 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 [[http://incubator.apache.org/hcatalog/docs/r0.5.0/index.html][HCatalog Documentation]] to know more about HCatalog.
+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
-[[http://incubator.apache.org/hcatalog/docs/r0.5.0/loadstore.html][HCatLoader and HCatStorer]].
+[HCatLoader and HCatStorer](http://incubator.apache.org/hcatalog/docs/r0.5.0/loadstore.html).
Working with HCatalog using MapReduce directly is detailed in
-[[http://incubator.apache.org/hcatalog/docs/r0.5.0/inputoutput.html][HCatInputFormat and HCatOutputFormat]].
+[HCatInputFormat and HCatOutputFormat](http://incubator.apache.org/hcatalog/docs/r0.5.0/inputoutput.html).
----+++ HCatalog notifications
+### 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 [[http://incubator.apache.org/hcatalog/docs/r0.5.0/notification.html][HCatalog Notification]] to know more about
+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 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
@@ -49,14 +49,14 @@
fallback polling is usually lower than the constant polling. Defaults are 10 minutes and 1 minute respectively.
----+++ Oozie Server Configuration
- Refer to [[AG_Install#HCatalog_Configuration][HCatalog Configuration]] section of [[AG_Install][Oozie Install]]
+### 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
+### 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.
+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:
@@ -67,14 +67,15 @@
hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value];...
For example,
-<verbatim>
+
+```
<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>
-</verbatim>
+```
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 (,).
@@ -84,59 +85,63 @@
hcat://[metastore_server]:[port],[metastore_server]:[port]/[database_name]/[table_name]/[partkey1]=[value];[partkey2]=[value];...
For example,
-<verbatim>
+
+```
<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>
-</verbatim>
+```
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=
+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
+`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
+`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
+`scheme://hostname3:3000,scheme://hostname4:4000,scheme://hostname5:5000/db/table/p1=12;p2=us`
-#HCatalogLibraries
----+++ HCatalog Libraries
+<a name="HCatalogLibraries"></a>
+### HCatalog Libraries
-A workflow action interacting with HCatalog requires the following jars in the classpath:
+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 [[WorkflowFunctionalSpec.html#a17_HDFS_Share_Libraries_for_Workflow_Applications_since_Oozie_2.3][Shared Libraries]] 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 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.
+ * 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
+### Coordinator
-Refer to [[CoordinatorFunctionalSpec][Coordinator Functional Specification]] for more information about
+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 [[WorkflowFunctionalSpec][Workflow Functional Specification]] for more information about
+### 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 [[DG_ActionAuthentication][Action Authentication]] for more information about
+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
+### 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.
-</noautolink>
\ No newline at end of file
diff --git a/docs/src/site/twiki/DG_Hive2ActionExtension.twiki b/docs/src/site/twiki/DG_Hive2ActionExtension.twiki
index efbe56d..d81ed02 100644
--- a/docs/src/site/twiki/DG_Hive2ActionExtension.twiki
+++ b/docs/src/site/twiki/DG_Hive2ActionExtension.twiki
@@ -1,41 +1,42 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie Hive 2 Action Extension
+# Oozie Hive 2 Action Extension
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
----++ Hive 2 Action
+## Hive 2 Action
-The =hive2= action runs Beeline to connect to Hive Server 2.
+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.
+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
+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.
+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
+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
-[WorkflowFunctionalSpec#FilesArchives][Adding Files and Archives for the Job]
+[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 is `${VARIABLES}`.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -75,44 +76,45 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-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 `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 `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
+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 `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
+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
+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
+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.
+`${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 `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 `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.
+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:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="my-hive2-action">
@@ -139,22 +141,21 @@
</action>
...
</workflow-app>
-</verbatim>
+```
----+++ Security
+### Security
-As mentioned above, =password= is only used in cases where Hive Server 2 is backed by something requiring a password (e.g. LDAP).
+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.
-See [[DG_UnifiedCredentialsModule][here]] for more information on the configuration for using the Hive Server 2 Action
-with a Kerberized Hive Server 2.
----++ Appendix, Hive 2 XML-Schema
+## Appendix, Hive 2 XML-Schema
----+++ AE.A Appendix A, Hive 2 XML-Schema
+### AE.A Appendix A, Hive 2 XML-Schema
----++++ Hive 2 Action Schema Version 1.0
-<verbatim>
+#### 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">
@@ -188,10 +189,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Hive 2 Action Schema Version 0.2
-<verbatim>
+#### 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">
@@ -248,10 +250,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Hive 2 Action Schema Version 0.1
-<verbatim>
+#### 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">
@@ -305,8 +308,8 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_HiveActionExtension.twiki b/docs/src/site/twiki/DG_HiveActionExtension.twiki
index aaa74fa..99a73c6 100644
--- a/docs/src/site/twiki/DG_HiveActionExtension.twiki
+++ b/docs/src/site/twiki/DG_HiveActionExtension.twiki
@@ -1,48 +1,49 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie Hive Action Extension
+# Oozie Hive Action Extension
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
-#HiveAction
----++ Hive Action
+<a name="HiveAction"></a>
+## Hive Action
-The =hive= action runs a Hive job.
+The `hive` action runs a Hive job.
The workflow job will wait until the Hive job completes before
continuing to the next action.
-To run the Hive job, you have to configure the =hive= action with the =resource-manager=, =name-node= and Hive =script=
-(or Hive =query=) elements as well as the necessary parameters and configuration.
+To run the Hive job, you have to configure the `hive` action with the `resource-manager`, `name-node` and Hive `script`
+(or Hive `query`) elements as well as the necessary parameters and configuration.
-A =hive= action can be configured to create or delete HDFS directories
+A `hive` action can be configured to create or delete HDFS directories
before starting the Hive job.
-Hive configuration can be specified with a file, using the =job-xml=
-element, and inline, using the =configuration= elements.
+Hive configuration can be specified with a file, using the `job-xml`
+element, and inline, using the `configuration` elements.
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.
+values specified in the `configuration` element override values specified
+in the `job-xml` file.
-Note that YARN =yarn.resourcemanager.address= (=resource-manager=) and HDFS =fs.default.name= (=name-node=) properties
+Note that YARN `yarn.resourcemanager.address` (`resource-manager`) and HDFS `fs.default.name` (`name-node`) properties
must not be present in the inline configuration.
-As with Hadoop =map-reduce= jobs, it is possible to add files and
+As with Hadoop `map-reduce` jobs, it is possible to add files and
archives in order to make them available to the Hive job. Refer to the
[WorkflowFunctionalSpec#FilesArchives][Adding Files and Archives for the Job]
section for more information about this feature.
Oozie Hive action supports Hive scripts with parameter variables, their
-syntax is =${VARIABLES}=.
+syntax is `${VARIABLES}`.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -77,37 +78,38 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-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 `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 the Hive job. As of schema 0.3, multiple =job-xml= elements are allowed in order to
-specify multiple =job.xml= files.
+The `job-xml` element, if present, specifies a file containing configuration
+for the Hive job. As of schema 0.3, multiple `job-xml` elements are allowed in order to
+specify multiple `job.xml` files.
-The =configuration= element, if present, contains configuration
+The `configuration` element, if present, contains configuration
properties that are passed to the Hive job.
-The =script= element must contain the path of the Hive script to
+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.
+`${VARIABLE}`. The values of these variables can then be specified
+using the `params` element.
-The =query= element available from uri:oozie:hive-action:0.6, 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 `query` element available from uri:oozie:hive-action:0.6, 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 `params` element, if present, contains parameters to be passed to
the Hive script.
All the above elements can be parameterized (templatized) using EL
expressions.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="myfirsthivejob">
@@ -132,14 +134,14 @@
</action>
...
</workflow-app>
-</verbatim>
+```
----+++ Hive Default and Site Configuration Files
+### Hive Default and Site Configuration Files
-Hive (as of Hive 0.8) ignores a =hive-default.xml= file. As a result, Oozie (as of Oozie 3.4) ignores the =oozie.hive.defaults=
+Hive (as of Hive 0.8) ignores a `hive-default.xml` file. As a result, Oozie (as of Oozie 3.4) ignores the `oozie.hive.defaults`
property that was previously required by earlier versions of Oozie for the Hive action.
----+++ Hive Action Logging
+### Hive Action Logging
Hive action logs are redirected to the Oozie Launcher map-reduce job task STDOUT/STDERR that runs Hive.
@@ -147,14 +149,15 @@
to navigate to the Oozie Launcher map-reduce job task logs via the Hadoop job-tracker web-console.
The logging level of the Hive action can set in the Hive action configuration using the
-property =oozie.hive.log.level=. The default value is =INFO=.
+property `oozie.hive.log.level`. The default value is `INFO`.
----++ Appendix, Hive XML-Schema
+## Appendix, Hive XML-Schema
----+++ AE.A Appendix A, Hive XML-Schema
+### AE.A Appendix A, Hive XML-Schema
----++++ Hive Action Schema Version 1.0
-<verbatim>
+#### Hive Action Schema Version 1.0
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:hive="uri:oozie:hive-action:1.0"
elementFormDefault="qualified"
@@ -187,10 +190,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Hive Action Schema Version 0.6
-<verbatim>
+#### Hive Action Schema Version 0.6
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:hive="uri:oozie:hive-action:0.6" elementFormDefault="qualified"
targetNamespace="uri:oozie:hive-action:0.6">
@@ -245,9 +249,10 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
----++++ Hive Action Schema Version 0.5
-<verbatim>
+```
+#### Hive Action Schema Version 0.5
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:hive="uri:oozie:hive-action:0.5" elementFormDefault="qualified"
targetNamespace="uri:oozie:hive-action:0.5">
@@ -299,10 +304,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Hive Action Schema Version 0.4
-<verbatim>
+#### Hive Action Schema Version 0.4
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:hive="uri:oozie:hive-action:0.4" elementFormDefault="qualified"
targetNamespace="uri:oozie:hive-action:0.4">
@@ -353,10 +359,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Hive Action Schema Version 0.3
-<verbatim>
+#### Hive Action Schema Version 0.3
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:hive="uri:oozie:hive-action:0.3" elementFormDefault="qualified"
targetNamespace="uri:oozie:hive-action:0.3">
@@ -407,10 +414,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Hive Action Schema Version 0.2
-<verbatim>
+#### Hive Action Schema Version 0.2
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:hive="uri:oozie:hive-action:0.2" elementFormDefault="qualified"
targetNamespace="uri:oozie:hive-action:0.2">
@@ -461,8 +469,8 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_JMSNotifications.twiki b/docs/src/site/twiki/DG_JMSNotifications.twiki
index a4b0f0d..e8f8a76 100644
--- a/docs/src/site/twiki/DG_JMSNotifications.twiki
+++ b/docs/src/site/twiki/DG_JMSNotifications.twiki
@@ -1,26 +1,26 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! JMS Notifications
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# JMS Notifications
----++ Overview
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Overview
Since Oozie 4.0, Oozie supports publishing notifications to a JMS Provider for job status changes and SLA met and miss
events. This provides an alternative to polling Oozie for Job or SLA related information and getting events as they
happen without any delay. Clients can be written to consume these notifications and integrate with different monitoring
and alerting systems.
----++ Oozie Server Configuration
-Refer to [[AG_Install#Notifications_Configuration][Notifications Configuration]] section of [[AG_Install][Oozie Install]]
+## Oozie Server Configuration
+Refer to [Notifications Configuration](AG_Install.html#Notifications_Configuration) section of [Oozie Install](AG_Install.html)
documentation for the Oozie server side configuration required to support publishing notifications to a JMS Provider.
The JNDI properties for the JMS provider, the topics to publish to and the notification types to publish (Job and/or SLA)
need to be configured.
----++ Consuming Notifications
+## Consuming Notifications
----+++ Notification types
+### Notification types
Job and SLA notifications are published to the configured JMS Provider on the configured topics.
Job status change notifications include job start, success, failure, suspended, etc. Currently only workflow job and
@@ -28,26 +28,28 @@
SLA notifications include START_MET, END_MET, DURATION_MET, START_MISS, END_MISS, DURATION_MISS events and are published
for a workflow job, workflow action or coordinator action for which SLA information is configured in the job xml. Refer
-to [[DG_SLAMonitoring#Configuring_SLA_in_Applications][SLA Configuration]] for information on configuring SLA for a workflow or
+to [SLA Configuration](DG_SLAMonitoring.html#Configuring_SLA_in_Applications) for information on configuring SLA for a workflow or
coordinator.
----+++ JMS Topic
+### JMS Topic
Consumers interested in notification on events will require to know the JNDI properties to connect to the JMS provider.
They will also need to know the JMS topic on which notifications for a particular job are published.
Oozie Client provides the following APIs :
-<verbatim>
+
+```
public JMSConnectionInfo getJMSConnectionInfo()
public String getJMSTopicName(String jobId)
-</verbatim>
+```
The JMSConnectionInfo exposes 3 methods:
-<verbatim>
+
+```
Properties getJNDIProperties();
String getTopicPattern(AppType appType);
String getTopicPrefix();
-</verbatim>
+```
The topic is obtained by concatenating topic prefix and the substituted value for topic pattern. The topic pattern
can be a constant value like workflow or coordinator which the administrator has configured or a variable (either ${username}
@@ -59,13 +61,14 @@
notifications for that job are published.
----+++ JMS Message Format
-JMS messages published are =javax.jms.TextMessage=. The body contains JSON and the header contains multiple properties
+### JMS Message Format
+JMS messages published are `javax.jms.TextMessage`. The body contains JSON and the header contains multiple properties
that can be used as selectors. The header properties are not repeated in the body of the message to keep the messages
small.
<b>Message Header:</b> <br/>
The different header properties are:
+
* msgType - Value can be JOB or SLA.
* user - The user who submitted the job.
* appName - Application name of the job.
@@ -86,77 +89,88 @@
<b>Message Body for Job Notifications:</b> <br/>
Sample JSON response for different job and sla events as below.
-<verbatim>
+
+```
Workflow Job in RUNNING state:
{"status":"RUNNING","id":"0000042-130618221729631-oozie-oozi-W","startTime":1342915200000}
-</verbatim>
+```
-<verbatim>
+
+```
Workflow Job in FAILED state:
{"status":"FAILED","errorCode":"EL_ERROR","errorMessage":"variable [dummyvalue] cannot be resolved",
"id":"0000042-130618221729631-oozie-oozi-W","startTime":1342915200000,"endTime":1366672183543}
-</verbatim>
+```
-<verbatim>
+
+```
Workflow Job in SUCCEEDED state:
{"status":"SUCCEEDED","id":"0000039-130618221729631-oozie-oozi-W","startTime":1342915200000,
"parentId":"0000025-130618221729631-oozie-oozi-C@1","endTime":1366676224154}
-</verbatim>
+```
-<verbatim>
+
+```
Workflow Job in SUSPENDED state:
{"status":"SUSPENDED","id":"0000039-130618221729631-oozie-oozi-W","startTime":1342915200000,
"parentId":"0000025-130618221729631-oozie-oozi-C@1"}
-</verbatim>
+```
-<verbatim>
+
+```
Coordinator Action in WAITING state:
{"status":"WAITING","nominalTime":1310342400000,"missingDependency":"hdfs://gsbl90107.blue.com:8020/user/john/dir1/file1",
"id":"0000025-130618221729631-oozie-oozi-C@1","startTime":1342915200000,"parentId":"0000025-130618221729631-oozie-oozi-C"}
-</verbatim>
+```
-<verbatim>
+
+```
Coordinator Action in RUNNING state:
{"status":"RUNNING","nominalTime":1310342400000,"id":"0000025-130618221729631-oozie-oozi-C@1",
"startTime":1342915200000,"parentId":"0000025-130618221729631-oozie-oozi-C"}
-</verbatim>
+```
-<verbatim>
+
+```
Coordinator Action in SUCCEEDED state:
{"status":"SUCCEEDED","nominalTime":1310342400000,"id":"0000025-130618221729631-oozie-oozi-C@1",
"startTime":1342915200000,"parentId":"0000025-130618221729631-oozie-oozi-C","endTime":1366677082799}
-</verbatim>
+```
-<verbatim>
+
+```
Coordinator Action in FAILED state:
{"status":"FAILED","errorCode":"E0101","errorMessage":"dummyError","nominalTime":1310342400000,
"id":"0000025-130618221729631-oozie-oozi-C@1","startTime":1342915200000,
"parentId":"0000025-130618221729631-oozie-oozi-C","endTime":1366677140818}
-</verbatim>
+```
<b>Message Body for SLA Notifications:</b> <br/>
-<verbatim>
+
+```
Workflow Job in sla END_MISS state:
{"id":"0000000-000000000000001-oozie-wrkf-C@1","parentId":"0000000-000000000000001-oozie-wrkf-C",
"expectedStartTime":1356998400000,"notificationMessage":"notification of start miss","actualStartTime":1357002000000,
"expectedDuration":-1, "actualDuration":3600,"expectedEndTime":1356998400000,"actualEndTime":1357002000000}
-</verbatim>
+```
----+++ JMS Client
+### JMS Client
Oozie provides a helper class JMSMessagingUtils for consumers to deserialize the JMS messages back to Java objects.
The below method getEventMessage() expects a sub type of EventMessage.
There are different implementations of EventMessage - WorkflowJobMessage, CoordinatorActionMessage and SLAMessage.
-<verbatim>
+
+```
<T extends EventMessage> T JMSMessagingUtils.getEventMessage(Message jmsMessage)
-</verbatim>
----++++ Example
+```
+#### Example
Below is a sample code to consume notifications.
First, create the Oozie client and retrieve the JNDI properties to make a connection to the JMS server.
-<verbatim>
+
+```
OozieClient oc = new OozieClient("http://localhost:11000/oozie");
JMSConnectionInfo jmsInfo = oc.getJMSConnectionInfo();
Properties jndiProperties = jmsInfo.getJNDIProperties();
@@ -181,13 +195,14 @@
MessageConsumer consumer = session.createConsumer(topic);
consumer.setMessageListener(this);
connection.start();
-</verbatim>
+```
-To start receiving messages, the JMS [[http://docs.oracle.com/javaee/6/api/javax/jms/MessageListener.html][MessageListener]]
+To start receiving messages, the JMS [MessageListener](http://docs.oracle.com/javaee/6/api/javax/jms/MessageListener.html)
interface needs to be implemented. Also, its onMessage() method needs to be implemented.
This method will be called whenever a message is available on the JMS bus.
-<verbatim>
+
+```
public void onMessage(Message message) {
if (message.getStringProperty(JMSHeaderConstants.MESSAGE_TYPE).equals(MessageType.SLA.name())){
SLAMessage slaMessage = JMSMessagingUtils.getEventMessage(message);
@@ -198,22 +213,24 @@
// Further processing
}
}
-</verbatim>
+```
----++++ Applying Selectors
+#### Applying Selectors
Below is a sample ActiveMQ text message header properties section.
-<verbatim>
+
+```
ActiveMQTextMessage
{properties = {appName = map-reduce-wf, msgType=JOB, appType=WORKFLOW_JOB, user=john, msgFormat=json, eventStatus=STARTED} ...}
-</verbatim>
+```
On the header properties, consumers can apply JMS selectors to filter messages from JMS provider.
-They are listed at [[../docs/client/apidocs/org/apache/oozie/client/event/jms/JMSHeaderConstants.html][JMSHeaderConstants]]
+They are listed at [JMSHeaderConstants](../docs/client/apidocs/org/apache/oozie/client/event/jms/JMSHeaderConstants.html)
Sample use of selector to filter events related to Job which have failed and has a particular app-name
-<verbatim>
+
+```
String selector=JMSHeaderConstants.EVENT_STATUS + "='FAILURE' AND " + JMSHeaderConstants.APP_NAME + "='app-name'";
MessageConsumer consumer = session.createConsumer(topic, selector);
-</verbatim>
+```
diff --git a/docs/src/site/twiki/DG_Overview.twiki b/docs/src/site/twiki/DG_Overview.twiki
index 3ec94a2..6a2b9d2 100644
--- a/docs/src/site/twiki/DG_Overview.twiki
+++ b/docs/src/site/twiki/DG_Overview.twiki
@@ -1,8 +1,8 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+ Oozie Workflow Overview
+[::Go back to Oozie Documentation Index::](index.html)
+
+# Oozie Workflow Overview
Oozie is a server based _Workflow Engine_ specialized in running workflow jobs with actions that run Hadoop Map/Reduce
and Pig jobs.
@@ -14,7 +14,7 @@
action can't run until the first action has completed.
Oozie workflows definitions are written in hPDL (a XML Process Definition Language similar to
-[[http://www.jboss.org/jbossjbpm/][JBOSS JBPM]] jPDL).
+[JBOSS JBPM](http://www.jboss.org/jbossjbpm/) jPDL).
Oozie workflow actions start jobs in remote systems (i.e. Hadoop, Pig). Upon action completion, the remote systems
callback Oozie to notify the action completion, at this point Oozie proceeds to the next action in the workflow.
@@ -25,26 +25,27 @@
Oozie workflows contain control flow nodes and action nodes.
-Control flow nodes define the beginning and the end of a workflow ( =start=, =end= and =fail= nodes) and provide a
-mechanism to control the workflow execution path ( =decision=, =fork= and =join= nodes).
+Control flow nodes define the beginning and the end of a workflow ( `start`, `end` and `fail` nodes) and provide a
+mechanism to control the workflow execution path ( `decision`, `fork` and `join` nodes).
Action nodes are the mechanism by which a workflow triggers the execution of a computation/processing task. Oozie
provides support for different types of actions: Hadoop map-reduce, Hadoop file system, Pig, SSH, HTTP, eMail and
Oozie sub-workflow. Oozie can be extended to support additional type of actions.
-Oozie workflows can be parameterized (using variables like =${inputDir}= within the workflow definition). When
+Oozie workflows can be parameterized (using variables like `${inputDir}` within the workflow definition). When
submitting a workflow job values for the parameters must be provided. If properly parameterized (i.e. using different
output directories) several identical workflow jobs can concurrently.
----++ WordCount Workflow Example
+## WordCount Workflow Example
-*Workflow Diagram:*
+**Workflow Diagram:**
-<img src="%ATTACHURLPATH%/DG_Overview.png"/>
+<img src="./DG_Overview.png"/>
-*hPDL Workflow Definition:*
+**hPDL Workflow Definition:**
-<verbatim>
+
+```
<workflow-app name='wordcount-wf' xmlns="uri:oozie:workflow:0.1">
<start to='wordcount'/>
<action name='wordcount'>
@@ -78,8 +79,8 @@
</kill/>
<end name='end'/>
</workflow-app>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_QuickStart.twiki b/docs/src/site/twiki/DG_QuickStart.twiki
deleted file mode 100644
index d6a0069..0000000
--- a/docs/src/site/twiki/DG_QuickStart.twiki
+++ /dev/null
@@ -1,230 +0,0 @@
-<noautolink>
-
-[[index][::Go back to Oozie Documentation Index::]]
-
----+!! Oozie Quick Start
-
-These instructions install and run Oozie using an embedded Jetty server and an embedded Derby database.
-
-For detailed install and configuration instructions refer to [[AG_Install][Oozie Install]].
-
-%TOC%
-
----++ Building Oozie
-
----+++ System Requirements:
- * Unix box (tested on Mac OS X and Linux)
- * Java JDK 1.8+
- * Maven 3.0.1+
- * Hadoop 2.6.0+
- * Pig 0.10.1+
-
-JDK commands (java, javac) must be in the command path.
-
-The Maven command (mvn) must be in the command path.
-
----+++ Building Oozie
-
-Download a source distribution of Oozie from the "Releases" drop down menu on the [[http://oozie.apache.org][Oozie site]].
-
-Expand the source distribution =tar.gz= and change directories into it.
-
-The simplest way to build Oozie is to run the =mkdistro.sh= script:
-<verbatim>
-$ bin/mkdistro.sh [-DskipTests]
-
-Running =mkdistro.sh= will create the binary distribution of Oozie. By default, oozie war will not contain hadoop and
-hcatalog libraries, however they are required for oozie to work. There are 2 options to add these libraries:
-
-1. At install time, copy the hadoop and hcatalog libraries to libext and run oozie-setup.sh to setup Oozie. This is
-suitable when same oozie package needs to be used in multiple set-ups with different hadoop/hcatalog versions.
-
-2. Build with -Puber which will bundle the required libraries in the oozie war. Further, the following options are
-available to customise the versions of the dependencies:
--Dhadoop.version=<version> - default 2.6.0
--Ptez - Bundle tez jars in hive and pig sharelibs. Useful if you want to use tez
-+as the execution engine for those applications.
--Dpig.version=<version> - default 0.16.0
--Dpig.classifier=<classifier> - default h2
--Dsqoop.version=<version> - default 1.4.3
--Dsqoop.classifier=<classifier> - default hadoop100
--Djetty.version=<version> - default 9.3.20.v20170531
--Dopenjpa.version=<version> - default 2.2.2
--Dxerces.version=<version> - default 2.10.0
--Dcurator.version=<version> - default 2.5.0
--Dhive.version=<version - default 1.2.0
--Dhbase.version=<version> - default 1.2.3
--Dtez.version=<version> - default 0.8.4
-
-*IMPORTANT:* Profile hadoop-3 must be activated if building against Hadoop 3
-</verbatim>
-
-More details on building Oozie can be found on the [[ENG_Building][Building Oozie]] page.
-
----++ Server Installation
-
----+++ System Requirements
-
- * Unix (tested in Linux and Mac OS X)
- * Java 1.8+
- * Hadoop
- * [[http://hadoop.apache.org][Apache Hadoop]] (tested with 1.2.1 & 2.6.0+)
- * ExtJS library (optional, to enable Oozie webconsole)
- * [[http://archive.cloudera.com/gplextras/misc/ext-2.2.zip][ExtJS 2.2]]
-
-The Java 1.8+ =bin= directory should be in the command path.
-
----+++ Server Installation
-
-*IMPORTANT:* Oozie ignores any set value for =OOZIE_HOME=, Oozie computes its home automatically.
-
- * Build an Oozie binary distribution
- * Download a Hadoop binary distribution
- * Download ExtJS library (it must be version 2.2)
-
-*NOTE:* The ExtJS library is not bundled with Oozie because it uses a different license.
-
-*NOTE:* Oozie UI browser compatibility Chrome (all), Firefox (3.5), Internet Explorer (8.0), Opera (10.5).
-
-*NOTE:* It is recommended to use a Oozie Unix user for the Oozie server.
-
-Expand the Oozie distribution =tar.gz=.
-
-Expand the Hadoop distribution =tar.gz= (as the Oozie Unix user).
-
-#HadoopProxyUser
-
-*NOTE:* Configure the Hadoop cluster with proxyuser for the Oozie process.
-
-The following two properties are required in Hadoop core-site.xml:
-
-<verbatim>
- <!-- OOZIE -->
- <property>
- <name>hadoop.proxyuser.[OOZIE_SERVER_USER].hosts</name>
- <value>[OOZIE_SERVER_HOSTNAME]</value>
- </property>
- <property>
- <name>hadoop.proxyuser.[OOZIE_SERVER_USER].groups</name>
- <value>[USER_GROUPS_THAT_ALLOW_IMPERSONATION]</value>
- </property>
-</verbatim>
-
-Replace the capital letter sections with specific values and then restart Hadoop.
-
-The ExtJS library is optional (only required for the Oozie web-console to work)
-
-*IMPORTANT:* all Oozie server scripts (=oozie-setup.sh=, =oozied.sh=, =oozie-start.sh=, =oozie-run.sh=
-and =oozie-stop.sh=) run only under the Unix user that owns the Oozie installation directory,
-if necessary use =sudo -u OOZIE_USER= when invoking the scripts.
-
-As of Oozie 3.3.2, use of =oozie-start.sh=, =oozie-run.sh=, and =oozie-stop.sh= has
-been deprecated and will print a warning. The =oozied.sh= script should be used
-instead; passing it =start=, =run=, or =stop= as an argument will perform the
-behaviors of =oozie-start.sh=, =oozie-run.sh=, and =oozie-stop.sh= respectively.
-
-Create a *libext/* directory in the directory where Oozie was expanded.
-
-If using the ExtJS library copy the ZIP file to the *libext/* directory. If hadoop and hcatalog libraries are not
-already included in the war, add the corresponding libraries to *libext/* directory.
-
-A "sharelib create -fs fs_default_name [-locallib sharelib]" command is available when running oozie-setup.sh
-for uploading new sharelib into hdfs where the first argument is the default fs name
-and the second argument is the Oozie sharelib to install, it can be a tarball or the expanded version of it.
-If the second argument is omitted, the Oozie sharelib tarball from the Oozie installation directory will be used.
-Upgrade command is deprecated, one should use create command to create new version of sharelib.
-Sharelib files are copied to new lib_<timestamped> directory. At start, server picks the sharelib from latest time-stamp directory.
-While starting server also purge sharelib directory which is older than sharelib retention days
-(defined as oozie.service.ShareLibService.temp.sharelib.retention.days and 7 days is default).
-
-db create|upgrade|postupgrade -run [-sqlfile <FILE>] command is for create, upgrade or postupgrade oozie db with an
-optional sql file
-
-Run the =oozie-setup.sh= script to configure Oozie with all the components added to the *libext/* directory.
-
-<verbatim>
-$ bin/oozie-setup.sh sharelib create -fs <FS_URI> [-locallib <PATH>]
- sharelib upgrade -fs <FS_URI> [-locallib <PATH>]
- db create|upgrade|postupgrade -run [-sqlfile <FILE>]
-</verbatim>
-
-*IMPORTANT*: If the Oozie server needs to establish secure connection with an external server with a self-signed certificate,
-make sure you specify the location of a truststore that contains required certificates. It can be done by configuring
-=oozie.https.truststore.file= in =oozie-site.xml=, or by setting the =javax.net.ssl.trustStore= system property.
-If it is set in both places, the value passed as system property will be used.
-
-Create the Oozie DB using the 'ooziedb.sh' command line tool:
-
-<verbatim>
-$ bin/ooziedb.sh create -sqlfile oozie.sql -run
-
-Validate DB Connection.
-DONE
-Check DB schema does not exist
-DONE
-Check OOZIE_SYS table does not exist
-DONE
-Create SQL schema
-DONE
-DONE
-Create OOZIE_SYS table
-DONE
-
-Oozie DB has been created for Oozie version '3.2.0'
-
-$
-</verbatim>
-
-Start Oozie as a daemon process run:
-
-<verbatim>
-$ bin/oozied.sh start
-</verbatim>
-
-To start Oozie as a foreground process run:
-
-<verbatim>
-$ bin/oozied.sh run
-</verbatim>
-
-Check the Oozie log file =logs/oozie.log= to ensure Oozie started properly.
-
-Using the Oozie command line tool check the status of Oozie:
-
-<verbatim>
-$ bin/oozie admin -oozie http://localhost:11000/oozie -status
-</verbatim>
-
-Using a browser go to the [[http://localhost:11000/oozie][Oozie web console]], Oozie status should be *NORMAL*.
-
-Refer to the [[DG_Examples][Running the Examples]] document for details on running the examples.
-
----++ Client Installation
-
----+++ System Requirements
-
- * Unix (tested in Linux and Mac OS X)
- * Java 1.8+
-
-The Java 1.8+ =bin= directory should be in the command path.
-
----+++ Client Installation
-
-Copy and expand the =oozie-client= TAR.GZ file bundled with the distribution. Add the =bin/= directory to the =PATH=.
-
-Refer to the [[DG_CommandLineTool][Command Line Interface Utilities]] document for a full reference of the =oozie=
-command line tool.
-
-NOTE: The Oozie server installation includes the Oozie client. The Oozie client should be installed in remote machines
-only.
-
-#OozieShareLib
----++ Oozie Share Lib Installation
-
-Oozie share lib has been installed by oozie-setup.sh create command explained in the earlier section.
-
-See the [[WorkflowFunctionalSpec#ShareLib][Workflow Functional Specification]] and [[AG_Install#Oozie_Share_Lib][Installation]] for more information about the Oozie ShareLib.
-
-[[index][::Go back to Oozie Documentation Index::]]
-
-</noautolink>
diff --git a/docs/src/site/twiki/DG_SLAMonitoring.twiki b/docs/src/site/twiki/DG_SLAMonitoring.twiki
index c91c227..0831b93 100644
--- a/docs/src/site/twiki/DG_SLAMonitoring.twiki
+++ b/docs/src/site/twiki/DG_SLAMonitoring.twiki
@@ -1,12 +1,12 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Oozie SLA Monitoring
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Oozie SLA Monitoring
----++ Overview
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## Overview
Critical jobs can have certain SLA requirements associated with them. This SLA can be in terms of time
i.e. a maximum allowed time limit associated with when the job should start, by when should it end,
@@ -23,27 +23,30 @@
of how their jobs fared against their SLAs.
----++ Oozie Server Configuration
+## Oozie Server Configuration
-Refer to [[AG_Install#Notifications_Configuration][Notifications Configuration]] for configuring Oozie server to track
+Refer to [Notifications Configuration](AG_Install.html#Notifications_Configuration) for configuring Oozie server to track
SLA for jobs and send notifications.
----++ SLA Tracking
+## SLA Tracking
Oozie allows tracking SLA for meeting the following criteria:
+
* Start time
* End time
* Job Duration
----++++ Event Status
+### Event Status
Corresponding to each of these 3 criteria, your jobs are processed for whether Met or Miss i.e.
+
* START_MET, START_MISS
* END_MET, END_MISS
* DURATION_MET, DURATION_MISS
----++++ SLA Status
+### SLA Status
Expected end-time is the most important criterion for majority of users while deciding overall SLA Met or Miss.
Hence the _"SLA_Status"_ for a job will transition through these four stages
+
* Not_Started <-- Job not yet begun
* In_Process <-- Job started and is running, and SLAs are being tracked
* Met <-- caused by an END_MET
@@ -52,19 +55,26 @@
In addition to overshooting expected end-time, and END_MISS (and so an eventual SLA MISS) also occurs when the
job does not end successfully e.g. goes to error state - Failed/Killed/Error/Timedout.
----++ Configuring SLA in Applications
+## Configuring SLA in Applications
-To make your jobs trackable for SLA, you simply need to add the =<sla:info>= tag to your workflow application definition.
-If you were already using the existing SLA schema in your workflows (Schema xmlns:sla="uri:oozie:sla:0.1"), you don't need to
+To make your jobs trackable for SLA, you simply need to add the `<sla:info>` tag to your workflow application definition.
+If you were already using the existing SLA schema
+in your workflows (Schema xmlns:sla="uri:oozie:sla:0.1"), you don't need to
do anything extra to receive SLA notifications via JMS messages. This new SLA monitoring framework is backward-compatible -
-no need to change application XML for now and you can continue to fetch old records via the [[DG_CommandLineTool#SLAOperations][command line API]].
-However, usage of old schema and API is deprecated and we strongly recommend using new schema.
- * New SLA schema is 'uri:oozie:sla:0.2'
- * In order to use new SLA schema, you will need to upgrade your workflow/coordinator schema to 0.5 i.e. 'uri:oozie:workflow:0.5'
+no need to change application XML for now and you can continue to fetch old records via the [command line API](DG_CommandLineTool.html#SLAOperations).
+However, usage of old schema
+and API is deprecated and we strongly recommend using new schema.
----+++ SLA Definition in Workflow
+ * New SLA schema
+is 'uri:oozie:sla:0.2'
+ * In order to use new SLA schema,
+ you will need to upgrade your workflow/coordinator schema
+to 0.5 i.e. 'uri:oozie:workflow:0.5'
+
+### SLA Definition in Workflow
Example:
-<verbatim>
+
+```
<workflow-app name="test-wf-job-sla"
xmlns="uri:oozie:workflow:0.5"
xmlns:sla="uri:oozie:sla:0.2">
@@ -97,25 +107,28 @@
<sla:alert-contact>joe@example.com</sla:alert-contact>
</sla:info>
</workflow-app>
-</verbatim>
+```
-For the list of tags usable under =<sla:info>=, refer to [[WorkflowFunctionalSpec#SLASchema][Schemas Appendix]].
-This new schema is much more compact and meaningful, getting rid of redundant and unused tags.
+For the list of tags usable under `<sla:info>`, refer to [Schemas Appendix](WorkflowFunctionalSpec.html#SLASchema).
+This new schema
+is much more compact and meaningful, getting rid of redundant and unused tags.
- * ==nominal-time==: As the name suggests, this is the time relative to which your jobs' SLAs will be calculated. Generally since Oozie workflows are aligned with synchronous data dependencies, this nominal time can be parameterized to be passed the value of your coordinator nominal time. Nominal time is also required in case of independent workflows and you can specify the time in which you expect the workflow to be run if you don't have a synchronous dataset associated with it.
- * ==should-start==: Relative to =nominal-time= this is the amount of time (along with time-unit - MINUTES, HOURS, DAYS) within which your job should *start running* to meet SLA. This is optional.
- * ==should-end==: Relative to =nominal-time= this is the amount of time (along with time-unit - MINUTES, HOURS, DAYS) within which your job should *finish* to meet SLA.
- * ==max-duration==: This is the maximum amount of time (along with time-unit - MINUTES, HOURS, DAYS) your job is expected to run. This is optional.
- * ==alert-events==: Specify the types of events for which *Email* alerts should be sent. Allowable values in this comma-separated list are start_miss, end_miss and duration_miss. *_met events can generally be deemed low priority and hence email alerting for these is not necessary. However, note that this setting is only for alerts via *email* alerts and not via JMS messages, where all events send out notifications, and user can filter them using desired selectors. This is optional and only applicable when alert-contact is configured.
- * ==alert-contact==: Specify a comma separated list of email addresses where you wish your alerts to be sent. This is optional and need not be configured if you just want to view your job SLA history in the UI and do not want to receive email alerts.
+ * `nominal-time`: As the name suggests, this is the time relative to which your jobs' SLAs will be calculated. Generally since Oozie workflows are aligned with synchronous data dependencies, this nominal time can be parameterized to be passed the value of your coordinator nominal time. Nominal time is also required in case of independent workflows and you can specify the time in which you expect the workflow to be run if you don't have a synchronous dataset associated with it.
+ * `should-start`: Relative to `nominal-time` this is the amount of time (along with time-unit - MINUTES, HOURS, DAYS) within which your job should *start running* to meet SLA. This is optional.
+ * `should-end`: Relative to `nominal-time` this is the amount of time (along with time-unit - MINUTES, HOURS, DAYS) within which your job should *finish* to meet SLA.
+ * `max-duration`: This is the maximum amount of time (along with time-unit - MINUTES, HOURS, DAYS) your job is expected to run. This is optional.
+ * `alert-events`: Specify the types of events for which **Email** alerts should be sent. Allowable values in this comma-separated list are start_miss, end_miss and duration_miss. *_met events can generally be deemed low priority and hence email alerting for these is not necessary. However, note that this setting is only for alerts via *email* alerts and not via JMS messages, where all events send out notifications, and user can filter them using desired selectors. This is optional and only applicable when alert-contact is configured.
+ * `alert-contact`: Specify a comma separated list of email addresses where you wish your alerts to be sent. This is optional and need not be configured if you just want to view your job SLA history in the UI and do not want to receive email alerts.
NOTE: All tags can be parameterized as a EL function or a fixed value.
-Same schema can be applied to and embedded under Workflow-Action as well as Coordinator-Action XML.
+Same schema
+can be applied to and embedded under Workflow-Action as well as Coordinator-Action XML.
----+++ SLA Definition in Workflow Action
+### SLA Definition in Workflow Action
-<verbatim>
+
+```
<workflow-app name="test-wf-action-sla" xmlns="uri:oozie:workflow:0.5" xmlns:sla="uri:oozie:sla:0.2">
<start to="grouper"/>
<action name="grouper">
@@ -130,10 +143,11 @@
</action>
<end name="end"/>
</workflow-app>
-</verbatim>
+```
----+++ SLA Definition in Coordinator Action
-<verbatim>
+### SLA Definition in Coordinator Action
+
+```
<coordinator-app name="test-coord-sla" frequency="${coord:days(1)}" freq_timeunit="DAY"
end_of_duration="NONE" start="2013-06-20T08:01Z" end="2013-12-01T08:01Z"
timezone="America/Los_Angeles" xmlns="uri:oozie:coordinator:0.4" xmlns:sla="uri:oozie:sla:0.2">
@@ -147,22 +161,24 @@
</sla:info>
</action>
</coordinator-app>
-</verbatim>
+```
----++ Accessing SLA Information
+## Accessing SLA Information
SLA information is accessible via the following ways:
+
* Through the SLA tab of the Oozie Web UI.
* JMS messages sent to a configured JMS provider for instantaneous tracking.
* RESTful API to query for SLA summary.
- * As an =Instrumentation.Counter= entry that is accessible via RESTful API and reflects to the number of all SLA tracked external
- entities. Name of this counter is =sla-calculator.sla-map=.
+ * As an `Instrumentation.Counter` entry that is accessible via RESTful API and reflects to the number of all SLA tracked external
+ entities. Name of this counter is `sla-calculator.sla-map`.
For JMS Notifications, you have to have a message broker in place, on which Oozie publishes messages and you can
hook on a subscriber to receive those messages. For more info on setting up and consuming JMS messages, refer
-[[DG_JMSNotifications][JMS Notifications]] documentation.
+[JMS Notifications](DG_JMSNotifications.html) documentation.
In the REST API, the following filters can be applied while fetching SLA information:
+
* app_name - Application name
* id - id of the workflow job, workflow action or coordinator action
* parent_id - Parent id of the workflow job, workflow action or coordinator action
@@ -179,14 +195,16 @@
The examples below demonstrate the use of REST API and explains the JSON response.
----+++ Scenario 1: Workflow Job Start_Miss
-*Request:*
-<verbatim>
-GET <oozie-host>:<port>/oozie/v2/sla?timezone=GMT&filter=nominal_start=2013-06-18T00:01Z;nominal_end=2013-06-23T00:01Z;app_name=my-sla-app
-</verbatim>
+### Scenario 1: Workflow Job Start_Miss
+**Request:**
-*JSON Response*
-<verbatim>
+```
+GET <oozie-host>:<port>/oozie/v2/sla?timezone=GMT&filter=nominal_start=2013-06-18T00:01Z;nominal_end=2013-06-23T00:01Z;app_name=my-sla-app
+```
+
+**JSON Response**
+
+```
{
id : "000056-1238791320234-oozie-joe-W"
@@ -208,16 +226,18 @@
upstreamApps: "dependent-app-1, dependent-app-2"
}
-</verbatim>
+```
----+++ Scenario 2: Workflow Action End_Miss
-*Request:*
-<verbatim>
+### Scenario 2: Workflow Action End_Miss
+**Request:**
+
+```
GET <oozie-host>:<port>/oozie/v2/sla?timezone=GMT&filter=parent_id=000056-1238791320234-oozie-joe-W
-</verbatim>
+```
-*JSON Response*
-<verbatim>
+**JSON Response**
+
+```
{
id : "000056-1238791320234-oozie-joe-W@map-reduce-action"
@@ -239,16 +259,18 @@
upstreamApps: "dependent-app-1, dependent-app-2"
}
-</verbatim>
+```
----+++ Scenario 3: Coordinator Action Duration_Miss
-*Request:*
-<verbatim>
+### Scenario 3: Coordinator Action Duration_Miss
+**Request:**
+
+```
GET <oozie-host>:<port>/oozie/v2/sla?timezone=GMT&filter=id=000001-1238791320234-oozie-joe-C
-</verbatim>
+```
-*JSON Response*
-<verbatim>
+**JSON Response**
+
+```
{
id : "000001-1238791320234-oozie-joe-C@2"
@@ -270,19 +292,21 @@
upstreamApps: "dependent-app-1, dependent-app-2"
}
-</verbatim>
+```
Scenario #3 is particularly interesting because it is an overall "MET" because it met its expected End-time,
but it is "Duration_Miss" because the actual run (between actual start and actual end) exceeded expected duration.
----+++ Scenario 4: All Coordinator actions in a Bundle
-*Request:*
-<verbatim>
-GET <oozie-host>:<port>/oozie/v2/sla?timezone=GMT&filter=bundle=1234567-150130225116604-oozie-B;event_status=END_MISS
-</verbatim>
+### Scenario 4: All Coordinator actions in a Bundle
+**Request:**
-*JSON Response*
-<verbatim>
+```
+GET <oozie-host>:<port>/oozie/v2/sla?timezone=GMT&filter=bundle=1234567-150130225116604-oozie-B;event_status=END_MISS
+```
+
+**JSON Response**
+
+```
{
id : "000001-1238791320234-oozie-joe-C@1"
parentId : "000001-1238791320234-oozie-joe-C"
@@ -323,13 +347,14 @@
actualDuration: 3360000 <-- (actual duration in milliseconds)
durationDelay: -4 <-- (duration delay in minutes)
}
-</verbatim>
+```
Scenario #4 (All Coordinator actions in a Bundle) is to get SLA information of all coordinator actions under bundle job in one call.
startDelay/durationDelay/endDelay values returned indicate how much delay compared to expected time (positive values in case of MISS, and negative values in case of MET).
----+++ Sample Email Alert
-<verbatim>
+### Sample Email Alert
+
+```
Subject: OOZIE - SLA END_MISS (AppName=wf-sla-job, JobID=0000004-130610225200680-oozie-oozi-W)
@@ -353,77 +378,82 @@
Expected End Time - Mon Jun 10 23:38:00 UTC 2013
Expected Duration (in mins) - 5
Actual Duration (in mins) - -1
-</verbatim>
+```
----+++ Changing job SLA definition and alerting
+### Changing job SLA definition and alerting
Following are ways to enable/disable SLA alerts for coordinator actions.
----++++ 1. Specify in Bundle XML during submission.
+#### 1. Specify in Bundle XML during submission.
Following properties can be specified in bundle xml as properties for coordinator.
-=oozie.sla.disable.alerts.older.than= this property can be specified in hours, the SLA notification for
+`oozie.sla.disable.alerts.older.than` this property can be specified in hours, the SLA notification for
coord actions will be disabled whose nominal is time older then this value. Default is 48 hours.
-<verbatim>
+
+```
<property>
<name>oozie.sla.disable.alerts.older.than</name>
<value>12</value>
</property>
-</verbatim>
+```
-=oozie.sla.disable.alerts= List of coord actions to be disabled. Value can be specified as list of coord actions or date range.
-<verbatim>
+`oozie.sla.disable.alerts` List of coord actions to be disabled. Value can be specified as list of coord actions or date range.
+
+```
<property>
<name>oozie.sla.disable.alerts</name>
<value>1,3-4,7-10</value>
</property>
-</verbatim>
+```
Will disable alert for coord actions 1,3,5,7,8,9,10
-=oozie.sla.enable.alerts= List of coord actions to be enabled. Value can be specified as list of coord actions or date range.
-<verbatim>
+`oozie.sla.enable.alerts` List of coord actions to be enabled. Value can be specified as list of coord actions or date range.
+
+```
<property>
<name>oozie.sla.enable.alerts</name>
<value>2009-01-01T01:00Z::2009-05-31T23:59Z</value>
</property>
-</verbatim>
+```
This will enable SLA alert for coord actions whose nominal time is in between (inclusive) 2009-01-01T01:00Z and 2009-05-31T23:59Z.
ALL keyword can be specified to specify all actions. Below property will disable SLA notifications for all coord actions.
-<verbatim>
+
+```
<property>
<name>oozie.sla.disable.alerts</name>
<value>ALL</value>
</property>
-</verbatim>
+```
----++++ 2. Specify during Coordinator job submission or update
+#### 2. Specify during Coordinator job submission or update
Above properties can be specified in job.properties in
-[[DG_CommandLineTool#Updating_coordinator_definition_and_properties][Coord job update command]],
-in [[DG_CommandLineTool#Submitting_a_Workflow_Coordinator_or_Bundle_Job][Coord job submit command]]
-or in [[DG_CommandLineTool#Running_a_Workflow_Coordinator_or_Bundle_Job][Coord job run command]]
+[Coord job update command](DG_CommandLineTool.html#Updating_coordinator_definition_and_properties),
+in [Coord job submit command](DG_CommandLineTool.html#Submitting_a_Workflow_Coordinator_or_Bundle_Job)
+or in [Coord job run command](DG_CommandLineTool.html#Running_a_Workflow_Coordinator_or_Bundle_Job)
----++++ 3. Change using command line
-Refer [[DG_CommandLineTool#Changing_job_SLA_definition_and_alerting][Changing job SLA definition and alerting]] for commandline usage.
+#### 3. Change using command line
+Refer [Changing job SLA definition and alerting](DG_CommandLineTool.html#Changing_job_SLA_definition_and_alerting) for commandline usage.
----++++ 4. Change using REST API
-Refer the REST API [[WebServicesAPI#Changing_job_SLA_definition_and_alerting][Changing job SLA definition and alerting]].
+#### 4. Change using REST API
+Refer the REST API [Changing job SLA definition and alerting](WebServicesAPI.html#Changing_job_SLA_definition_and_alerting).
----++ In-memory SLA entries and database content
+## In-memory SLA entries and database content
-There are special circumstances when the in-memory =SLACalcStatus= entries can exist without the workflow or coordinator job or
+There are special circumstances when the in-memory `SLACalcStatus` entries can exist without the workflow or coordinator job or
action instances in database. For example:
- * SLA tracked database content may already have been deleted, and =SLA_SUMMARY= entry is not present anymore in database
- * SLA tracked database content and =SLA_SUMMARY= entry aren't yet present in database
-By the time =SLAService= scheduled job will be running, SLA map contents are checked. When the =SLA_SUMMARY= entry for the in-memory
+ * SLA tracked database content may already have been deleted, and `SLA_SUMMARY` entry is not present anymore in database
+ * SLA tracked database content and `SLA_SUMMARY` entry aren't yet present in database
+
+By the time `SLAService` scheduled job will be running, SLA map contents are checked. When the `SLA_SUMMARY` entry for the in-memory
SLA entry is missing, a counter is increased. When this counter reaches the server-wide preconfigured value
-=oozie.sla.service.SLAService.maximum.retry.count= (by default =3=), in-memory SLA entry will get purged.
+`oozie.sla.service.SLAService.maximum.retry.count` (by default `3`), in-memory SLA entry will get purged.
----++ Known issues
+## Known issues
There are two known issues when you define SLA for a workflow action.
* If there are decision nodes and SLA is defined for a workflow action not in the execution path because of the decision node, you will still get an SLA_MISS notification.
* If you have dangling action nodes in your workflow definition and SLA is defined for it, you will still get an SLA_MISS notification.
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_ShellActionExtension.twiki b/docs/src/site/twiki/DG_ShellActionExtension.twiki
index 5894c28..eff4b08 100644
--- a/docs/src/site/twiki/DG_ShellActionExtension.twiki
+++ b/docs/src/site/twiki/DG_ShellActionExtension.twiki
@@ -1,39 +1,39 @@
-<<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie Shell Action Extension
+# Oozie Shell Action Extension
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
-#ShellAction
----++ Shell Action
+<a name="ShellAction"></a>
+## Shell Action
-The =shell= action runs a Shell command.
+The `shell` action runs a Shell command.
The workflow job will wait until the Shell command completes before
continuing to the next action.
-To run the Shell job, you have to configure the =shell= action with the
-=job-tracker=, =name-node= and Shell =exec= elements as
+To run the Shell job, you have to configure the `shell` action with the
+`job-tracker`, `name-node` and Shell `exec` elements as
well as the necessary arguments and configuration.
-A =shell= action can be configured to create or delete HDFS directories
+A `shell` action can be configured to create or delete HDFS directories
before starting the Shell job.
-Shell _launcher_ configuration can be specified with a file, using the =job-xml=
-element, and inline, using the =configuration= elements.
+Shell _launcher_ configuration can be specified with a file, using the `job-xml`
+element, and inline, using the `configuration` elements.
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.
+values specified in the `configuration` element override values specified
+in the `job-xml` file.
-Note that YARN =yarn.resourcemanager.address= (=resource-manager=) and HDFS =fs.default.name= (=name-node=) properties
+Note that YARN `yarn.resourcemanager.address` (`resource-manager`) and HDFS `fs.default.name` (`name-node`) properties
must not be present in the inline configuration.
-As with Hadoop =map-reduce= jobs, it is possible to add files and
+As with Hadoop `map-reduce` jobs, it is possible to add files and
archives in order to make them available to the Shell job. Refer to the
[WorkflowFunctionalSpec#FilesArchives][Adding Files and Archives for the Job]
section for more information about this feature.
@@ -45,9 +45,10 @@
* The format of the output must be a valid Java Properties file.
* The size of the output must not exceed 2KB.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -86,49 +87,50 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-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 `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 the Shell job. As of schema 0.2, multiple =job-xml= elements are allowed in order to
-specify multiple =job.xml= files.
+The `job-xml` element, if present, specifies a file containing configuration
+for the Shell job. As of schema 0.2, multiple `job-xml` elements are allowed in order to
+specify multiple `job.xml` files.
-The =configuration= element, if present, contains configuration
+The `configuration` element, if present, contains configuration
properties that are passed to the Shell job.
-The =exec= element must contain the path of the Shell command to
+The `exec` element must contain the path of the Shell command to
execute. The arguments of Shell command can then be specified
-using one or more =argument= element.
+using one or more `argument` element.
-The =argument= element, if present, contains argument to be passed to
+The `argument` element, if present, contains argument to be passed to
the Shell command.
-The =env-var= element, if present, contains the environment to be passed
-to the Shell command. =env-var= should contain only one pair of environment variable
+The `env-var` element, if present, contains the environment to be passed
+to the Shell command. `env-var` should contain only one pair of environment variable
and value. If the pair contains the variable such as $PATH, it should follow the
Unix convention such as PATH=$PATH:mypath. Don't use ${PATH} which will be
substituted by Oozie's EL evaluator.
-A =shell= action creates a Hadoop configuration. The Hadoop configuration is made available as a local file to the
+A `shell` action creates a Hadoop configuration. The Hadoop configuration is made available as a local file to the
Shell application in its running directory. The exact file path is exposed to the spawned shell using the environment
-variable called =OOZIE_ACTION_CONF_XML=.The Shell application can access the environment variable to read the action
+variable called `OOZIE_ACTION_CONF_XML`.The Shell application can access the environment variable to read the action
configuration XML file path.
-
-If the =capture-output= element is present, it indicates Oozie to capture output of the STDOUT of the shell command
+
+If the `capture-output` element is present, it indicates Oozie to capture output of the STDOUT of the shell command
execution. The Shell command output must be in Java Properties file format and it must not exceed 2KB. From within the
-workflow definition, the output of an Shell action node is accessible via the =String action:output(String node,
-String key)= function (Refer to section '4.2.6 Action EL Functions').
+workflow definition, the output of an Shell action node is accessible via the `String action:output(String node,
+String key)` function (Refer to section '4.2.6 Action EL Functions').
All the above elements can be parameterized (templatized) using EL
expressions.
-*Example:*
+**Example:**
How to run any shell script or perl script or CPP executable
-<verbatim>
+
+```
<workflow-app xmlns='uri:oozie:workflow:1.0' name='shell-wf'>
<start to='shell1' />
<action name='shell1'>
@@ -154,11 +156,12 @@
</kill>
<end name='end' />
</workflow-app>
-</verbatim>
+```
The corresponding job properties file used to submit Oozie job could be as follows:
-<verbatim>
+
+```
oozie.wf.application.path=hdfs://localhost:8020/user/kamrul/workflows/script
#Execute is expected to be in the Workflow directory.
@@ -173,11 +176,12 @@
nameNode=hdfs://localhost:8020
queueName=default
-</verbatim>
+```
How to run any java program bundles in a jar.
-<verbatim>
+
+```
<workflow-app xmlns='uri:oozie:workflow:1.0' name='shell-wf'>
<start to='shell1' />
<action name='shell1'>
@@ -204,11 +208,12 @@
</kill>
<end name='end' />
</workflow-app>
-</verbatim>
+```
The corresponding job properties file used to submit Oozie job could be as follows:
-<verbatim>
+
+```
oozie.wf.application.path=hdfs://localhost:8020/user/kamrul/workflows/script
#Hello.jar file is expected to be in the Workflow directory.
@@ -217,42 +222,45 @@
resourceManager=localhost:8032
nameNode=hdfs://localhost:8020
queueName=default
-</verbatim>
+```
----+++ Shell Action Configuration
+### Shell Action Configuration
-=oozie.action.shell.setup.hadoop.conf.dir= - Generates a config directory with various core/hdfs/yarn/mapred-site.xml files and points =HADOOP_CONF_DIR= and =YARN_CONF_DIR= env-vars to it, before the Script is invoked. XML is sourced from the action configuration. Useful when the Shell script passed uses various =hadoop= commands. Default is false.
-=oozie.action.shell.setup.hadoop.conf.dir.write.log4j.properties= - When =oozie.action.shell.setup.hadoop.conf.dir= is enabled, toggle if a log4j.properties file should also be written under the configuration files directory. Default is true.
-=oozie.action.shell.setup.hadoop.conf.dir.log4j.content= - When =oozie.action.shell.setup.hadoop.conf.dir.write.log4j.properties= is enabled, the content to write into the log4j.properties file under the configuration files directory. Default is a simple console based stderr logger, as presented below:
-<verbatim>
+ * `oozie.action.shell.setup.hadoop.conf.dir` - Generates a config directory with various core/hdfs/yarn/mapred-site.xml files and points `HADOOP_CONF_DIR` and `YARN_CONF_DIR` env-vars to it, before the Script is invoked. XML is sourced from the action configuration. Useful when the Shell script passed uses various `hadoop` commands. Default is false.
+ * `oozie.action.shell.setup.hadoop.conf.dir.write.log4j.properties` - When `oozie.action.shell.setup.hadoop.conf.dir` is enabled, toggle if a log4j.properties file should also be written under the configuration files directory. Default is true.
+ * `oozie.action.shell.setup.hadoop.conf.dir.log4j.content` - When `oozie.action.shell.setup.hadoop.conf.dir.write.log4j.properties` is enabled, the content to write into the log4j.properties file under the configuration files directory. Default is a simple console based stderr logger, as presented below:
+
+```
log4j.rootLogger=${hadoop.root.logger}
hadoop.root.logger=INFO,console
log4j.appender.console=org.apache.log4j.ConsoleAppender
log4j.appender.console.target=System.err
log4j.appender.console.layout=org.apache.log4j.PatternLayout
log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
-</verbatim>
+```
----+++ Shell Action Logging
+### Shell Action Logging
Shell action's stdout and stderr output are redirected to the Oozie Launcher map-reduce job task STDOUT that runs the shell command.
From Oozie web-console, from the Shell action pop up using the 'Console URL' link, it is possible
to navigate to the Oozie Launcher map-reduce job task logs via the Hadoop job-tracker web-console.
----+++ Shell Action Limitations
+### Shell Action Limitations
Although Shell action can execute any shell command, there are some limitations.
+
* No interactive command is supported.
* Command can't be executed as different user using sudo.
- * User has to explicitly upload the required 3rd party packages (such as jar, so lib, executable etc). Oozie provides a way using <file> and <archive> tag through Hadoop's Distributed Cache to upload.
+ * User has to explicitly upload the required 3rd party packages (such as jar, so lib, executable etc). Oozie provides a way using \<file\> and \<archive\> tag through Hadoop's Distributed Cache to upload.
* Since Oozie will execute the shell command into a Hadoop compute node, the default installation of utility in the compute node might not be fixed. However, the most common unix utilities are usually installed on all compute nodes. It is important to note that Oozie could only support the commands that are installed into the compute nodes or that are uploaded through Distributed Cache.
----++ Appendix, Shell XML-Schema
+## Appendix, Shell XML-Schema
----+++ AE.A Appendix A, Shell XML-Schema
+### AE.A Appendix A, Shell XML-Schema
----++++ Shell Action Schema Version 1.0
-<verbatim>
+#### Shell Action Schema Version 1.0
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:shell="uri:oozie:shell-action:1.0"
elementFormDefault="qualified"
@@ -283,10 +291,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Shell Action Schema Version 0.3
-<verbatim>
+#### Shell Action Schema Version 0.3
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:shell="uri:oozie:shell-action:0.3" elementFormDefault="qualified"
targetNamespace="uri:oozie:shell-action:0.3">
@@ -341,10 +350,11 @@
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Shell Action Schema Version 0.2
-<verbatim>
+#### Shell Action Schema Version 0.2
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:shell="uri:oozie:shell-action:0.2" elementFormDefault="qualified"
targetNamespace="uri:oozie:shell-action:0.2">
@@ -399,10 +409,11 @@
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Shell Action Schema Version 0.1
-<verbatim>
+#### Shell Action Schema Version 0.1
+
+```
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one
@@ -476,8 +487,8 @@
</xs:complexType>
</xs:schema>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_SparkActionExtension.twiki b/docs/src/site/twiki/DG_SparkActionExtension.twiki
index ce80e45..5a56cca 100644
--- a/docs/src/site/twiki/DG_SparkActionExtension.twiki
+++ b/docs/src/site/twiki/DG_SparkActionExtension.twiki
@@ -1,36 +1,37 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie Spark Action Extension
+# Oozie Spark Action Extension
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
----++ Spark Action
+## Spark Action
-The =spark= action runs a Spark job.
+The `spark` action runs a Spark job.
The workflow job will wait until the Spark job completes before
continuing to the next action.
-To run the Spark job, you have to configure the =spark= action with
-the =resource-manager=, =name-node=, Spark =master= elements as
+To run the Spark job, you have to configure the `spark` action with
+the `resource-manager`, `name-node`, Spark `master` elements as
well as the necessary elements, arguments and configuration.
-Spark options can be specified in an element called =spark-opts=.
+Spark options can be specified in an element called `spark-opts`.
-A =spark= action can be configured to create or delete HDFS directories
+A `spark` action can be configured to create or delete HDFS directories
before starting the Spark 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.
+values specified in the `configuration` element override values specified
+in the `job-xml` file.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -67,41 +68,43 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-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 `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 the Spark job. Multiple =job-xml= elements are allowed in order to
-specify multiple =job.xml= files.
+The `job-xml` element, if present, specifies a file containing configuration
+for the Spark job. Multiple `job-xml` elements are allowed in order to
+specify multiple `job.xml` files.
-The =configuration= element, if present, contains configuration
+The `configuration` element, if present, contains configuration
properties that are passed to the Spark job.
-The =master= element indicates the url of the Spark Master. Ex: spark://host:port, mesos://host:port, yarn-cluster, yarn-client,
+The `master` element indicates the url of the Spark Master. Ex: `spark://host:port`, `mesos://host:port`, yarn-cluster, yarn-client,
or local.
-The =mode= element if present indicates the mode of spark, where to run spark driver program. Ex: client,cluster. This is typically
-not required because you can specify it as part of =master= (i.e. master=yarn, mode=client is equivalent to master=yarn-client).
-A local =master= always runs in client mode.
+The `mode` element if present indicates the mode of spark, where to run spark driver program. Ex: client,cluster. This is typically
+not required because you can specify it as part of `master` (i.e. master`yarn, mode`client is equivalent to master=yarn-client).
+A local `master` always runs in client mode.
-Depending on the =master= (and =mode=) entered, the Spark job will run differently as follows:
+Depending on the `master` (and `mode`) entered, the Spark job will run differently as follows:
+
* local mode: everything runs here in the Launcher Job.
* yarn-client mode: the driver runs here in the Launcher Job and the executor in Yarn.
* yarn-cluster mode: the driver and executor run in Yarn.
-The =name= element indicates the name of the spark application.
+The `name` element indicates the name of the spark application.
-The =class= element if present, indicates the spark's application main class.
+The `class` element if present, indicates the spark's application main class.
-The =jar= element indicates a comma separated list of jars or python files.
+The `jar` element indicates a comma separated list of jars or python files.
-The =spark-opts= element, if present, contains a list of Spark options that can be passed to Spark. Spark configuration
+The `spark-opts` element, if present, contains a list of Spark options that can be passed to Spark. Spark configuration
options can be passed by specifying '--conf key=value' or other Spark CLI options.
Values containing whitespaces can be enclosed by double quotes.
-Some examples of the =spark-opts= element:
+Some examples of the `spark-opts` element:
+
* '--conf key=value'
* '--conf key1=value1 value2'
* '--conf key1="value1 value2"'
@@ -109,32 +112,35 @@
* '--conf key=value --verbose --properties-file user.properties'
There are several ways to define properties that will be passed to Spark. They are processed in the following order:
- * propagated from =oozie.service.SparkConfigurationService.spark.configurations=
- * read from a localized =spark-defaults.conf= file
- * read from a file defined in =spark-opts= via the =--properties-file=
- * properties defined in =spark-opts= element
+
+ * propagated from `oozie.service.SparkConfigurationService.spark.configurations`
+ * read from a localized `spark-defaults.conf` file
+ * read from a file defined in `spark-opts` via the `--properties-file`
+ * properties defined in `spark-opts` element
(The latter takes precedence over the former.)
-The server propagated properties, the =spark-defaults.conf= and the user-defined properties file are merged together into a
-single properties file as Spark handles only one file in its =--properties-file= option.
+The server propagated properties, the `spark-defaults.conf` and the user-defined properties file are merged together into a
+single properties file as Spark handles only one file in its `--properties-file` option.
-The =arg= element if present, contains arguments that can be passed to spark application.
+The `arg` element if present, contains arguments that can be passed to spark application.
-In case some property values are present both in =spark-defaults.conf= and as property key/value pairs generated by Oozie, the user
-configured values from =spark-defaults.conf= are prepended to the ones generated by Oozie, as part of the Spark arguments list.
+In case some property values are present both in `spark-defaults.conf` and as property key/value pairs generated by Oozie, the user
+configured values from `spark-defaults.conf` are prepended to the ones generated by Oozie, as part of the Spark arguments list.
Following properties to prepend to Spark arguments:
- * =spark.executor.extraClassPath=
- * =spark.driver.extraClassPath=
- * =spark.executor.extraJavaOptions=
- * =spark.driver.extraJavaOptions=
+
+ * `spark.executor.extraClassPath`
+ * `spark.driver.extraClassPath`
+ * `spark.executor.extraJavaOptions`
+ * `spark.driver.extraJavaOptions`
All the above elements can be parameterized (templatized) using EL
expressions.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="myfirstsparkjob">
@@ -165,34 +171,35 @@
</action>
...
</workflow-app>
-</verbatim>
+```
----+++ Spark Action Logging
+### Spark Action Logging
Spark action logs are redirected to the Oozie Launcher map-reduce job task STDOUT/STDERR that runs Spark.
From Oozie web-console, from the Spark action pop up using the 'Console URL' link, it is possible
to navigate to the Oozie Launcher map-reduce job task logs via the Hadoop job-tracker web-console.
----+++ Spark on YARN
+### Spark on YARN
To ensure that your Spark job shows up in the Spark History Server, make sure to specify these three Spark configuration properties
-either in =spark-opts= with =--conf= or from =oozie.service.SparkConfigurationService.spark.configurations= in oozie-site.xml.
+either in `spark-opts` with `--conf` or from `oozie.service.SparkConfigurationService.spark.configurations` in oozie-site.xml.
1. spark.yarn.historyServer.address=SPH-HOST:18088
-2. spark.eventLog.dir=hdfs://NN:8020/user/spark/applicationHistory
+2. spark.eventLog.dir=`hdfs://NN:8020/user/spark/applicationHistory`
3. spark.eventLog.enabled=true
----+++ PySpark with Spark Action
+### PySpark with Spark Action
To submit PySpark scripts with Spark Action, pyspark dependencies must be available in sharelib or in workflow's lib/ directory.
-For more information, please refer to [[AG_Install#Oozie_Share_Lib][installation document.]]
+For more information, please refer to [installation document.](AG_Install.html#Oozie_Share_Lib)
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
....
<action name="myfirstpysparkjob">
@@ -220,24 +227,24 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-The =jar= element indicates python file. Refer to the file by it's localized name, because only local files are allowed
-in PySpark. The py file should be in the lib/ folder next to the workflow.xml or added using the =file= element so that
+The `jar` element indicates python file. Refer to the file by it's localized name, because only local files are allowed
+in PySpark. The py file should be in the lib/ folder next to the workflow.xml or added using the `file` element so that
it's localized to the working directory with just its name.
----+++ Using Symlink in <jar>
+### Using Symlink in \<jar\>
-A symlink must be specified using =[[WorkflowFunctionalSpec#a3.2.2.
-1_Adding_Files_and_Archives_for_the_Job][file]]= element. Then, you can use
-the symlink name in =jar= element.
+A symlink must be specified using [file](WorkflowFunctionalSpec.html#a3.2.2.1_Adding_Files_and_Archives_for_the_Job) element. Then, you can use
+the symlink name in `jar` element.
-*Example:*
+**Example:**
Specifying relative path for symlink:
-Make sure that the file is within the application directory i.e. =oozie.wf.application.path= .
-<verbatim>
+Make sure that the file is within the application directory i.e. `oozie.wf.application.path` .
+
+```
<spark xmlns="uri:oozie:spark-action:1.0">
...
<jar>py-spark-example-symlink.py</jar>
@@ -246,10 +253,11 @@
<file>py-spark.py#py-spark-example-symlink.py</file>
...
</spark>
-</verbatim>
+```
Specifying full path for symlink:
-<verbatim>
+
+```
<spark xmlns="uri:oozie:spark-action:1.0">
...
<jar>spark-example-symlink.jar</jar>
@@ -258,16 +266,17 @@
<file>hdfs://localhost:8020/user/testjars/all-oozie-examples.jar#spark-example-symlink.jar</file>
...
</spark>
-</verbatim>
+```
----++ Appendix, Spark XML-Schema
+## Appendix, Spark XML-Schema
----+++ AE.A Appendix A, Spark XML-Schema
+### AE.A Appendix A, Spark XML-Schema
----++++ Spark Action Schema Version 1.0
-<verbatim>
+#### Spark Action Schema Version 1.0
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:spark="uri:oozie:spark-action:1.0" elementFormDefault="qualified"
targetNamespace="uri:oozie:spark-action:1.0">
@@ -300,10 +309,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Spark Action Schema Version 0.2
-<verbatim>
+#### Spark Action Schema Version 0.2
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:spark="uri:oozie:spark-action:0.2" elementFormDefault="qualified"
targetNamespace="uri:oozie:spark-action:0.2">
@@ -359,10 +369,11 @@
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Spark Action Schema Version 0.1
-<verbatim>
+#### Spark Action Schema Version 0.1
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:spark="uri:oozie:spark-action:0.1" elementFormDefault="qualified"
targetNamespace="uri:oozie:spark-action:0.1">
@@ -416,10 +427,10 @@
</xs:complexType>
</xs:schema>
-</verbatim>
-[[index][::Go back to Oozie Documentation Index::]]
+```
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_SqoopActionExtension.twiki b/docs/src/site/twiki/DG_SqoopActionExtension.twiki
index 0317d0c..b186c5a 100644
--- a/docs/src/site/twiki/DG_SqoopActionExtension.twiki
+++ b/docs/src/site/twiki/DG_SqoopActionExtension.twiki
@@ -1,46 +1,47 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie Sqoop Action Extension
+# Oozie Sqoop Action Extension
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
----++ Sqoop Action
+## Sqoop Action
-*IMPORTANT:* The Sqoop action requires Apache Hadoop 1.x or 2.x.
+**IMPORTANT:** The Sqoop action requires Apache Hadoop 1.x or 2.x.
-The =sqoop= action runs a Sqoop job.
+The `sqoop` action runs a Sqoop job.
The workflow job will wait until the Sqoop job completes before
continuing to the next action.
-To run the Sqoop job, you have to configure the =sqoop= action with the =resource-manager=, =name-node= and Sqoop =command=
-or =arg= elements as well as configuration.
+To run the Sqoop job, you have to configure the `sqoop` action with the `resource-manager`, `name-node` and Sqoop `command`
+or `arg` elements as well as configuration.
-A =sqoop= action can be configured to create or delete HDFS directories
+A `sqoop` action can be configured to create or delete HDFS directories
before starting the Sqoop job.
-Sqoop configuration can be specified with a file, using the =job-xml=
-element, and inline, using the =configuration= elements.
+Sqoop configuration can be specified with a file, using the `job-xml`
+element, and inline, using the `configuration` elements.
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.
+values specified in the `configuration` element override values specified
+in the `job-xml` file.
-Note that YARN =yarn.resourcemanager.address= / =resource-manager= and HDFS =fs.default.name= / =name-node= properties must not
+Note that YARN `yarn.resourcemanager.address` / `resource-manager` and HDFS `fs.default.name` / `name-node` properties must not
be present in the inline configuration.
-As with Hadoop =map-reduce= jobs, it is possible to add files and
+As with Hadoop `map-reduce` jobs, it is possible to add files and
archives in order to make them available to the Sqoop job. Refer to the
[WorkflowFunctionalSpec#FilesArchives][Adding Files and Archives for the Job]
section for more information about this feature.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -73,40 +74,41 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-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 `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 the Sqoop job. As of schema 0.3, multiple =job-xml= elements are allowed in order to
-specify multiple =job.xml= files.
+The `job-xml` element, if present, specifies a file containing configuration
+for the Sqoop job. As of schema 0.3, multiple `job-xml` elements are allowed in order to
+specify multiple `job.xml` files.
-The =configuration= element, if present, contains configuration
+The `configuration` element, if present, contains configuration
properties that are passed to the Sqoop job.
-*Sqoop command*
+**Sqoop command**
-The Sqoop command can be specified either using the =command= element or multiple =arg=
+The Sqoop command can be specified either using the `command` element or multiple `arg`
elements.
-When using the =command= element, Oozie will split the command on every space
+When using the `command` element, Oozie will split the command on every space
into multiple arguments.
-When using the =arg= elements, Oozie will pass each argument value as an argument to Sqoop.
+When using the `arg` elements, Oozie will pass each argument value as an argument to Sqoop.
-The =arg= variant should be used when there are spaces within a single argument.
+The `arg` variant should be used when there are spaces within a single argument.
Consult the Sqoop documentation for a complete list of valid Sqoop commands.
All the above elements can be parameterized (templatized) using EL
expressions.
-*Examples:*
+**Examples:**
-Using the =command= element:
+Using the `command` element:
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="myfirsthivejob">
@@ -129,11 +131,12 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-The same Sqoop action using =arg= elements:
+The same Sqoop action using `arg` elements:
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="myfirstsqoopjob">
@@ -164,20 +167,20 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-NOTE: The =arg= elements syntax, while more verbose, allows to have spaces in a single argument, something useful when
+NOTE: The `arg` elements syntax, while more verbose, allows to have spaces in a single argument, something useful when
using free from queries.
----+++ Sqoop Action Counters
+### Sqoop Action Counters
The counters of the map-reduce job run by the Sqoop action are available to be used in the workflow via the
-[[WorkflowFunctionalSpec#HadoopCountersEL][hadoop:counters() EL function]].
+[hadoop:counters() EL function](WorkflowFunctionalSpec.html#HadoopCountersEL).
-If the Sqoop action run an import all command, the =hadoop:counters()= EL will return the aggregated counters
+If the Sqoop action run an import all command, the `hadoop:counters()` EL will return the aggregated counters
of all map-reduce jobs run by the Sqoop import all command.
----+++ Sqoop Action Logging
+### Sqoop Action Logging
Sqoop action logs are redirected to the Oozie Launcher map-reduce job task STDOUT/STDERR that runs Sqoop.
@@ -185,14 +188,15 @@
to navigate to the Oozie Launcher map-reduce job task logs via the Hadoop job-tracker web-console.
The logging level of the Sqoop action can set in the Sqoop action configuration using the
-property =oozie.sqoop.log.level=. The default value is =INFO=.
+property `oozie.sqoop.log.level`. The default value is `INFO`.
----++ Appendix, Sqoop XML-Schema
+## Appendix, Sqoop XML-Schema
----+++ AE.A Appendix A, Sqoop XML-Schema
+### AE.A Appendix A, Sqoop XML-Schema
----++++ Sqoop Action Schema Version 1.0
-<verbatim>
+#### Sqoop Action Schema Version 1.0
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:sqoop="uri:oozie:sqoop-action:1.0"
elementFormDefault="qualified"
@@ -223,10 +227,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Sqoop Action Schema Version 0.3
-<verbatim>
+#### Sqoop Action Schema Version 0.3
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:sqoop="uri:oozie:sqoop-action:0.3" elementFormDefault="qualified"
targetNamespace="uri:oozie:sqoop-action:0.3">
@@ -279,10 +284,11 @@
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Sqoop Action Schema Version 0.2
-<verbatim>
+#### Sqoop Action Schema Version 0.2
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:sqoop="uri:oozie:sqoop-action:0.2" elementFormDefault="qualified"
targetNamespace="uri:oozie:sqoop-action:0.2">
@@ -335,8 +341,8 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_SshActionExtension.twiki b/docs/src/site/twiki/DG_SshActionExtension.twiki
index 5a51d49..e53e1c3 100644
--- a/docs/src/site/twiki/DG_SshActionExtension.twiki
+++ b/docs/src/site/twiki/DG_SshActionExtension.twiki
@@ -1,16 +1,16 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie Ssh Action Extension
+# Oozie Ssh Action Extension
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
----++ Ssh Action
+## Ssh Action
-The =ssh= action starts a shell command on a remote machine as a remote secure shell in background. The workflow job
+The `ssh` action starts a shell command on a remote machine as a remote secure shell in background. The workflow job
will wait until the remote shell command completes before continuing to the next action.
The shell command must be present in the remote machine and it must be available for execution via the command path.
@@ -32,9 +32,10 @@
The first retry will wait a configurable period of time ( 3 seconds by default) before check.
The following retries will wait 2 times of previous wait time.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -50,32 +51,33 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-The =host= indicates the user and host where the shell will be executed.
+The `host` indicates the user and host where the shell will be executed.
-*IMPORTANT:* The =oozie.action.ssh.allow.user.at.host= property, in the =oozie-site.xml= configuration, indicates if
+**IMPORTANT:** The `oozie.action.ssh.allow.user.at.host` property, in the `oozie-site.xml` configuration, indicates if
an alternate user than the one submitting the job can be used for the ssh invocation. By default this property is set
-to =true=.
+to `true`.
-The =command= element indicates the shell command to execute.
+The `command` element indicates the shell command to execute.
-The =args= element, if present, contains parameters to be passed to the shell command. If more than one =args= element
-is present they are concatenated in order. When an =args= element contains a space, even when quoted, it will be considered as
-separate arguments (i.e. "Hello World" becomes "Hello" and "World"). Starting with ssh schema 0.2, you can use the =arg= element
-(note that this is different than the =args= element) to specify arguments that have a space in them (i.e. "Hello World" is
-preserved as "Hello World"). You can use either =args= elements, =arg= elements, or neither; but not both in the same action.
+The `args` element, if present, contains parameters to be passed to the shell command. If more than one `args` element
+is present they are concatenated in order. When an `args` element contains a space, even when quoted, it will be considered as
+separate arguments (i.e. "Hello World" becomes "Hello" and "World"). Starting with ssh schema 0.2, you can use the `arg` element
+(note that this is different than the `args` element) to specify arguments that have a space in them (i.e. "Hello World" is
+preserved as "Hello World"). You can use either `args` elements, `arg` elements, or neither; but not both in the same action.
-If the =capture-output= element is present, it indicates Oozie to capture output of the STDOUT of the ssh command
+If the `capture-output` element is present, it indicates Oozie to capture output of the STDOUT of the ssh command
execution. The ssh command output must be in Java Properties file format and it must not exceed 2KB. From within the
workflow definition, the output of an ssh action node is accessible via the =String action:output(String node,
String key)= function (Refer to section '4.2.6 Action EL Functions').
-The configuration of the =ssh= action can be parameterized (templatized) using EL expressions.
+The configuration of the `ssh` action can be parameterized (templatized) using EL expressions.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="myssjob">
@@ -90,22 +92,23 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-In the above example, the =uploaddata= shell command is executed with two arguments, =jdbc:derby://foo.com:1527/myDB=
-and =hdfs://foobar.com:8020/usr/tucu/myData=.
+In the above example, the `uploaddata` shell command is executed with two arguments, `jdbc:derby://foo.com:1527/myDB`
+and `hdfs://foobar.com:8020/usr/tucu/myData`.
-The =uploaddata= shell must be available in the remote host and available in the command path.
+The `uploaddata` shell must be available in the remote host and available in the command path.
-The output of the command will be ignored because the =capture-output= element is not present.
+The output of the command will be ignored because the `capture-output` element is not present.
----++ Appendix, Ssh XML-Schema
+## Appendix, Ssh XML-Schema
----+++ AE.A Appendix A, Ssh XML-Schema
+### AE.A Appendix A, Ssh XML-Schema
----++++ Ssh Action Schema Version 0.2
+#### Ssh Action Schema Version 0.2
-<verbatim>
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:ssh="uri:oozie:ssh-action:0.2" elementFormDefault="qualified"
targetNamespace="uri:oozie:ssh-action:0.2">
@@ -127,11 +130,12 @@
<xs:complexType name="FLAG"/>
.
</xs:schema>
-</verbatim>
+```
----++++ Ssh Action Schema Version 0.1
+#### Ssh Action Schema Version 0.1
-<verbatim>
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:ssh="uri:oozie:ssh-action:0.1" elementFormDefault="qualified"
targetNamespace="uri:oozie:ssh-action:0.1">
@@ -150,8 +154,8 @@
<xs:complexType name="FLAG"/>
.
</xs:schema>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/DG_WorkflowReRun.twiki b/docs/src/site/twiki/DG_WorkflowReRun.twiki
index 88d982b..c128681 100644
--- a/docs/src/site/twiki/DG_WorkflowReRun.twiki
+++ b/docs/src/site/twiki/DG_WorkflowReRun.twiki
@@ -1,33 +1,34 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Workflow ReRrun
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
----++ Configs
+# Workflow ReRrun
+
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+## Configs
* oozie.wf.application.path
- * Only one of following two configurations is mandatory. Both should not be defined at the same time
+ * Only one of following two configurations is mandatory. Both should not be defined at the same time
* oozie.wf.rerun.skip.nodes
* oozie.wf.rerun.failnodes
* Skip nodes are comma separated list of action names. They can be any action nodes including decision node.
- * The valid value of =oozie.wf.rerun.failnodes= is true or false.
+ * The valid value of `oozie.wf.rerun.failnodes` is true or false.
* If secured hadoop version is used, the following two properties needs to be specified as well
* mapreduce.jobtracker.kerberos.principal
* dfs.namenode.kerberos.principal.
* Configurations can be passed as -D param.
-<verbatim>
-$ oozie job -oozie http://localhost:11000/oozie -rerun 14-20090525161321-oozie-joe -Doozie.wf.rerun.skip.nodes=<>
-</verbatim>
----++ Pre-Conditions
+```
+$ oozie job -oozie http://localhost:11000/oozie -rerun 14-20090525161321-oozie-joe -Doozie.wf.rerun.skip.nodes=<>
+```
+
+## Pre-Conditions
* Workflow with id wfId should exist.
* Workflow with id wfId should be in SUCCEEDED/KILLED/FAILED.
* If specified , nodes in the config oozie.wf.rerun.skip.nodes must be completed successfully.
----++ ReRun
+## ReRun
* Reloads the configs.
* If no configuration is passed, existing coordinator/workflow configuration will be used. If configuration is passed then, it will be merged with existing workflow configuration. Input configuration will take the precedence.
@@ -36,6 +37,6 @@
* Deletes the actions that are not skipped from the DB and copies data from old Workflow Instance to new one for skipped actions.
* Action handler will skip the nodes given in the config with the same exit transition as before.
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/ENG_Building.twiki b/docs/src/site/twiki/ENG_Building.twiki
deleted file mode 100644
index b861026..0000000
--- a/docs/src/site/twiki/ENG_Building.twiki
+++ /dev/null
@@ -1,270 +0,0 @@
-<noautolink>
-
-[[index][::Go back to Oozie Documentation Index::]]
-
----+!! Building Oozie
-
-%TOC%
-
----++ System Requirements
-
- * Unix box (tested on Mac OS X and Linux)
- * Java JDK 1.8+
- * [[http://maven.apache.org/][Maven 3.0.1+]]
- * [[http://hadoop.apache.org/core/releases.html][Hadoop 2.6.0+]]
- * [[http://hadoop.apache.org/pig/releases.html][Pig 0.10.1+]]
-
-JDK commands (java, javac) must be in the command path.
-
-The Maven command (mvn) must be in the command path.
-
----++ Oozie Documentation Generation
-
-To generate the documentation, Oozie uses a patched Doxia plugin for Maven with improved twiki support.
-
-The source of the modified plugin is available in the Oozie GitHub repository, in the =ydoxia= branch.
-
-To build and install it locally run the following command in the =ydoxia= branch:
-
-<verbatim>
-$ mvn install
-</verbatim>
-
-#SshSetup
----++ Passphrase-less SSH Setup
-
-*NOTE: SSH actions are deprecated in Oozie 2.*
-
-To run SSH Testcases and for easier Hadoop start/stop configure SSH to localhost to be passphrase-less.
-
-Create your SSH keys without a passphrase and add the public key to the authorized file:
-
-<verbatim>
-$ ssh-keygen -t dsa
-$ cat ~/.ssh/id_dsa.pub >> ~/.ssh/authorized_keys2
-</verbatim>
-
-Test that you can ssh without password:
-
-<verbatim>
-$ ssh localhost
-</verbatim>
-
----++ Building with different Java Versions
-
-Oozie requires a minimum Java version of 1.8. Any newer version can be used but by default bytecode will be generated
-which is compatible with 1.8. This can be changed by specifying the build property *targetJavaVersion*.
-
----++ Building and Testing Oozie
-
-The JARs for the specified Hadoop and Pig versions must be available in one of the Maven repositories defined in Oozie
-main 'pom.xml' file. Or they must be installed in the local Maven cache.
-
----+++ Examples Running Oozie Testcases with Different Configurations
-
-*Using embedded Hadoop minicluster with 'simple' authentication:*
-
-<verbatim>
-$ mvn clean test
-</verbatim>
-
-*Using a Hadoop cluster with 'simple' authentication:*
-
-<verbatim>
-$ mvn clean test -Doozie.test.hadoop.minicluster=false
-</verbatim>
-
-*Using embedded Hadoop minicluster with 'simple' authentication and Derby database:*
-
-<verbatim>
-$ mvn clean test -Doozie.test.hadoop.minicluster=false -Doozie.test.db=derby
-</verbatim>
-
-*Using a Hadoop cluster with 'kerberos' authentication:*
-
-<verbatim>
-$ mvn clean test -Doozie.test.hadoop.minicluster=false -Doozie.test.hadoop.security=kerberos
-</verbatim>
-
-NOTE: The embedded minicluster cannot be used when testing with 'kerberos' authentication.
-
-*Using a custom Oozie configuration for testcases:*
-
-<verbatim>
-$ mvn clean test -Doozie.test.config.file=/home/tucu/custom-oozie-sitel.xml
-</verbatim>
-
-*Running the testcases with different databases:*
-
-<verbatim>
-$ mvn clean test -Doozie.test.db=[hsqldb*|derby|mysql|postgres|oracle]
-</verbatim>
-
-Using =mysql= and =oracle= enables profiles that will include their JARs files in the build. If using
- =oracle=, the Oracle JDBC JAR file must be manually installed in the local Maven cache (the JAR is
-not available in public Maven repos).
-
----+++ Build Options Reference
-
-All these options can be set using *-D*.
-
-Except for the options marked with =(*)=, the options can be specified in the =test.properties= in the root
-of the Oozie project. The options marked with =(*)= are used in Maven POMs, thus they don't take effect if
-specified in the =test.properties= file (which is loaded by the =XTestCase= class at class initialization time).
-
-*hadoop.version* =(*)=: indicates the Hadoop version you wish to build Oozie against specifically. It will
-substitute this value in the Oozie POM properties and pull the corresponding Hadoop artifacts from Maven.
-The default version is 2.6.0 and that is the minimum supported Hadoop version.
-
-*generateSite* (*): generates Oozie documentation, default is undefined (no documentation is generated)
-
-*skipTests* (*): skips the execution of all testcases, no value required, default is undefined
-
-*test*= (*): runs a single test case, to run a test give the test class name without package and extension, no default
-
-*oozie.test.db*= (*): indicates the database to use for running the testcases, supported values are 'hsqldb', 'derby',
- 'mysql', 'postgres' and 'oracle'; default value is 'hsqldb'. For each database there is
- =core/src/test/resources/DATABASE-oozie-site.xml= file preconfigured.
-
-*oozie.test.properties* (*): indicates the file to load the test properties from, by default is =test.properties=.
-Having this option allows having different test properties sets, for example: minicluster, simple & kerberos.
-
-*oozie.test.waitfor.ratio*= : multiplication factor for testcases using waitfor, the ratio is used to adjust the
-effective time out. For slow machines the ratio should be increased. The default value is =1=.
-
-*oozie.test.config.file*= : indicates a custom Oozie configuration file for running the testcases. The specified file
-must be an absolute path. For example, it can be useful to specify different database than HSQL for running the
-testcases.
-
-*oozie.test.hadoop.minicluster*= : indicates if Hadoop minicluster should be started for testcases, default value 'true'
-
-*oozie.test.job.tracker*= : indicates the URI of the JobTracker when using a Hadoop cluster for testing, default value
-'localhost:8021'
-
-*oozie.test.name.node*= : indicates the URI of the NameNode when using a Hadoop cluster for testing, default value
-'hdfs://localhost:8020'
-
-*oozie.test.hadoop.security*= : indicates the type of Hadoop authentication for testing, valid values are 'simple' or
-'kerberos, default value 'simple'
-
-*oozie.test.kerberos.keytab.file*= : indicates the location of the keytab file, default value
-'${user.home}/oozie.keytab'
-
-*oozie.test.kerberos.realm*= : indicates the Kerberos real, default value 'LOCALHOST'
-
-*oozie.test.kerberos.oozie.principal*= : indicates the Kerberos principal for oozie, default value
-'${user.name}/localhost'
-
-*oozie.test.kerberos.jobtracker.principal*= : indicates the Kerberos principal for the JobTracker, default value
-'mapred/localhost'
-
-*oozie.test.kerberos.namenode.principal*= : indicates the Kerberos principal for the NameNode, default value
-'hdfs/localhost'
-
-*oozie.test.user.oozie*= : specifies the user ID used to start Oozie server in testcases, default value
-is =${user.name}=.
-
-*oozie.test.user.test*= : specifies primary user ID used as the user submitting jobs to Oozie Server in testcases,
-default value is =test=.
-
-*oozie.test.user.test2*= : specifies secondary user ID used as the user submitting jobs to Oozie Server in testcases,
-default value is =test2=.
-
-*oozie.test.user.test3*= : specifies secondary user ID used as the user submitting jobs to Oozie Server in testcases,
-default value is =test3=.
-
-*oozie.test.group*= : specifies group ID used as group when submitting jobs to Oozie Server in testcases,
-default value is =testg=.
-
-NOTE: The users/group specified in *oozie.test.user.test2*, *oozie.test.user.test3*= and *oozie.test.user.group*=
-are used for the authorization testcases only.
-
-*oozie.test.dir*= : specifies the directory where the =oozietests= directory will be created, default value is =/tmp=.
-The =oozietests= directory is used by testcases when they need a local filesystem directory.
-
-*hadoop.log.dir*= : specifies the directory where Hadoop minicluster will write its logs during testcases, default
-value is =/tmp=.
-
-*test.exclude*= : specifies a testcase class (just the class name) to exclude for the tests run, for example =TestSubmitCommand=.
-
-*test.exclude.pattern*= : specifies one or more patterns for testcases to exclude, for example =**/Test*Command.java=.
-
----+++ Testing Map Reduce Pipes Action
-
-Pipes testcases require Hadoop's *wordcount-simple* pipes binary example to run. The *wordcount-simple* pipes binary
-should be compiled for the build platform and copied into Oozie's *core/src/test/resources/* directory. The binary file
-must be named *wordcount-simple*.
-
-If the *wordcount-simple* pipes binary file is not available the testcase will do a NOP and it will print to its output
-file the following message 'SKIPPING TEST: TestPipesMain, binary 'wordcount-simple' not available in the classpath'.
-
-There are 2 testcases that use the *wordcount-simple* pipes binary, *TestPipesMain* and *TestMapReduceActionExecutor*,
-the 'SKIPPING TEST..." message would appear in the testcase log file of both testcases.
-
----+++ Testing using dist_test and grind
-
-Testing using [[https://github.com/cloudera/dist_test][dist_test]] framework with
-[[https://github.com/cloudera/dist_test/blob/master/docs/grind.md][grind]] front end might not work using the default 3.0.2
-version of the maven dependency plugin. It is necessary to downgrade to version 2.10 using
-<code>-Dmaven-dependency-plugin.version=2.10</code> .
-
-Maven flags for grind can be specified using <code>GRIND_MAVEN_FLAGS</code> environment variable:
-
-<verbatim>
-export GRIND_MAVEN_FLAGS=-Dmaven.dependency.plugin.version=2.10
-grind test --java-version 8
-</verbatim>
-
----++ Building an Oozie Distribution
-
-An Oozie distribution bundles an embedded Jetty server.
-
-The simplest way to build Oozie is to run the =mkdistro.sh= script:
-<verbatim>
-$ bin/mkdistro.sh [-DskipTests]
-Running =mkdistro.sh= will create the binary distribution of Oozie. The following options are available to customise
-the versions of the dependencies:
--Puber - Bundle required hadoop and hcatalog libraries in oozie war
--Dhadoop.version=<version> - default 2.6.0
--Ptez - Bundle tez jars in hive and pig sharelibs. Useful if you want to use tez
-as the execution engine for those applications.
--Dpig.version=<version> - default 0.16.0
--Dpig.classifier=<classifier> - default h2
--Dsqoop.version=<version> - default 1.4.3
--Dsqoop.classifier=<classifier> - default hadoop100
--jetty.version=<version> - default 9.3.20.v20170531
--Dopenjpa.version=<version> - default 2.2.2
--Dxerces.version=<version> - default 2.10.0
--Dcurator.version=<version> - default 2.5.0
--Dhive.version=<version> - default 1.2.0
--Dhbase.version=<version> - default 1.2.3
--Dtez.version=<version> - default 0.8.4
-</verbatim>
-
-*IMPORTANT:* Profile hadoop-3 must be activated if building against Hadoop 3
-
-The following properties should be specified when building a release:
-
- * -DgenerateDocs : forces the generation of Oozie documentation
- * -Dbuild.time= : timestamps the distribution
- * -Dvc.revision= : specifies the source control revision number of the distribution
- * -Dvc.url= : specifies the source control URL of the distribution
-
-The provided <code>bin/mkdistro.sh</code> script runs the above Maven invocation setting all these properties to the
-right values (the 'vc.*' properties are obtained from the local git repository).
-
----++ IDE Setup
-
-Eclipse and IntelliJ can use directly Oozie Maven project files.
-
-The only special consideration is that the following source directories from the =client= module must be added to
-the =core= module source path:
-
- * =client/src/main/java= : as source directory
- * =client/src/main/resources= : as source directory
- * =client/src/test/java= : as test-source directory
- * =client/src/test/resources= : as test-source directory
-
-[[index][::Go back to Oozie Documentation Index::]]
-
-</noautolink>
diff --git a/docs/src/site/twiki/ENG_Custom_Authentication.twiki b/docs/src/site/twiki/ENG_Custom_Authentication.twiki
deleted file mode 100644
index 3b8202d..0000000
--- a/docs/src/site/twiki/ENG_Custom_Authentication.twiki
+++ /dev/null
@@ -1,79 +0,0 @@
-<noautolink>
-
-[[index][::Go back to Oozie Documentation Index::]]
-
----+!! Creating Custom Authentication
-
-%TOC%
-
----++ Hadoop-Auth Authentication Interfaces and classes
-
-1. =org.apache.hadoop.security.authentication.client.Authenticator:= Interface for client authentication mechanisms.
-
-The following authenticators are provided in hadoop-auth:
-
- * KerberosAuthenticator : the authenticator implements the Kerberos SPNEGO authentication sequence.
- * PseudoAuthenticator : the authenticator implementation provides an authentication equivalent to Hadoop's Simple
- authentication, it trusts the value of the 'user.name' Java System property.
-
-2. =org.apache.hadoop.security.authentication.server.AuthenticationHandler:= Interface for server authentication mechanisms.
-
- * KerberosAuthenticationHandler : the authenticator handler implements the Kerberos SPNEGO authentication mechanism for HTTP.
- * PseudoAuthenticationHandler : the authenticator handler provides a pseudo authentication mechanism that accepts the user
- name specified as a query string parameter.
-
-3. =org.apache.hadoop.security.authentication.server.AuthenticationFilter:= A servlet filter enables protecting web application
-resources with different authentication mechanisms provided by AuthenticationHandler. To enable the filter, web application
-resources file (ex. web.xml) needs to include a filter class derived from =AuthenticationFilter=.
-
-For more information have a look at the appropriate
-[[https://hadoop.apache.org/docs/r2.7.2/hadoop-auth/index.html][Hadoop documentation]].
-
----++ Provide Custom Authentication to Oozie Client
-
-Apache Oozie contains a default class =org.apache.oozie.client.AuthOozieClient= to support Kerberos HTTP SPNEGO authentication,
-pseudo/simple authentication and anonymous access for client connections.
-
-To provide other authentication mechanisms, an Oozie client should extend from =AuthOozieClient= and provide the following
-methods should be overridden by derived classes to provide custom authentication:
-
- * getAuthenticator() : return corresponding Authenticator based on value specified by user at =auth= command option.
- * createConnection() : create a singleton class at Authenticator to allow client set and get key-value configuration for
- authentication.
-
----++ Provide Custom Authentication to Oozie Server
-
-To accept custom authentication in Oozie server, a filter extends from AuthenticationFilter must be provided. This filter
-delegates to the configured authentication handler for authentication and once it obtains an =AuthenticationToken= from it, sets
-a signed HTTP cookie with the token. If HTTP cookie is provided with different key name, its cookie value can be retrieved by
-overriding =getToken()= method. Please note, only when =getToken()= return NULL, a custom authentication can be invoked and
-processed in =AuthenticationFilter.doFilter()=.
-
-The following method explains how to read it and return NULL token.
-<verbatim>
-protected AuthenticationToken getToken(HttpServletRequest request) throws IOException, AuthenticationException {
- String tokenStr = null;
- Cookie[] cookies = request.getCookies();
-
- if (cookies != null) {
- for (Cookie cookie : cookies) {
- if (cookie.getName().equals(AuthenticatedURL.AUTH_COOKIE)) {
- tokenStr = cookie.getValue();
- LOG.info("Got 'hadoop.auth' cookie from request = " + tokenStr);
- if (tokenStr != null && !tokenStr.trim().isEmpty()) {
- AuthenticationToken retToken = super.getToken(request);
- return retToken;
- }
- } else if (cookie.getName().equals("NEWAUTH")) {
- tokenStr = cookie.getValue();
- // DO NOT return the token string so request can authenticated.
- }
- }
- }
- return null;
- }
-</verbatim>
-
-[[index][::Go back to Oozie Documentation Index::]]
-
-</noautolink>
diff --git a/docs/src/site/twiki/ENG_MiniOozie.twiki b/docs/src/site/twiki/ENG_MiniOozie.twiki
index 0b16289..e793676 100644
--- a/docs/src/site/twiki/ENG_MiniOozie.twiki
+++ b/docs/src/site/twiki/ENG_MiniOozie.twiki
@@ -1,43 +1,46 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
----+!! Running MiniOozie Tests
+[::Go back to Oozie Documentation Index::](index.html)
-%TOC%
+# Running MiniOozie Tests
----++ System Requirements
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
+
+## System Requirements
* Unix box (tested on Mac OS X and Linux)
* Java JDK 1.8+
* Eclipse (tested on 3.5 and 3.6)
- * [[http://maven.apache.org/][Maven 3.0.1+]]
+ * [Maven 3.0.1+](http://maven.apache.org/)
The Maven command (mvn) must be in the command path.
----++ Installing Oozie Jars To Maven Cache
+## Installing Oozie Jars To Maven Cache
Oozie source tree is at Apache SVN or Apache GIT. MiniOozie sample project is under Oozie source tree.
The following command downloads Oozie trunk to local:
-<verbatim>
+
+```
$ svn co https://svn.apache.org/repos/asf/incubator/oozie/trunk
-</verbatim>
+```
OR
-<verbatim>
+
+```
$ git clone git://github.com/apache/oozie.git
-</verbatim>
+```
To run MiniOozie tests, the required jars like oozie-core, oozie-client, oozie-core-tests need to be
available in remote maven repositories or local maven repository. The local maven cache for the above
jars can be created and installed using the command:
-<verbatim>
+
+```
$ mvn clean install -DskipTests -DtestJarSimple
-</verbatim>
+```
The following properties should be specified to install correct jars for MiniOozie:
@@ -47,33 +50,34 @@
MiniOozie is a folder named 'minitest' under Oozie source tree. Two sample tests are included in the project.
The following command to execute tests under MiniOozie:
-<verbatim>
+
+```
$ cd minitest
$ mvn clean test
-</verbatim>
+```
----++ Create Tests Using MiniOozie
+## Create Tests Using MiniOozie
MiniOozie is a JUnit test class to test Oozie applications such as workflow and coordinator. The test case
needs to extend from MiniOozieTestCase and does the same as the example class 'WorkflowTest.java' to create Oozie
workflow application properties and workflow XML. The example file is under Oozie source tree:
- * =minitest/src/test/java/org/apache/oozie/test/WorkflowTest.java=
+ * `minitest/src/test/java/org/apache/oozie/test/WorkflowTest.java`
----++ IDE Setup
+## IDE Setup
Eclipse and IntelliJ can use directly MiniOozie Maven project files. MiniOozie project can be imported to
Eclipse and IntelliJ as independent project.
The test directories under MiniOozie are:
- * =minitest/src/test/java= : as test-source directory
- * =minitest/src/test/resources= : as test-resource directory
+ * `minitest/src/test/java` : as test-source directory
+ * `minitest/src/test/resources` : as test-resource directory
-Also asynchronous actions like FS action can be used / tested using =LocalOozie= / =OozieClient= API.
-Please see =fs-decision.xml= workflow example.
+Also asynchronous actions like FS action can be used / tested using `LocalOozie` / `OozieClient` API.
+Please see `fs-decision.xml` workflow example.
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/WebServicesAPI.twiki b/docs/src/site/twiki/WebServicesAPI.twiki
index f9008a6..a303802 100644
--- a/docs/src/site/twiki/WebServicesAPI.twiki
+++ b/docs/src/site/twiki/WebServicesAPI.twiki
@@ -1,34 +1,34 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
----++ Oozie Web Services API, V1 (Workflow, Coordinator, And Bundle)
+## Oozie Web Services API, V1 (Workflow, Coordinator, And Bundle)
The Oozie Web Services API is a HTTP REST JSON API.
-All responses are in =UTF-8=.
+All responses are in `UTF-8`.
-Assuming Oozie is running at =OOZIE_URL=, the following web services end points are supported:
+Assuming Oozie is running at `OOZIE_URL`, the following web services end points are supported:
- * <OOZIE_URL>/versions
- * <OOZIE_URL>/v1/admin
- * <OOZIE_URL>/v1/job
- * <OOZIE_URL>/v1/jobs
- * <OOZIE_URL>/v2/job
- * <OOZIE_URL>/v2/jobs
- * <OOZIE_URL>/v2/admin
- * <OOZIE_URL>/v2/sla
+ * \<OOZIE_URL\>/versions
+ * \<OOZIE_URL\>/v1/admin
+ * \<OOZIE_URL\>/v1/job
+ * \<OOZIE_URL\>/v1/jobs
+ * \<OOZIE_URL\>/v2/job
+ * \<OOZIE_URL\>/v2/jobs
+ * \<OOZIE_URL\>/v2/admin
+ * \<OOZIE_URL\>/v2/sla
Documentation on the API is below; in some cases, looking at the corresponding command in the
-[[DG_CommandLineTool][Command Line Documentation]] page will provide additional details and examples. Most of the functionality
-offered by the Oozie CLI is using the WS API. If you export <code>OOZIE_DEBUG</code> then the Oozie CLI will output the WS API
+[Command Line Documentation](DG_CommandLineTool.html) page will provide additional details and examples. Most of the functionality
+offered by the Oozie CLI is using the WS API. If you export `OOZIE_DEBUG` then the Oozie CLI will output the WS API
details used by any commands you execute. This is useful for debugging purposes to or see how the Oozie CLI works with the WS API.
----+++ Versions End-Point
+### Versions End-Point
_Identical to the corresponding Oozie v0 WS API_
@@ -38,79 +38,87 @@
It returns the supported Oozie protocol versions by the server.
-Current returned values are =0, 1, 2=.
+Current returned values are `0, 1, 2`.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/versions
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
[0,1]
-</verbatim>
+```
----+++ Admin End-Point
+### Admin End-Point
This endpoint is for obtaining Oozie system status and configuration information.
-It supports the following sub-resources: =status, os-env, sys-props, configuration, instrumentation, systems, available-timezones=.
+It supports the following sub-resources: `status, os-env, sys-props, configuration, instrumentation, systems, available-timezones`.
----++++ System Status
+#### System Status
_Identical to the corresponding Oozie v0 WS API_
A HTTP GET request returns the system status.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/admin/status
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
{"systemMode":NORMAL}
-</verbatim>
+```
-With a HTTP PUT request it is possible to change the system status between =NORMAL=, =NOWEBSERVICE=, and =SAFEMODE=.
+With a HTTP PUT request it is possible to change the system status between `NORMAL`, `NOWEBSERVICE`, and `SAFEMODE`.
-*Request:*
+**Request:**
-<verbatim>
+
+```
PUT /oozie/v1/admin/status?systemmode=SAFEMODE
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
-</verbatim>
+```
----++++ OS Environment
+#### OS Environment
_Identical to the corresponding Oozie v0 WS API_
A HTTP GET request returns the Oozie system OS environment.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/admin/os-env
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -127,23 +135,25 @@
LANG: "en_US.UTF-8",
...
}
-</verbatim>
+```
----++++ Java System Properties
+#### Java System Properties
_Identical to the corresponding Oozie v0 WS API_
A HTTP GET request returns the Oozie Java system properties.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/admin/java-sys-properties
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -154,23 +164,25 @@
java.vm.info: "mixed mode",
...
}
-</verbatim>
+```
----++++ Oozie Configuration
+#### Oozie Configuration
_Identical to the corresponding Oozie v0 WS API_
A HTTP GET request returns the Oozie system configuration.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/admin/configuration
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -185,9 +197,9 @@
oozie.service.DBLiteWorkflowStoreService.oozie.autoinstall: "true",
...
}
-</verbatim>
+```
----++++ Oozie Instrumentation
+#### Oozie Instrumentation
_Identical to the corresponding Oozie v0 WS API_
@@ -196,17 +208,19 @@
A HTTP GET request returns the Oozie instrumentation information. Keep in mind that timers and counters that the Oozie server
hasn't incremented yet will not show up.
-*Note:* If Instrumentation is enabled, then Metrics is unavailable.
+**Note:** If Instrumentation is enabled, then Metrics is unavailable.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/admin/instrumentation
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -273,9 +287,9 @@
...
]
}
-</verbatim>
+```
----++++ Oozie Metrics
+#### Oozie Metrics
_Available in the Oozie v2 WS API and later_
@@ -283,19 +297,21 @@
hasn't incremented yet will not show up.
-*Note:* If Metrics is enabled, then Instrumentation is unavailable.
+**Note:** If Metrics is enabled, then Instrumentation is unavailable.
-*Note:* by default enabled since 5.0.0.
+**Note:** by default enabled since 5.0.0.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v2/admin/metrics
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -360,42 +376,46 @@
...
}
}
-</verbatim>
+```
----++++ Version
+#### Version
_Identical to the corresponding Oozie v0 WS API_
A HTTP GET request returns the Oozie build version.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/admin/build-version
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
{buildVersion: "3.0.0-SNAPSHOT" }
-</verbatim>
+```
----++++ Available Time Zones
+#### Available Time Zones
A HTTP GET request returns the available time zones.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/admin/available-timezones
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -436,33 +456,36 @@
...
]
}
-</verbatim>
+```
----++++ Queue Dump
+#### Queue Dump
A HTTP GET request returns the queue dump of the Oozie system. This is an administrator debugging feature.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/admin/queue-dump
-</verbatim>
+```
----++++ Available Oozie Servers
+#### Available Oozie Servers
A HTTP GET request returns the list of available Oozie Servers. This is useful when Oozie is configured
-for [[AG_Install#HA][High Availability]]; if not, it will simply return the one Oozie Server.
+for [High Availability](AG_Install.html#HA); if not, it will simply return the one Oozie Server.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v2/admin/available-oozie-servers
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -471,21 +494,23 @@
"hostB": "http://hostB:11000/oozie",
"hostC": "http://hostC:11000/oozie",
}
-</verbatim>
+```
----++++ List available sharelib
+#### List available sharelib
A HTTP GET request to get list of available sharelib.
If the name of the sharelib is passed as an argument (regex supported) then all corresponding files are also listed.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v2/admin/list_sharelib
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
{
@@ -500,17 +525,19 @@
"pig"
]
}
-</verbatim>
+```
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v2/admin/list_sharelib?lib=pig*
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
@@ -529,24 +556,26 @@
}
]
}
-</verbatim>
+```
----++++ Update system sharelib
+#### Update system sharelib
This webservice call makes the oozie server(s) to pick up the latest version of sharelib present
under oozie.service.WorkflowAppService.system.libpath directory based on the sharelib directory timestamp or reloads
the sharelib metafile if one is configured. The main purpose is to update the sharelib on the oozie server without restarting.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v2/admin/update_sharelib
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
[
@@ -573,59 +602,62 @@
}
}
]
-</verbatim>
+```
----++++ Purge Command
+#### Purge Command
Oozie admin purge command cleans up the Oozie Workflow/Coordinator/Bundle records based on the parameters.
The unit for parameters is day.
Purge command will delete the workflow records (wf=30) older than 30 days, coordinator records (coord=7) older than 7 days and
bundle records (bundle=7) older than 7 days. The limit (limit=10) defines, number of records to be fetch at a time. Turn
-(oldCoordAction=true/false) =on/off= coordinator action record purging for long running coordinators. If any of the parameter is
-not provided, then it will be taken from the =oozie-default/oozie-site= configuration.
+(oldCoordAction`true/false) `on/off= coordinator action record purging for long running coordinators. If any of the parameter is
+not provided, then it will be taken from the `oozie-default/oozie-site` configuration.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v2/admin/purge?wf=30&coord=7&bundle=7&limit=10&oldCoordAction=true
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
{
"purge": "Purge command executed successfully"
}
-</verbatim>
+```
----+++ Job and Jobs End-Points
+### Job and Jobs End-Points
_Modified in Oozie v1 WS API_
These endpoints are for submitting, managing and retrieving information of workflow, coordinator, and bundle jobs.
----++++ Job Submission
+#### Job Submission
----++++ Standard Job Submission
+#### Standard Job Submission
An HTTP POST request with an XML configuration as payload creates a job.
The type of job is determined by the presence of one of the following 3 properties:
- * =oozie.wf.application.path= : path to a workflow application directory, creates a workflow job
- * =oozie.coord.application.path= : path to a coordinator application file, creates a coordinator job
- * =oozie.bundle.application.path= : path to a bundle application file, creates a bundle job
-
+ * `oozie.wf.application.path` : path to a workflow application directory, creates a workflow job
+ * `oozie.coord.application.path` : path to a coordinator application file, creates a coordinator job
+ * `oozie.bundle.application.path` : path to a bundle application file, creates a bundle job
+
Or, if none of those are present, the jobtype parameter determines the type of job to run. It can either be mapreduce or pig.
-*Request:*
+**Request:**
-<verbatim>
+
+```
POST /oozie/v1/jobs
Content-Type: application/xml;charset=UTF-8
.
@@ -641,47 +673,50 @@
</property>
...
</configuration>
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 201 CREATED
Content-Type: application/json;charset=UTF-8
.
{
id: "job-3"
}
-</verbatim>
+```
-A created job will be in =PREP= status. If the query string parameter 'action=start' is provided in
-the POST URL, the job will be started immediately and its status will be =RUNNING=.
+A created job will be in `PREP` status. If the query string parameter 'action=start' is provided in
+the POST URL, the job will be started immediately and its status will be `RUNNING`.
-Coordinator jobs with start time in the future they will not create any action until the start time
+Coordinator jobs with start time in the future they will not create any action until the start time
happens.
-A coordinator job will remain in =PREP= status until it's triggered, in which case it will change to =RUNNING= status.
+A coordinator job will remain in `PREP` status until it's triggered, in which case it will change to `RUNNING` status.
The 'action=start' parameter is not valid for coordinator jobs.
----++++ Proxy MapReduce Job Submission
+#### Proxy MapReduce Job Submission
You can submit a Workflow that contains a single MapReduce action without writing a workflow.xml. Any required Jars or other files
must already exist in HDFS.
The following properties are required; any additional parameters needed by the MapReduce job can also be specified here:
- * =fs.default.name=: The NameNode
- * =mapred.job.tracker=: The JobTracker
- * =mapred.mapper.class=: The map-task classname
- * =mapred.reducer.class=: The reducer-task classname
- * =mapred.input.dir=: The map-task input directory
- * =mapred.output.dir=: The reduce-task output directory
- * =user.name=: The username of the user submitting the job
- * =oozie.libpath=: A directory in HDFS that contains necessary Jars for your job
- * =oozie.proxysubmission=: Must be set to =true=
-*Request:*
+ * `fs.default.name`: The NameNode
+ * `mapred.job.tracker`: The JobTracker
+ * `mapred.mapper.class`: The map-task classname
+ * `mapred.reducer.class`: The reducer-task classname
+ * `mapred.input.dir`: The map-task input directory
+ * `mapred.output.dir`: The reduce-task output directory
+ * `user.name`: The username of the user submitting the job
+ * `oozie.libpath`: A directory in HDFS that contains necessary Jars for your job
+ * `oozie.proxysubmission`: Must be set to `true`
-<verbatim>
+**Request:**
+
+
+```
POST /oozie/v1/jobs?jobtype=mapreduce
Content-Type: application/xml;charset=UTF-8
.
@@ -724,52 +759,58 @@
<value>true</value>
</property>
</configuration>
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 201 CREATED
Content-Type: application/json;charset=UTF-8
.
{
id: "job-3"
}
-</verbatim>
+```
----++++ Proxy Pig Job Submission
+#### Proxy Pig Job Submission
You can submit a Workflow that contains a single Pig action without writing a workflow.xml. Any required Jars or other files must
already exist in HDFS.
The following properties are required:
- * =fs.default.name=: The NameNode
- * =mapred.job.tracker=: The JobTracker
- * =user.name=: The username of the user submitting the job
- * =oozie.pig.script=: Contains the pig script you want to run (the actual script, not a file path)
- * =oozie.libpath=: A directory in HDFS that contains necessary Jars for your job
- * =oozie.proxysubmission=: Must be set to =true=
+
+ * `fs.default.name`: The NameNode
+ * `mapred.job.tracker`: The JobTracker
+ * `user.name`: The username of the user submitting the job
+ * `oozie.pig.script`: Contains the pig script you want to run (the actual script, not a file path)
+ * `oozie.libpath`: A directory in HDFS that contains necessary Jars for your job
+ * `oozie.proxysubmission`: Must be set to `true`
The following properties are optional:
- * =oozie.pig.script.params.size=: The number of parameters you'll be passing to Pig
- required =oozie.pig.script.params.n=: A parameter (variable definition for the script) in 'key=value' format, the 'n' should be an integer starting with 0 to indicate the parameter number
- * =oozie.pig.options.size=: The number of options you'll be passing to Pig
- * =oozie.pig.options.n=: An argument to pass to Pig, the 'n' should be an integer starting with 0 to indicate the option number
-The =oozie.pig.options.n= parameters are sent directly to Pig without any modification unless they start with =-D=, in which case
-they are put into the <code><configuration></code> element of the action.
+ * `oozie.pig.script.params.size`: The number of parameters you'll be passing to Pig
+ required
+ * `oozie.pig.script.params.n`: A parameter (variable definition for the script) in 'key=value' format, the 'n' should be an integer starting with 0 to indicate the parameter number
+ * `oozie.pig.options.size`: The number of options you'll be passing to Pig
+ * `oozie.pig.options.n`: An argument to pass to Pig, the 'n' should be an integer starting with 0 to indicate the option number
-In addition to passing parameters to Pig with =oozie.pig.script.params.n=, you can also create a properties file on HDFS and
-reference it with the =-param_file= option in =oozie.pig.script.options.n=; both are shown in the following example.
+The `oozie.pig.options.n` parameters are sent directly to Pig without any modification unless they start with `-D`, in which case
+they are put into the `<configuration>` element of the action.
-<verbatim>
+In addition to passing parameters to Pig with `oozie.pig.script.params.n`, you can also create a properties file on HDFS and
+reference it with the `-param_file` option in `oozie.pig.script.options.n`; both are shown in the following example.
+
+
+```
$ hadoop fs -cat /user/rkanter/pig_params.properties
INPUT=/user/rkanter/examples/input-data/text
-</verbatim>
+```
-*Request:*
+**Request:**
-<verbatim>
+
+```
POST /oozie/v1/jobs?jobtype=pig
Content-Type: application/xml;charset=UTF-8
.
@@ -824,44 +865,48 @@
<value>true</value>
</property>
</configuration>
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 201 CREATED
Content-Type: application/json;charset=UTF-8
.
{
id: "job-3"
}
-</verbatim>
+```
----++++ Proxy Hive Job Submission
+#### Proxy Hive Job Submission
You can submit a Workflow that contains a single Hive action without writing a workflow.xml. Any required Jars or other files must
already exist in HDFS.
The following properties are required:
- * =fs.default.name=: The NameNode
- * =mapred.job.tracker=: The JobTracker
- * =user.name=: The username of the user submitting the job
- * =oozie.hive.script=: Contains the hive script you want to run (the actual script, not a file path)
- * =oozie.libpath=: A directory in HDFS that contains necessary Jars for your job
- * =oozie.proxysubmission=: Must be set to =true=
+
+ * `fs.default.name`: The NameNode
+ * `mapred.job.tracker`: The JobTracker
+ * `user.name`: The username of the user submitting the job
+ * `oozie.hive.script`: Contains the hive script you want to run (the actual script, not a file path)
+ * `oozie.libpath`: A directory in HDFS that contains necessary Jars for your job
+ * `oozie.proxysubmission`: Must be set to `true`
The following properties are optional:
- * =oozie.hive.script.params.size=: The number of parameters you'll be passing to Hive
- * =oozie.hive.script.params.n=: A parameter (variable definition for the script) in 'key=value' format, the 'n' should be an integer starting with 0 to indicate the parameter number
- * =oozie.hive.options.size=: The number of options you'll be passing to Hive
- * =oozie.hive.options.n=: An argument to pass to Hive, the 'n' should be an integer starting with 0 to indicate the option number
-The =oozie.hive.options.n= parameters are sent directly to Hive without any modification unless they start with =-D=, in which case
-they are put into the <code><configuration></code> element of the action.
+ * `oozie.hive.script.params.size`: The number of parameters you'll be passing to Hive
+ * `oozie.hive.script.params.n`: A parameter (variable definition for the script) in 'key=value' format, the 'n' should be an integer starting with 0 to indicate the parameter number
+ * `oozie.hive.options.size`: The number of options you'll be passing to Hive
+ * `oozie.hive.options.n`: An argument to pass to Hive, the 'n' should be an integer starting with 0 to indicate the option number
-*Request:*
+The `oozie.hive.options.n` parameters are sent directly to Hive without any modification unless they start with `-D`, in which case
+they are put into the `<configuration>` element of the action.
-<verbatim>
+**Request:**
+
+
+```
POST /oozie/v1/jobs?jobtype=hive
Content-Type: application/xml;charset=UTF-8
.
@@ -907,39 +952,43 @@
<value>true</value>
</property>
</configuration>
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 201 CREATED
Content-Type: application/json;charset=UTF-8
.
{
id: "job-3"
}
-</verbatim>
+```
----++++ Proxy Sqoop Job Submission
+#### Proxy Sqoop Job Submission
You can submit a Workflow that contains a single Sqoop command without writing a workflow.xml. Any required Jars or other
files must already exist in HDFS.
The following properties are required:
- * =fs.default.name=: The NameNode
- * =mapred.job.tracker=: The JobTracker
- * =user.name=: The username of the user submitting the job
- * =oozie.sqoop.command=: The sqoop command you want to run where each argument occupies one line or separated by "\n"
- * =oozie.libpath=: A directory in HDFS that contains necessary Jars for your job
- * =oozie.proxysubmission=: Must be set to =true=
+
+ * `fs.default.name`: The NameNode
+ * `mapred.job.tracker`: The JobTracker
+ * `user.name`: The username of the user submitting the job
+ * `oozie.sqoop.command`: The sqoop command you want to run where each argument occupies one line or separated by "\n"
+ * `oozie.libpath`: A directory in HDFS that contains necessary Jars for your job
+ * `oozie.proxysubmission`: Must be set to `true`
The following properties are optional:
- * =oozie.sqoop.options.size=: The number of options you'll be passing to Sqoop Hadoop job
- * =oozie.sqoop.options.n=: An argument to pass to Sqoop hadoop job conf, the 'n' should be an integer starting with 0 to indicate the option number
-*Request:*
+ * `oozie.sqoop.options.size`: The number of options you'll be passing to Sqoop Hadoop job
+ * `oozie.sqoop.options.n`: An argument to pass to Sqoop hadoop job conf, the 'n' should be an integer starting with 0 to indicate the option number
-<verbatim>
+**Request:**
+
+
+```
POST /oozie/v1/jobs?jobtype=sqoop
Content-Type: application/xml;charset=UTF-8
.
@@ -981,52 +1030,56 @@
<value>true</value>
</property>
</configuration>
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 201 CREATED
Content-Type: application/json;charset=UTF-8
.
{
id: "job-3"
}
-</verbatim>
+```
----++++ Managing a Job
+#### Managing a Job
A HTTP PUT request starts, suspends, resumes, kills, update or dryruns a job.
-*Request:*
+**Request:**
-<verbatim>
+
+```
PUT /oozie/v1/job/job-3?action=start
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
-</verbatim>
+```
Valid values for the 'action' parameter are 'start', 'suspend', 'resume', 'kill', 'dryrun', 'rerun', and 'change'.
Rerunning and changing a job require additional parameters, and are described below:
----+++++ Re-Running a Workflow Job
+##### Re-Running a Workflow Job
-A workflow job in =SUCCEEDED=, =KILLED= or =FAILED= status can be partially rerun specifying a list
+A workflow job in `SUCCEEDED`, `KILLED` or `FAILED` status can be partially rerun specifying a list
of workflow nodes to skip during the rerun. All the nodes in the skip list must have complete its
execution.
The rerun job will have the same job ID.
-A rerun request is done with a HTTP PUT request with a =rerun= action.
+A rerun request is done with a HTTP PUT request with a `rerun` action.
-*Request:*
+**Request:**
-<verbatim>
+
+```
PUT /oozie/v1/job/job-3?action=rerun
Content-Type: application/xml;charset=UTF-8
.
@@ -1046,140 +1099,153 @@
</property>
...
</configuration>
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
-</verbatim>
+```
----+++++ Re-Running a coordinator job
+##### Re-Running a coordinator job
-A coordinator job in =RUNNING= =SUCCEEDED=, =KILLED= or =FAILED= status can be partially rerun by specifying the coordinator actions
+A coordinator job in `RUNNING` `SUCCEEDED`, `KILLED` or `FAILED` status can be partially rerun by specifying the coordinator actions
to re-execute.
-A rerun request is done with an HTTP PUT request with a =coord-rerun= =action=.
+A rerun request is done with an HTTP PUT request with a `coord-rerun` `action`.
-The =type= of the rerun can be =date= or =action=.
+The `type` of the rerun can be `date` or `action`.
-The =scope= of the rerun depends on the type:
-* =date=: a comma-separated list of date ranges. Each date range element is specified with dates separated by =::=
-* =action=: a comma-separated list of action ranges. Each action range is specified with two action numbers separated by =-=
+The `scope` of the rerun depends on the type:
+* `date`: a comma-separated list of date ranges. Each date range element is specified with dates separated by `::`
+* `action`: a comma-separated list of action ranges. Each action range is specified with two action numbers separated by `-`
-The =refresh= parameter can be =true= or =false= to specify if the user wants to refresh an action's input and output events.
+The `refresh` parameter can be `true` or `false` to specify if the user wants to refresh an action's input and output events.
-The =nocleanup= parameter can be =true= or =false= to specify is the user wants to cleanup output events for the rerun actions.
+The `nocleanup` parameter can be `true` or `false` to specify is the user wants to cleanup output events for the rerun actions.
-*Request:*
+**Request:**
-<verbatim>
+
+```
PUT /oozie/v1/job/job-3?action=coord-rerun&type=action&scope=1-2&refresh=false&nocleanup=false
.
-</verbatim>
+```
-or
+or
-<verbatim>
+
+```
PUT /oozie/v1/job/job-3?action=coord-rerun&type=date2009-02-01T00:10Z::2009-03-01T00:10Z&scope=&refresh=false&nocleanup=false
.
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
-</verbatim>
+```
----+++++ Re-Running a bundle job
+##### Re-Running a bundle job
-A coordinator job in =RUNNING= =SUCCEEDED=, =KILLED= or =FAILED= status can be partially rerun by specifying the coordinators to
+A coordinator job in `RUNNING` `SUCCEEDED`, `KILLED` or `FAILED` status can be partially rerun by specifying the coordinators to
re-execute.
-A rerun request is done with an HTTP PUT request with a =bundle-rerun= =action=.
+A rerun request is done with an HTTP PUT request with a `bundle-rerun` `action`.
-A comma separated list of coordinator job names (not IDs) can be specified in the =coord-scope= parameter.
+A comma separated list of coordinator job names (not IDs) can be specified in the `coord-scope` parameter.
-The =date-scope= parameter is a comma-separated list of date ranges. Each date range element is specified with dates separated
-by =::=. If empty or not included, Oozie will figure this out for you
+The `date-scope` parameter is a comma-separated list of date ranges. Each date range element is specified with dates separated
+by `::`. If empty or not included, Oozie will figure this out for you
-The =refresh= parameter can be =true= or =false= to specify if the user wants to refresh the coordinator's input and output events.
+The `refresh` parameter can be `true` or `false` to specify if the user wants to refresh the coordinator's input and output events.
-The =nocleanup= parameter can be =true= or =false= to specify is the user wants to cleanup output events for the rerun coordinators.
+The `nocleanup` parameter can be `true` or `false` to specify is the user wants to cleanup output events for the rerun coordinators.
-*Request:*
+**Request:**
-<verbatim>
+
+```
PUT /oozie/v1/job/job-3?action=bundle-rerun&coord-scope=coord-1&refresh=false&nocleanup=false
.
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
-</verbatim>
+```
----+++++ Changing endtime/concurrency/pausetime of a Coordinator Job
+##### Changing endtime/concurrency/pausetime of a Coordinator Job
-A coordinator job not in =KILLED= status can have it's endtime, concurrency, or pausetime changed.
+A coordinator job not in `KILLED` status can have it's endtime, concurrency, or pausetime changed.
-A change request is done with an HTTP PUT request with a =change= =action=.
+A change request is done with an HTTP PUT request with a `change` `action`.
-The =value= parameter can contain any of the following:
+The `value` parameter can contain any of the following:
* endtime: the end time of the coordinator job.
* concurrency: the concurrency of the coordinator job.
* pausetime: the pause time of the coordinator job.
-Multiple arguments can be passed to the =value= parameter by separating them with a ';' character.
+Multiple arguments can be passed to the `value` parameter by separating them with a ';' character.
If an already-succeeded job changes its end time, its status will become running.
-*Request:*
+**Request:**
-<verbatim>
+
+```
PUT /oozie/v1/job/job-3?action=change&value=endtime=2011-12-01T05:00Z
.
-</verbatim>
+```
or
-<verbatim>
+
+```
PUT /oozie/v1/job/job-3?action=change&value=concurrency=100
.
-</verbatim>
+```
or
-<verbatim>
+
+```
PUT /oozie/v1/job/job-3?action=change&value=pausetime=2011-12-01T05:00Z
.
-</verbatim>
+```
or
-<verbatim>
+
+```
PUT /oozie/v1/job/job-3?action=change&value=endtime=2011-12-01T05:00Z;concurrency=100;pausetime=2011-12-01T05:00Z
.
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
-</verbatim>
+```
----+++++ Updating coordinator definition and properties
-Existing coordinator definition and properties will be replaced by new definition and properties. Refer [[DG_CommandLineTool#Updating_coordinator_definition_and_properties][Updating coordinator definition and properties]]
+##### Updating coordinator definition and properties
+Existing coordinator definition and properties will be replaced by new definition and properties. Refer [Updating coordinator definition and properties](DG_CommandLineTool.html#Updating_coordinator_definition_and_properties)
-<verbatim>
+
+```
PUT oozie/v2/job/0000000-140414102048137-oozie-puru-C?action=update
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
{"update":
@@ -1197,26 +1263,28 @@
<name>queueName<\/name>\r\n******************************************\n"
}
}
-</verbatim>
+```
----++++ Job Information
+#### Job Information
A HTTP GET request retrieves the job information.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/job/job-3?show=info&timezone=GMT
-</verbatim>
+```
-*Response for a workflow job:*
+**Response for a workflow job:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
-{
+{
id: "0-200905191240-oozie-W",
appName: "indexer-workflow",
appPath: "hdfs://user/bansalm/indexer.wf",
@@ -1250,11 +1318,12 @@
...
]
}
-</verbatim>
+```
-*Response for a coordinator job:*
+**Response for a coordinator job:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -1282,11 +1351,12 @@
nominalTime: "Fri, 01 Jan 2010 01:00:00 GMT",
...
}
-</verbatim>
+```
-*Response for a bundle job:*
+**Response for a bundle job:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -1315,9 +1385,9 @@
}
...
}
-</verbatim>
+```
-*Getting all the Workflows corresponding to a Coordinator Action:*
+**Getting all the Workflows corresponding to a Coordinator Action:**
A coordinator action kicks off different workflows for its original run and all subsequent reruns.
Getting a list of those workflow ids is a useful tool to keep track of your actions' runs and
@@ -1326,13 +1396,15 @@
Both v1 and v2 API are supported. v0 is not supported.
-<verbatim>
+
+```
GET /oozie/v2/job/0000001-111219170928042-oozie-joe-C@1?show=allruns
-</verbatim>
+```
-*Response*
+**Response**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -1356,45 +1428,51 @@
"endTime":"Mon, 24 Mar 2014 23:44:24 GMT"
}
]}
-</verbatim>
+```
-An alternate API is also available for the same output. With this API, one can pass the coordinator *JOB* Id
-followed by query params - type=action and scope=<action-number>. One single action number can be passed at a time.
+An alternate API is also available for the same output. With this API, one can pass the coordinator **JOB** Id
+followed by query params - `type=action` and `scope=<action-number>`. One single action number can be passed at a time.
-<verbatim>
+
+```
GET /oozie/v2/job/0000001-111219170928042-oozie-joe-C?show=allruns&type=action&scope=1
-</verbatim>
+```
-*Retrieve a subset of actions*
+**Retrieve a subset of actions**
-Query parameters, =offset= and =length= can be specified with a workflow job to retrieve specific actions. Default is offset=0, len=1000
-<verbatim>
+Query parameters, `offset` and `length` can be specified with a workflow job to retrieve specific actions. Default is offset=0, len=1000
+
+```
GET /oozie/v1/job/0000002-130507145349661-oozie-joe-W?show=info&offset=5&len=10
-</verbatim>
-Query parameters, =offset=, =length=, =filter= can be specified with a coordinator job to retrieve specific actions.
-Query parameter, =order= with value "desc" can be used to retrieve the latest coordinator actions materialized instead of actions from @1.
-Query parameters =filter= can be used to retrieve coordinator actions matching specific status.
+```
+Query parameters, `offset`, `length`, `filter` can be specified with a coordinator job to retrieve specific actions.
+Query parameter, `order` with value "desc" can be used to retrieve the latest coordinator actions materialized instead of actions from @1.
+Query parameters `filter` can be used to retrieve coordinator actions matching specific status.
Default is offset=0, len=0 for v2/job (i.e., does not return any coordinator actions) and offset=0, len=1000 with v1/job and v0/job.
-So if you need actions to be returned with v2 API, specifying =len= parameter is necessary.
-Default =order= is "asc".
-<verbatim>
+So if you need actions to be returned with v2 API, specifying `len` parameter is necessary.
+Default `order` is "asc".
+
+```
GET /oozie/v1/job/0000001-111219170928042-oozie-joe-C?show=info&offset=5&len=10&filter=status%3DKILLED&order=desc
-</verbatim>
-Note that the filter is URL encoded, its decoded value is <code>status=KILLED</code>.
-<verbatim>
+```
+Note that the filter is URL encoded, its decoded value is `status=KILLED`.
+
+```
GET /oozie/v1/job/0000001-111219170928042-oozie-joe-C?show=info&filter=status%21%3DSUCCEEDED&order=desc
-</verbatim>
+```
This retrieves coordinator actions except for SUCCEEDED status, which is useful for debugging.
-*Retrieve information of the retry attempts of the workflow action:*
+**Retrieve information of the retry attempts of the workflow action:**
-<verbatim>
+
+```
GET oozie/v2/job/0000000-161212175234862-oozie-puru-W@pig-node?show=retries
-</verbatim>
+```
-*Response*
+**Response**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -1415,21 +1493,23 @@
}
]
}
-</verbatim>
+```
----++++ Job Application Definition
+#### Job Application Definition
A HTTP GET request retrieves the workflow or a coordinator job definition file.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/job/job-3?show=definition
-</verbatim>
+```
-*Response for a workflow job:*
+**Response for a workflow job:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/xml;charset=UTF-8
.
@@ -1439,53 +1519,57 @@
...
<end name='end' />
</workflow-app>
-</verbatim>
+```
-*Response for a coordinator job:*
+**Response for a coordinator job:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/xml;charset=UTF-8
.
<?xml version="1.0" encoding="UTF-8"?>
-<coordinator-app name='abc-app' xmlns="uri:oozie:coordinator:0.1" frequency="${days(1)}
+<coordinator-app name='abc-app' xmlns="uri:oozie:coordinator:0.1" frequency="${days(1)}
start="2009-01-01T00:00Z" end="2009-12-31T00:00Z" timezone="America/Los_Angeles">
<datasets>
...
</datasets>
...
</coordinator-app>
-</verbatim>
+```
-*Response for a bundle job:*
+**Response for a bundle job:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/xml;charset=UTF-8
.
<?xml version="1.0" encoding="UTF-8"?>
-<bundle-app name='abc-app' xmlns="uri:oozie:coordinator:0.1"
+<bundle-app name='abc-app' xmlns="uri:oozie:coordinator:0.1"
start="2009-01-01T00:00Z" end="2009-12-31T00:00Z"">
<datasets>
...
</datasets>
...
</bundle-app>
-</verbatim>
+```
----++++ Job Log
+#### Job Log
An HTTP GET request retrieves the job log.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/job/job-3?show=log
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: text/plain;charset=UTF-8
.
@@ -1493,21 +1577,23 @@
23:21:31,272 TRACE oozieapp:526 - USER[bansalm] GROUP[other] TOKEN[-] APP[test-wf] JOB[0-20090518232130-oozie-tucu] ACTION[mr-1] Start
23:21:31,305 TRACE oozieapp:526 - USER[bansalm] GROUP[other] TOKEN[-] APP[test-wf] JOB[0-20090518232130-oozie-tucu] ACTION[mr-1] End
...
-</verbatim>
+```
----++++ Job Error Log
+#### Job Error Log
An HTTP GET request retrieves the job error log.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v2/job/0000000-150121110331712-oozie-puru-B?show=errorlog
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: text/plain;charset=UTF-8
2015-01-21 11:33:29,090 WARN CoordSubmitXCommand:523 - SERVER[-] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0000000-150121110331712-oozie-puru-B] ACTION[] SAXException :
@@ -1515,46 +1601,50 @@
at org.apache.xerces.util.ErrorHandlerWrapper.createSAXParseException(Unknown Source)
at org.apache.xerces.util.ErrorHandlerWrapper.error(Unknown Source)
...
-</verbatim>
+```
----++++ Job Audit Log
+#### Job Audit Log
An HTTP GET request retrieves the job audit log.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v2/job/0000000-150322000230582-oozie-puru-C?show=auditlog
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: text/plain;charset=UTF-8
2015-03-22 00:04:35,494 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [start], PARAMETER [null], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]
2015-03-22 00:05:13,823 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [suspend], PARAMETER [0000000-150322000230582-oozie-puru-C], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]
2015-03-22 00:06:59,561 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [suspend], PARAMETER [0000000-150322000230582-oozie-puru-C], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]
2015-03-22 23:22:20,012 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [suspend], PARAMETER [0000000-150322000230582-oozie-puru-C], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]
-2015-03-22 23:28:48,218 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [resume], PARAMETER [0000000-150322000230582-oozie-puru-C], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]</verbatim>
-</verbatim>
+2015-03-22 23:28:48,218 INFO oozieaudit:520 - IP [-], USER [purushah], GROUP [null], APP [-], JOBID [0000000-150322000230582-oozie-puru-C], OPERATION [resume], PARAMETER [0000000-150322000230582-oozie-puru-C], STATUS [SUCCESS], HTTPCODE [200], ERRORCODE [null], ERRORMESSAGE [null]
+```
----++++ Filtering the server logs with logfilter options
+#### Filtering the server logs with logfilter options
User can provide multiple option to filter logs using -logfilter opt1=val1;opt2=val1;opt3=val1. This can be used to fetch only just logs of interest faster as fetching Oozie server logs is slow due to the overhead of pattern matching.
-<verbatim>
+
+```
GET /oozie/v1/job/0000003-140319184715726-oozie-puru-C?show=log&logfilter=limit=3;loglevel=WARN
-</verbatim>
+```
-Refer to the [[DG_CommandLineTool#Filtering_the_server_logs_with_logfilter_options][Filtering the server logs with logfilter options]] for more details.
+Refer to the [Filtering the server logs with logfilter options](DG_CommandLineTool.html#Filtering_the_server_logs_with_logfilter_options) for more details.
----++++ Job graph
+#### Job graph
-An =HTTP GET= request returns the image of the workflow DAG (rendered as a PNG or SVG image, or as a DOT string).
+An `HTTP GET` request returns the image of the workflow DAG (rendered as a PNG or SVG image, or as a DOT string).
+
* The nodes that are being executed are painted yellow
* The nodes that have successfully executed are painted green
* The nodes that have failed execution are painted red
@@ -1563,132 +1653,148 @@
* An arc painted red marks the failure of the node and highlights the _error_ action
* An arc painted gray marks a path not taken yet
-*PNG request:*
-<verbatim>
-GET /oozie/v1/job/job-3?show=graph[&show-kill=true][&format=png]
-</verbatim>
+**PNG request:**
-*PNG response:*
-<verbatim>
+```
+GET /oozie/v1/job/job-3?show=graph[&show-kill=true][&format=png]
+```
+
+**PNG response:**
+
+```
HTTP/1.1 200 OK
Content-Type: image/png
Content-Length: {image_size_in_bytes}
{image_bits}
+```
-*SVG request:*
-<verbatim>
+**SVG request:**
+
+```
GET /oozie/v1/job/job-3?show=graph[&show-kill=true]&format=svg
-</verbatim>
+```
-*SVG response:*
-<verbatim>
+**SVG response:**
+
+```
HTTP/1.1 200 OK
Content-Type: image/svg+xml
Content-Length: {image_size_in_bytes}
{image_bits}
+```
-*DOT request:*
-<verbatim>
+**DOT request:**
+
+```
GET /oozie/v1/job/job-3?show=graph[&show-kill=true]&format=dot
-</verbatim>
+```
-*DOT response:*
-<verbatim>
+**DOT response:**
+
+```
HTTP/1.1 200 OK
Content-Type: text/plain
Content-Length: {dot_size_in_bytes}
{dot_bytes}
-</verbatim>
+```
-The optional =show-kill= parameter shows =kill= node in the graph. Valid values for this parameter are =1=, =yes=, and =true=.
-This parameter has no effect when workflow fails and the failure node leads to the =kill= node; in that case =kill= node is shown
+The optional `show-kill` parameter shows `kill` node in the graph. Valid values for this parameter are `1`, `yes`, and `true`.
+This parameter has no effect when workflow fails and the failure node leads to the `kill` node; in that case `kill` node is shown
always.
-The optional =format= parameter describes whether the response has to be rendered as a PNG image, or an SVG image, or a DOT string.
-When omitted, =format= is considered as =png= for backwards compatibility. Oozie Web UI uses the =svg= =format=.
+The optional `format` parameter describes whether the response has to be rendered as a PNG image, or an SVG image, or a DOT string.
+When omitted, `format` is considered as `png` for backwards compatibility. Oozie Web UI uses the `svg` `format`.
The node labels are the node names provided in the workflow XML.
-This API returns =HTTP 400= when run on a resource other than a workflow, viz. bundle and coordinator.
+This API returns `HTTP 400` when run on a resource other than a workflow, viz. bundle and coordinator.
----++++ Job Status
+#### Job Status
-An =HTTP GET= request that returns the current status (e.g. =SUCCEEDED=, =KILLED=, etc) of a given job. If you are only interested
-in the status, and don't want the rest of the information that the =info= query provides, it is recommended to use this call
+An `HTTP GET` request that returns the current status (e.g. `SUCCEEDED`, `KILLED`, etc) of a given job. If you are only interested
+in the status, and don't want the rest of the information that the `info` query provides, it is recommended to use this call
as it is more efficient.
-*Request*
-<verbatim>
+**Request**
+
+```
GET /oozie/v2/job/0000000-140908152307821-oozie-rkan-C?show=status
-</verbatim>
+```
-*Response*
+**Response**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
{
"status" : "SUCCEEDED"
}
-</verbatim>
+```
It accepts any valid Workflow Job ID, Coordinator Job ID, Coordinator Action ID, or Bundle Job ID.
----++++ Changing job SLA definition and alerting
-An =HTTP PUT= request to change job SLA alert status/SLA definition.
+#### Changing job SLA definition and alerting
+An `HTTP PUT` request to change job SLA alert status/SLA definition.
* All sla commands takes actions-list or date parameter.
- * =date=: a comma-separated list of date ranges. Each date range element is specified with dates separated by =::=
- * =action-list=: a comma-separated list of action ranges. Each action range is specified with two action numbers separated by =-=
- * For bundle jobs additional =coordinators= (coord_name/id) parameter can be passed.
- * Sla change command need extra parameter =value= to specify new sla definition.
-
-
+ * `date`: a comma-separated list of date ranges. Each date range element is specified with dates separated by `::`
+ * `action-list`: a comma-separated list of action ranges. Each action range is specified with two action numbers separated by `-`
+ * For bundle jobs additional `coordinators` (coord_name/id) parameter can be passed.
+ * Sla change command need extra parameter `value` to specify new sla definition.
* Changing SLA definition
+
SLA definition of should-start, should-end, nominal-time and max-duration can be changed.
-<verbatim>
+
+```
PUT /oozie/v2/job/0000003-140319184715726-oozie-puru-C?action=sla-change&value=<key>=<value>;...;<key>=<value>
-</verbatim>
+```
* Disabling SLA alert
-<verbatim>
+
+```
PUT /oozie/v2/job/0000003-140319184715726-oozie-puru-C?action=sla-disable&action-list=3-4
-</verbatim>
+```
Will disable SLA alert for actions 3 and 4.
-<verbatim>
+
+```
PUT /oozie/v1/job/0000003-140319184715726-oozie-puru-C?action=sla-disable&date=2009-02-01T00:10Z::2009-03-01T00:10Z
-</verbatim>
+```
Will disable SLA alert for actions whose nominal time is in-between 2009-02-01T00:10Z 2009-03-01T00:10Z (inclusive).
-<verbatim>
+
+```
PUT /oozie/v1/job/0000004-140319184715726-oozie-puru-B?action=sla-disable&date=2009-02-01T00:10Z::2009-03-01T00:10Z&coordinators=abc
-</verbatim>
+```
For bundle jobs additional coordinators (list of comma separated coord_name/id) parameter can be passed.
* Enabling SLA alert
-<verbatim>
+
+```
PUT /oozie/v2/job/0000003-140319184715726-oozie-puru-C?action=sla-enable&action-list=1,14,17-20
-</verbatim>
+```
Will enable SLA alert for actions 1,14,17,18,19,20.
----+++ Getting missing dependencies of coordinator action(s)
+### Getting missing dependencies of coordinator action(s)
-<verbatim>
+
+```
GET oozie/v2/job/0000000-170104115137443-oozie-puru-C?show=missing-dependencies&action-list=1,20
-</verbatim>
+```
-*Response*
+**Response**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
@@ -1728,22 +1834,24 @@
"id": 20
}]
}
-</verbatim>
----++++ Jobs Information
+```
+#### Jobs Information
A HTTP GET request retrieves workflow and coordinator jobs information.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/jobs?filter=user%3Dbansalm&offset=1&len=50&timezone=GMT
-</verbatim>
+```
-Note that the filter is URL encoded, its decoded value is <code>user=bansalm</code>.
+Note that the filter is URL encoded, its decoded value is `user=bansalm`.
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -1793,11 +1901,12 @@
...
]
}
-</verbatim>
+```
No action information is returned when querying for multiple jobs.
-The syntax for the filter is <verbatim>[NAME=VALUE][;NAME=VALUE]*</verbatim>
+
+The syntax for the filter is `[NAME=VALUE][;NAME=VALUE]*`
Valid filter names are:
@@ -1809,36 +1918,38 @@
* status: the status of the job
* startCreatedTime : the start of the window about workflow job's created time
* endCreatedTime : the end of above window
- * sortby: order the results. Supported values for =sortby= are: =createdTime= and =lastModifiedTime=
+ * sortby: order the results. Supported values for `sortby` are: `createdTime` and `lastModifiedTime`
The query will do an AND among all the filter names.
The query will do an OR among all the filter values for the same name. Multiple values must be specified as different
name value pairs.
-Additionally the =offset= and =len= parameters can be used for pagination. The start parameter is base 1.
+Additionally the `offset` and `len` parameters can be used for pagination. The start parameter is base 1.
-Moreover, the =jobtype= parameter could be used to determine what type of job is looking for.
-The valid values of job type are: =wf=, =coordinator= or =bundle=.
+Moreover, the `jobtype` parameter could be used to determine what type of job is looking for.
+The valid values of job type are: `wf`, `coordinator` or `bundle`.
-startCreatedTime and endCreatedTime should be specified either in *ISO8601 (UTC)* format *(yyyy-MM-dd'T'HH:mm'Z')* or
+startCreatedTime and endCreatedTime should be specified either in **ISO8601 (UTC)** format **(yyyy-MM-dd'T'HH:mm'Z')** or
a offset value in days or hours or minutes from the current time. For example, -2d means the (current time - 2 days),
-3h means the (current time - 3 hours), -5m means the (current time - 5 minutes).
----++++ Bulk modify jobs
+#### Bulk modify jobs
A HTTP PUT request can kill, suspend, or resume all jobs that satisfy the url encoded parameters.
-*Request:*
+**Request:**
-<verbatim>
+
+```
PUT /oozie/v1/jobs?action=kill&filter=name%3Dcron-coord&offset=1&len=50&jobtype=coordinator
-</verbatim>
+```
This request will kill all the coordinators with name=cron-coord up to 50 of them.
-Note that the filter is URL encoded, its decoded value is <code>name=cron-coord</code>.
-The syntax for the filter is <verbatim>[NAME=VALUE][;NAME=VALUE]*</verbatim>
+Note that the filter is URL encoded, its decoded value is `name=cron-coord`.
+
+The syntax for the filter is `[NAME=VALUE][;NAME=VALUE]*`
Valid filter names are:
@@ -1852,13 +1963,14 @@
The query will do an OR among all the filter values for the same name. Multiple values must be specified as different
name value pairs.
-Additionally the =offset= and =len= parameters can be used for pagination. The start parameter is base 1.
+Additionally the `offset` and `len` parameters can be used for pagination. The start parameter is base 1.
-Moreover, the =jobtype= parameter could be used to determine what type of job is looking for.
-The valid values of job type are: =wf=, =coordinator= or =bundle=
+Moreover, the `jobtype` parameter could be used to determine what type of job is looking for.
+The valid values of job type are: `wf`, `coordinator` or `bundle`
-*Response:*
-<verbatim>
+**Response:**
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -1895,17 +2007,19 @@
...
]
}
-</verbatim>
+```
-<verbatim>
+
+```
PUT /oozie/v1/jobs?action=suspend&filter=status%3Drunning&offset=1&len=50&jobtype=wf
-</verbatim>
+```
This request will suspend all the workflows with status=running up to 50 of them.
-Note that the filter is URL encoded, its decoded value is <code>status=running</code>.
+Note that the filter is URL encoded, its decoded value is `status=running`.
-*Response:*
-<verbatim>
+**Response:**
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -1943,21 +2057,21 @@
...
]
}
-</verbatim>
+```
----++++ Jobs information using Bulk API
+#### Jobs information using Bulk API
A HTTP GET request retrieves a bulk response for all actions, corresponding to a particular bundle, that satisfy user specified criteria.
This is useful for monitoring purposes, where user can find out about the status of downstream jobs with a single bulk request.
The criteria are used for filtering the actions returned. Valid options (_case insensitive_) for these request criteria are:
- * *bundle*: the application name from the bundle definition
- * *coordinators*: the application name(s) from the coordinator definition.
- * *actionStatus*: the status of coordinator action (Valid values are WAITING, READY, SUBMITTED, RUNNING, SUSPENDED, TIMEDOUT, SUCCEEDED, KILLED, FAILED)
- * *startCreatedTime*: the start of the window you want to look at, of the actions' created time
- * *endCreatedTime*: the end of above window
- * *startScheduledTime*: the start of the window you want to look at, of the actions' scheduled i.e. nominal time.
- * *endScheduledTime*: the end of above window
+ * **bundle**: the application name from the bundle definition
+ * **coordinators**: the application name(s) from the coordinator definition.
+ * **actionStatus**: the status of coordinator action (Valid values are WAITING, READY, SUBMITTED, RUNNING, SUSPENDED, TIMEDOUT, SUCCEEDED, KILLED, FAILED)
+ * **startCreatedTime**: the start of the window you want to look at, of the actions' created time
+ * **endCreatedTime**: the end of above window
+ * **startScheduledTime**: the start of the window you want to look at, of the actions' scheduled i.e. nominal time.
+ * **endScheduledTime**: the end of above window
Specifying 'bundle' is REQUIRED. All the rest are OPTIONAL but that might result in thousands of results depending on the size of your job. (pagination comes into play then)
@@ -1966,29 +2080,32 @@
The query will do an AND among all the filter names, and OR among each filter name's values.
-The syntax for the request criteria is <verbatim>[NAME=VALUE][;NAME=VALUE]*</verbatim>
+
+The syntax for the request criteria is `[NAME=VALUE][;NAME=VALUE]*`
For 'coordinators' and 'actionStatus', if user wants to check for multiple values, they can be passed in a comma-separated manner.
-*Note*: The query will do an OR among them. Hence no need to repeat the criteria name
+**Note**: The query will do an OR among them. Hence no need to repeat the criteria name
-All the time values should be specified in *ISO8601 (UTC)* format i.e. *yyyy-MM-dd'T'HH:mm'Z'*
+All the time values should be specified in **ISO8601 (UTC)** format i.e. **yyyy-MM-dd'T'HH:mm'Z'**
-Additionally the =offset= and =len= parameters can be used as usual for pagination. The start parameter is base 1.
+Additionally the `offset` and `len` parameters can be used as usual for pagination. The start parameter is base 1.
If you specify a coordinator in the list, that does not exist, no error is thrown; simply the response will be empty or pertaining to the other valid coordinators.
However, if bundle name provided does not exist, an error is thrown.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v1/jobs?bulk=bundle%3Dmy-bundle-app;coordinators%3Dmy-coord-1,my-coord-5;actionStatus%3DKILLED&offset=1&len=50
-</verbatim>
+```
-Note that the filter is URL encoded, its decoded value is <code>user=chitnis</code>. If typing in browser URL, one can type decoded value itself i.e. using '='
+Note that the filter is URL encoded, its decoded value is `user=chitnis`. If typing in browser URL, one can type decoded value itself i.e. using '='
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -2054,22 +2171,22 @@
...
]
}
-</verbatim>
+```
----++ Oozie Web Services API, V2 (Workflow , Coordinator And Bundle)
+## Oozie Web Services API, V2 (Workflow , Coordinator And Bundle)
The Oozie Web Services API is a HTTP REST JSON API.
-All responses are in =UTF-8=.
+All responses are in `UTF-8`.
-Assuming Oozie is running at =OOZIE_URL=, the following web services end points are supported:
+Assuming Oozie is running at `OOZIE_URL`, the following web services end points are supported:
- * <OOZIE_URL>/versions
- * <OOZIE_URL>/v2/admin
- * <OOZIE_URL>/v2/job
- * <OOZIE_URL>/v2/jobs
+ * \<OOZIE_URL\>/versions
+ * \<OOZIE_URL\>/v2/admin
+ * \<OOZIE_URL\>/v2/job
+ * \<OOZIE_URL\>/v2/jobs
-*Changes in v2 job API:*
+**Changes in v2 job API:**
There is a difference in the JSON format of Job Information API (*/job) particularly for map-reduce action.
No change for other actions.
@@ -2078,39 +2195,43 @@
v2 supports retrieving of JMS topic on which job notifications are sent
-*REST API URL:*
+**REST API URL:**
-<verbatim>
+
+```
GET http://localhost:11000/oozie/v2/job/0000002-130507145349661-oozie-vira-W?show=jmstopic
-</verbatim>
+```
-*Changes in v2 admin API:*
+**Changes in v2 admin API:**
v2 adds support for retrieving JMS connection information related to JMS notifications.
-*REST API URL:*
+**REST API URL:**
-<verbatim>
+
+```
GET http://localhost:11000/oozie/v2/admin/jmsinfo
-</verbatim>
+```
v2/jobs remain the same as v1/jobs
----+++ Job and Jobs End-Points
+### Job and Jobs End-Points
----++++ Job Information
+#### Job Information
A HTTP GET request retrieves the job information.
-*Request:*
+**Request:**
-<verbatim>
+
+```
GET /oozie/v2/job/job-3?show=info&timezone=GMT
-</verbatim>
+```
-*Response for a workflow job:*
+**Response for a workflow job:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
@@ -2152,38 +2273,41 @@
...
]
}
-</verbatim>
+```
----++++ Managing a Job
----+++++ Ignore a Coordinator Job or Action
+#### Managing a Job
+##### Ignore a Coordinator Job or Action
-A ignore request is done with an HTTP PUT request with a =ignore=
+A ignore request is done with an HTTP PUT request with a `ignore`
-The =type= parameter supports =action= only.
-The =scope= parameter can contain coordinator action id(s) to be ignored.
-Multiple action ids can be passed to the =scope= parameter
+The `type` parameter supports `action` only.
+The `scope` parameter can contain coordinator action id(s) to be ignored.
+Multiple action ids can be passed to the `scope` parameter
-*Request:*
+**Request:**
Ignore a coordinator job
-<verbatim>
+
+```
PUT /oozie/v2/job/job-3?action=ignore
-</verbatim>
+```
Ignore coordinator actions
-<verbatim>
-PUT /oozie/v2/job/job-3?action=ignore&type=action&scope=3-4
-</verbatim>
----+++ Validate End-Point
+```
+PUT /oozie/v2/job/job-3?action=ignore&type=action&scope=3-4
+```
+
+### Validate End-Point
This endpoint is to validate a workflow, coordinator, bundle XML file.
----++++ Validate a local file
+#### Validate a local file
-*Request:*
+**Request:**
-<verbatim>
+
+```
POST /oozie/v2/validate?file=/home/test/myApp/workflow.xml
Content-Type: application/xml;charset=UTF-8
.
@@ -2207,43 +2331,46 @@
</kill>
<end name="end"/>
</workflow-app>
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
{
validate: "Valid workflow-app"
}
-</verbatim>
+```
----++++ Validate a file in HDFS
+#### Validate a file in HDFS
You can validate a workflow, coordinator, bundle XML file in HDFS. The XML file must already exist in HDFS.
-*Request:*
+**Request:**
-<verbatim>
+
+```
POST /oozie/v2/validate?file=hdfs://localhost:8020/user/test/myApp/workflow.xml
Content-Type: application/xml;charset=UTF-8
.
-</verbatim>
+```
-*Response:*
+**Response:**
-<verbatim>
+
+```
HTTP/1.1 200 OK
Content-Type: application/json;charset=UTF-8
.
{
validate: "Valid workflow-app"
}
-</verbatim>
+```
-</noautolink>
+
diff --git a/docs/src/site/twiki/WorkflowFunctionalSpec.twiki b/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
index 46a454c..463635b 100644
--- a/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
+++ b/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
@@ -1,135 +1,157 @@
-<noautolink>
-[[index][::Go back to Oozie Documentation Index::]]
+
+[::Go back to Oozie Documentation Index::](index.html)
-----
----+!! Oozie Specification, a Hadoop Workflow System
-*<center>(v5.0)</center>*
+# Oozie Specification, a Hadoop Workflow System
+**<center>(v5.0)</center>**
The goal of this document is to define a workflow engine system specialized in coordinating the execution of Hadoop
Map/Reduce and Pig jobs.
-%TOC%
+<!-- MACRO{toc|fromDepth=1|toDepth=4} -->
----++ Changelog
+## Changelog
----+++!! 2016FEB19
- * #3.2.7 Updated notes on System.exit(int n) behavior
----+++!! 2015APR29
+**2016FEB19**
- * #3.2.1.4 Added notes about Java action retries
- * #3.2.7 Added notes about Java action retries
----+++!! 2014MAY08
+ * 3.2.7 Updated notes on System.exit(int n) behavior
- * #3.2.2.4 Added support for fully qualified job-xml path
----+++!! 2013JUL03
+**2015APR29**
- * #Appendix A, Added new workflow schema 0.5 and SLA schema 0.2
----+++!! 2012AUG30
+ * 3.2.1.4 Added notes about Java action retries
+ * 3.2.7 Added notes about Java action retries
- * #4.2.2 Added two EL functions (replaceAll and appendAll)
----+++!! 2012JUL26
+**2014MAY08**
- * #Appendix A, updated XML schema 0.4 to include =parameters= element
- * #4.1 Updated to mention about =parameters= element as of schema 0.4
----+++!! 2012JUL23
+ * 3.2.2.4 Added support for fully qualified job-xml path
- * #Appendix A, updated XML schema 0.4 (Fs action)
- * #3.2.4 Updated to mention that a =name-node=, a =job-xml=, and a =configuration= element are allowed in the Fs action as of
+**2013JUL03**
+
+ * Appendix A, Added new workflow schema 0.5 and SLA schema 0.2
+
+**2012AUG30**
+
+ * 4.2.2 Added two EL functions (replaceAll and appendAll)
+
+**2012JUL26**
+
+ * Appendix A, updated XML schema 0.4 to include `parameters` element
+ * 4.1 Updated to mention about `parameters` element as of schema 0.4
+
+**2012JUL23**
+
+ * Appendix A, updated XML schema 0.4 (Fs action)
+ * 3.2.4 Updated to mention that a `name-node`, a `job-xml`, and a `configuration` element are allowed in the Fs action as of
schema 0.4
----+++!! 2012JUN19
- * #Appendix A, added XML schema 0.4
- * #3.2.2.4 Updated to mention that multiple =job-xml= elements are allowed as of schema 0.4
- * #3.2.3 Updated to mention that multiple =job-xml= elements are allowed as of schema 0.4
----+++!! 2011AUG17
+**2012JUN19**
- * #3.2.4 fs 'chmod' xml closing element typo in Example corrected
----+++!! 2011AUG12
+ * Appendix A, added XML schema 0.4
+ * 3.2.2.4 Updated to mention that multiple `job-xml` elements are allowed as of schema 0.4
+ * 3.2.3 Updated to mention that multiple `job-xml` elements are allowed as of schema 0.4
- * #3.2.4 fs 'move' action characteristics updated, to allow for consistent source and target paths and existing target path only if directory
- * #18, Update the doc for user-retry of workflow action.
----+++!! 2011FEB19
+**2011AUG17**
- * #10, Update the doc to rerun from the failed node.
----+++!! 2010OCT31
+ * 3.2.4 fs 'chmod' xml closing element typo in Example corrected
- * #17, Added new section on Shared Libraries
----+++!! 2010APR27
+**2011AUG12**
- * #3.2.3 Added new "arguments" tag to PIG actions
- * #3.2.5 SSH actions are deprecated in Oozie schema 0.1 and removed in Oozie schema 0.2
- * #Appendix A, Added schema version 0.2
-
----+++!! 2009OCT20
+ * 3.2.4 fs 'move' action characteristics updated, to allow for consistent source and target paths and existing target path only if directory
+ * 18, Update the doc for user-retry of workflow action.
- * #Appendix A, updated XML schema
+**2011FEB19**
----+++!! 2009SEP15
+ * 10, Update the doc to rerun from the failed node.
- * #3.2.6 Removing support for sub-workflow in a different Oozie instance (removing the 'oozie' element)
+**2010OCT31**
----+++!! 2009SEP07
+ * 17, Added new section on Shared Libraries
- * #3.2.2.3 Added Map Reduce Pipes specifications.
- * #3.2.2.4 Map-Reduce Examples. Previously was 3.2.2.3.
+**2010APR27**
----+++!! 2009SEP02
+ * 3.2.3 Added new "arguments" tag to PIG actions
+ * 3.2.5 SSH actions are deprecated in Oozie schema 0.1 and removed in Oozie schema 0.2
+ * Appendix A, Added schema version 0.2
- * #10 Added missing skip nodes property name.
- * #3.2.1.4 Reworded action recovery explanation.
----+++!! 2009AUG26
+**2009OCT20**
- * #3.2.9 Added =java= action type
- * #3.1.4 Example uses EL constant to refer to counter group/name
+ * Appendix A, updated XML schema
----+++!! 2009JUN09
- * #12.2.4 Added build version resource to admin end-point
- * #3.2.6 Added flag to propagate workflow configuration to sub-workflows
- * #10 Added behavior for workflow job parameters given in the rerun
- * #11.3.4 workflows info returns pagination information
+**2009SEP15**
----+++!! 2009MAY18
+ * 3.2.6 Removing support for sub-workflow in a different Oozie instance (removing the 'oozie' element)
- * #3.1.4 decision node, 'default' element, 'name' attribute changed to 'to'
- * #3.1.5 fork node, 'transition' element changed to 'start', 'to' attribute change to 'path'
- * #3.1.5 join node, 'transition' element remove, added 'to' attribute to 'join' element
- * #3.2.1.4 Rewording on action recovery section
- * #3.2.2 map-reduce action, added 'job-tracker', 'name-node' actions, 'file', 'file' and 'archive' elements
- * #3.2.2.1 map-reduce action, remove from 'streaming' element 'file', 'file' and 'archive' elements
- * #3.2.2.2 map-reduce action, reorganized streaming section
- * #3.2.3 pig action, removed information about implementation (SSH), changed elements names
- * #3.2.4 fs action, removed 'fs-uri' and 'user-name' elements, file system URI is now specified in path, user is propagated
- * #3.2.6 sub-workflow action, renamed elements 'oozie-url' to 'oozie' and 'workflow-app' to 'app-path'
- * #4 Properties that are valid Java identifiers can be used as ${NAME}
- * #4.1 Renamed default properties file from 'configuration.xml' to 'default-configuration.xml'
- * #4.2 Changes in EL Constants and Functions
- * #5 Updated notification behavior and tokens
- * #6 Changed user propagation behavior
- * #7 Changed application packaging from ZIP to HDFS directory
+
+**2009SEP07**
+
+ * 3.2.2.3 Added Map Reduce Pipes specifications.
+ * 3.2.2.4 Map-Reduce Examples. Previously was 3.2.2.3.
+
+
+**2009SEP02**
+
+ * 10 Added missing skip nodes property name.
+ * 3.2.1.4 Reworded action recovery explanation.
+
+
+**2009AUG26**
+
+ * 3.2.9 Added `java` action type
+ * 3.1.4 Example uses EL constant to refer to counter group/name
+
+
+**2009JUN09**
+
+ * 12.2.4 Added build version resource to admin end-point
+ * 3.2.6 Added flag to propagate workflow configuration to sub-workflows
+ * 10 Added behavior for workflow job parameters given in the rerun
+ * 11.3.4 workflows info returns pagination information
+
+
+**2009MAY18**
+
+ * 3.1.4 decision node, 'default' element, 'name' attribute changed to 'to'
+ * 3.1.5 fork node, 'transition' element changed to 'start', 'to' attribute change to 'path'
+ * 3.1.5 join node, 'transition' element remove, added 'to' attribute to 'join' element
+ * 3.2.1.4 Rewording on action recovery section
+ * 3.2.2 map-reduce action, added 'job-tracker', 'name-node' actions, 'file', 'file' and 'archive' elements
+ * 3.2.2.1 map-reduce action, remove from 'streaming' element 'file', 'file' and 'archive' elements
+ * 3.2.2.2 map-reduce action, reorganized streaming section
+ * 3.2.3 pig action, removed information about implementation (SSH), changed elements names
+ * 3.2.4 fs action, removed 'fs-uri' and 'user-name' elements, file system URI is now specified in path, user is propagated
+ * 3.2.6 sub-workflow action, renamed elements 'oozie-url' to 'oozie' and 'workflow-app' to 'app-path'
+ * 4 Properties that are valid Java identifiers can be used as ${NAME}
+ * 4.1 Renamed default properties file from 'configuration.xml' to 'default-configuration.xml'
+ * 4.2 Changes in EL Constants and Functions
+ * 5 Updated notification behavior and tokens
+ * 6 Changed user propagation behavior
+ * 7 Changed application packaging from ZIP to HDFS directory
* Removed application lifecycle and self containment model sections
- * #10 Changed workflow job recovery, simplified recovery behavior
- * #11 Detailed Web Services API
- * #12 Updated Client API section
- * #15 Updated Action Executor API section
- * #Appendix A XML namespace updated to 'uri:oozie:workflow:0.1'
- * #Appendix A Updated XML schema to changes in map-reduce/pig/fs/ssh actions
- * #Appendix B Updated workflow example to schema changes
+ * 10 Changed workflow job recovery, simplified recovery behavior
+ * 11 Detailed Web Services API
+ * 12 Updated Client API section
+ * 15 Updated Action Executor API section
+ * Appendix A XML namespace updated to 'uri:oozie:workflow:0.1'
+ * Appendix A Updated XML schema to changes in map-reduce/pig/fs/ssh actions
+ * Appendix B Updated workflow example to schema changes
----+++!! 2009MAR25
+
+**2009MAR25**
* Changing all references of HWS to Oozie (project name)
* Typos, XML Formatting
* XML Schema URI correction
----+++!! 2009MAR09
- * Changed =CREATED= job state to =PREP= to have same states as Hadoop
+**2009MAR09**
+
+ * Changed `CREATED` job state to `PREP` to have same states as Hadoop
* Renamed 'hadoop-workflow' element to 'workflow-app'
* Decision syntax changed to be 'switch/case' with no transition indirection
* Action nodes common root element 'action', with the action type as sub-element (using a single built-in XML schema)
@@ -142,32 +164,34 @@
* Added Oozie XML schema to Appendix A
* Added workflow example to Appendix B
----+++!! 2009FEB22
- * Opened [[https://issues.apache.org/jira/browse/HADOOP-5303][JIRA HADOOP-5303]]
+**2009FEB22**
----+++!! 27/DEC/2012:
+ * Opened [JIRA HADOOP-5303](https://issues.apache.org/jira/browse/HADOOP-5303)
+
+
+**27/DEC/2012:**
* Added information on dropping hcatalog table partitions in prepare block
* Added hcatalog EL functions section
----++ 0 Definitions
+## 0 Definitions
-*Action:* An execution/computation task (Map-Reduce job, Pig job, a shell command). It can also be referred as task or
+**Action:** An execution/computation task (Map-Reduce job, Pig job, a shell command). It can also be referred as task or
'action node'.
-*Workflow:* A collection of actions arranged in a control dependency DAG (Directed Acyclic Graph). "control dependency"
+**Workflow:** A collection of actions arranged in a control dependency DAG (Directed Acyclic Graph). "control dependency"
from one action to another means that the second action can't run until the first action has completed.
-*Workflow Definition:* A programmatic description of a workflow that can be executed.
+**Workflow Definition:** A programmatic description of a workflow that can be executed.
-*Workflow Definition Language:* The language used to define a Workflow Definition.
+**Workflow Definition Language:** The language used to define a Workflow Definition.
-*Workflow Job:* An executable instance of a workflow definition.
+**Workflow Job:** An executable instance of a workflow definition.
-*Workflow Engine:* A system that executes workflows jobs. It can also be referred as a DAG engine.
+**Workflow Engine:** A system that executes workflows jobs. It can also be referred as a DAG engine.
----++ 1 Specification Highlights
+## 1 Specification Highlights
A Workflow application is DAG that coordinates the following types of actions: Hadoop, Pig, and
sub-workflows.
@@ -176,7 +200,7 @@
workflows are not supported.
Actions and decisions can be parameterized with job properties, actions output (i.e. Hadoop counters) and file information (file exists, file size, etc). Formal parameters are expressed in the workflow
-definition as =${VAR}= variables.
+definition as `${VAR}` variables.
A Workflow application is a ZIP file that contains the workflow definition (an XML file), all the necessary files to
run all the actions: JAR files for Map/Reduce jobs, shells for streaming Map/Reduce jobs, native libraries, Pig
@@ -191,7 +215,7 @@
When submitting a workflow job, a set of properties resolving all the formal parameters in the workflow definitions
must be provided. This set of properties is a Hadoop configuration.
-Possible states for a workflow jobs are: =PREP=, =RUNNING=, =SUSPENDED=, =SUCCEEDED=, =KILLED= and =FAILED=.
+Possible states for a workflow jobs are: `PREP`, `RUNNING`, `SUSPENDED`, `SUCCEEDED`, `KILLED` and `FAILED`.
In the case of a action start failure in a workflow job, depending on the type of failure, Oozie will attempt automatic
retries, it will request a manual retry or it will fail the workflow job.
@@ -202,119 +226,124 @@
Before doing a resubmission the workflow application could be updated with a patch to fix a problem in the workflow
application code.
-#WorkflowDefinition
----++ 2 Workflow Definition
+<a name="WorkflowDefinition"></a>
+## 2 Workflow Definition
A workflow definition is a DAG with control flow nodes (start, end, decision, fork, join, kill) or action nodes
(map-reduce, pig, etc.), nodes are connected by transitions arrows.
The workflow definition language is XML based and it is called hPDL (Hadoop Process Definition Language).
-Refer to the Appendix A for the[[WorkflowFunctionalSpec#OozieWFSchema][Oozie Workflow Definition XML Schema]]. Appendix
-B has [[WorkflowFunctionalSpec#OozieWFExamples][Workflow Definition Examples]].
+Refer to the Appendix A for the[Oozie Workflow Definition XML Schema](WorkflowFunctionalSpec.html#OozieWFSchema). Appendix
+B has [Workflow Definition Examples](WorkflowFunctionalSpec.html#OozieWFExamples).
----+++ 2.1 Cycles in Workflow Definitions
+### 2.1 Cycles in Workflow Definitions
Oozie does not support cycles in workflow definitions, workflow definitions must be a strict DAG.
At workflow application deployment time, if Oozie detects a cycle in the workflow definition it must fail the
deployment.
----++ 3 Workflow Nodes
+## 3 Workflow Nodes
Workflow nodes are classified in control flow nodes and action nodes:
- * *Control flow nodes:* nodes that control the start and end of the workflow and workflow job execution path.
- * *Action nodes:* nodes that trigger the execution of a computation/processing task.
+ * **Control flow nodes:** nodes that control the start and end of the workflow and workflow job execution path.
+ * **Action nodes:** nodes that trigger the execution of a computation/processing task.
-Node names and transitions must be conform to the following pattern =[a-zA-Z][\-_a-zA-Z0-0]*=, of up to 20 characters
+Node names and transitions must be conform to the following pattern `[a-zA-Z][\-_a-zA-Z0-0]*`, of up to 20 characters
long.
----+++ 3.1 Control Flow Nodes
+### 3.1 Control Flow Nodes
-Control flow nodes define the beginning and the end of a workflow (the =start=, =end= and =kill= nodes) and provide a
-mechanism to control the workflow execution path (the =decision=, =fork= and =join= nodes).
+Control flow nodes define the beginning and the end of a workflow (the `start`, `end` and `kill` nodes) and provide a
+mechanism to control the workflow execution path (the `decision`, `fork` and `join` nodes).
-#StartNode
----++++ 3.1.1 Start Control Node
+<a name="StartNode"></a>
+#### 3.1.1 Start Control Node
-The =start= node is the entry point for a workflow job, it indicates the first workflow node the workflow job must
+The `start` node is the entry point for a workflow job, it indicates the first workflow node the workflow job must
transition to.
-When a workflow is started, it automatically transitions to the node specified in the =start=.
+When a workflow is started, it automatically transitions to the node specified in the `start`.
-A workflow definition must have one =start= node.
+A workflow definition must have one `start` node.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<start to="[NODE-NAME]"/>
...
</workflow-app>
-</verbatim>
+```
-The =to= attribute is the name of first workflow node to execute.
+The `to` attribute is the name of first workflow node to execute.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="foo-wf" xmlns="uri:oozie:workflow:1.0">
...
<start to="firstHadoopJob"/>
...
</workflow-app>
-</verbatim>
+```
-#EndNode
----++++ 3.1.2 End Control Node
+<a name="EndNode"></a>
+#### 3.1.2 End Control Node
-The =end= node is the end for a workflow job, it indicates that the workflow job has completed successfully.
+The `end` node is the end for a workflow job, it indicates that the workflow job has completed successfully.
-When a workflow job reaches the =end= it finishes successfully (SUCCEEDED).
+When a workflow job reaches the `end` it finishes successfully (SUCCEEDED).
-If one or more actions started by the workflow job are executing when the =end= node is reached, the actions will be
+If one or more actions started by the workflow job are executing when the `end` node is reached, the actions will be
killed. In this scenario the workflow job is still considered as successfully run.
-A workflow definition must have one =end= node.
+A workflow definition must have one `end` node.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<end name="[NODE-NAME]"/>
...
</workflow-app>
-</verbatim>
+```
-The =name= attribute is the name of the transition to do to end the workflow job.
+The `name` attribute is the name of the transition to do to end the workflow job.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="foo-wf" xmlns="uri:oozie:workflow:1.0">
...
<end name="end"/>
</workflow-app>
-</verbatim>
+```
-#KillNode
----++++ 3.1.3 Kill Control Node
+<a name="KillNode"></a>
+#### 3.1.3 Kill Control Node
-The =kill= node allows a workflow job to kill itself.
+The `kill` node allows a workflow job to kill itself.
-When a workflow job reaches the =kill= it finishes in error (KILLED).
+When a workflow job reaches the `kill` it finishes in error (KILLED).
-If one or more actions started by the workflow job are executing when the =kill= node is reached, the actions will be
+If one or more actions started by the workflow job are executing when the `kill` node is reached, the actions will be
killed.
-A workflow definition may have zero or more =kill= nodes.
+A workflow definition may have zero or more `kill` nodes.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<kill name="[NODE-NAME]">
@@ -322,17 +351,18 @@
</kill>
...
</workflow-app>
-</verbatim>
+```
-The =name= attribute in the =kill= node is the name of the Kill action node.
+The `name` attribute in the `kill` node is the name of the Kill action node.
-The content of the =message= element will be logged as the kill reason for the workflow job.
+The content of the `message` element will be logged as the kill reason for the workflow job.
-A =kill= node does not have transition elements because it ends the workflow job, as =KILLED=.
+A `kill` node does not have transition elements because it ends the workflow job, as `KILLED`.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="foo-wf" xmlns="uri:oozie:workflow:1.0">
...
<kill name="killBecauseNoInput">
@@ -340,29 +370,31 @@
</kill>
...
</workflow-app>
-</verbatim>
+```
-#DecisionNode
----++++ 3.1.4 Decision Control Node
+<a name="DecisionNode"></a>
+#### 3.1.4 Decision Control Node
-A =decision= node enables a workflow to make a selection on the execution path to follow.
+A `decision` node enables a workflow to make a selection on the execution path to follow.
-The behavior of a =decision= node can be seen as a switch-case statement.
+The behavior of a `decision` node can be seen as a switch-case statement.
-A =decision= node consists of a list of predicates-transition pairs plus a default transition. Predicates are evaluated
-in order or appearance until one of them evaluates to =true= and the corresponding transition is taken. If none of the
-predicates evaluates to =true= the =default= transition is taken.
+A `decision` node consists of a list of predicates-transition pairs plus a default transition. Predicates are evaluated
+in order or appearance until one of them evaluates to `true` and the corresponding transition is taken. If none of the
+predicates evaluates to `true` the `default` transition is taken.
Predicates are JSP Expression Language (EL) expressions (refer to section 4.2 of this document) that resolve into a
-boolean value, =true= or =false=. For example:
+boolean value, `true` or `false`. For example:
-<verbatim>
+
+```
${fs:fileSize('/usr/foo/myinputdir') gt 10 * GB}
-</verbatim>
+```
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<decision name="[NODE-NAME]">
@@ -375,22 +407,23 @@
</decision>
...
</workflow-app>
-</verbatim>
+```
-The =name= attribute in the =decision= node is the name of the decision node.
+The `name` attribute in the `decision` node is the name of the decision node.
-Each =case= elements contains a predicate and a transition name. The predicate ELs are evaluated
-in order until one returns =true= and the corresponding transition is taken.
+Each `case` elements contains a predicate and a transition name. The predicate ELs are evaluated
+in order until one returns `true` and the corresponding transition is taken.
-The =default= element indicates the transition to take if none of the predicates evaluates
-to =true=.
+The `default` element indicates the transition to take if none of the predicates evaluates
+to `true`.
-All decision nodes must have a =default= element to avoid bringing the workflow into an error
+All decision nodes must have a `default` element to avoid bringing the workflow into an error
state if none of the predicates evaluates to true.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="foo-wf" xmlns="uri:oozie:workflow:1.0">
...
<decision name="mydecision">
@@ -408,21 +441,22 @@
</decision>
...
</workflow-app>
-</verbatim>
+```
-#ForkJoinNodes
----++++ 3.1.5 Fork and Join Control Nodes
+<a name="ForkJoinNodes"></a>
+#### 3.1.5 Fork and Join Control Nodes
-A =fork= node splits one path of execution into multiple concurrent paths of execution.
+A `fork` node splits one path of execution into multiple concurrent paths of execution.
-A =join= node waits until every concurrent execution path of a previous =fork= node arrives to it.
+A `join` node waits until every concurrent execution path of a previous `fork` node arrives to it.
-The =fork= and =join= nodes must be used in pairs. The =join= node assumes concurrent execution paths are children of
-the same =fork= node.
+The `fork` and `join` nodes must be used in pairs. The `join` node assumes concurrent execution paths are children of
+the same `fork` node.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<fork name="[FORK-NODE-NAME]">
@@ -434,18 +468,19 @@
<join name="[JOIN-NODE-NAME]" to="[NODE-NAME]" />
...
</workflow-app>
-</verbatim>
+```
-The =name= attribute in the =fork= node is the name of the workflow fork node. The =start= attribute in the =path=
-elements in the =fork= node indicate the name of the workflow node that will be part of the concurrent execution paths.
+The `name` attribute in the `fork` node is the name of the workflow fork node. The `start` attribute in the `path`
+elements in the `fork` node indicate the name of the workflow node that will be part of the concurrent execution paths.
-The =name= attribute in the =join= node is the name of the workflow join node. The =to= attribute in the =join= node
+The `name` attribute in the `join` node is the name of the workflow join node. The `to` attribute in the `join` node
indicates the name of the workflow node that will executed after all concurrent execution paths of the corresponding
fork arrive to the join node.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<fork name="forking">
@@ -473,37 +508,37 @@
<join name="joining" to="nextaction"/>
...
</workflow-app>
-</verbatim>
+```
By default, Oozie performs some validation that any forking in a workflow is valid and won't lead to any incorrect behavior or
instability. However, if Oozie is preventing a workflow from being submitted and you are very certain that it should work, you can
disable forkjoin validation so that Oozie will accept the workflow. To disable this validation just for a specific workflow, simply
-set =oozie.wf.validate.ForkJoin= to =false= in the job.properties file. To disable this validation for all workflows, simply set
-=oozie.validate.ForkJoin= to =false= in the oozie-site.xml file. Disabling this validation is determined by the AND of both of
+set `oozie.wf.validate.ForkJoin` to `false` in the job.properties file. To disable this validation for all workflows, simply set
+`oozie.validate.ForkJoin` to `false` in the oozie-site.xml file. Disabling this validation is determined by the AND of both of
these properties, so it will be disabled if either or both are set to false and only enabled if both are set to true (or not
specified).
-#ActionNodes
----+++ 3.2 Workflow Action Nodes
+<a name="ActionNodes"></a>
+### 3.2 Workflow Action Nodes
Action nodes are the mechanism by which a workflow triggers the execution of a computation/processing task.
----++++ 3.2.1 Action Basis
+#### 3.2.1 Action Basis
The following sub-sections define common behavior and capabilities for all action types.
----+++++ 3.2.1.1 Action Computation/Processing Is Always Remote
+##### 3.2.1.1 Action Computation/Processing Is Always Remote
All computation/processing tasks triggered by an action node are remote to Oozie. No workflow application specific
computation/processing task is executed within Oozie.
----+++++ 3.2.1.2 Actions Are Asynchronous
+##### 3.2.1.2 Actions Are Asynchronous
All computation/processing tasks triggered by an action node are executed asynchronously by Oozie. For most types of
computation/processing tasks triggered by workflow action, the workflow job has to wait until the
computation/processing task completes before transitioning to the following node in the workflow.
-The exception is the =fs= action that is handled as a synchronous action.
+The exception is the `fs` action that is handled as a synchronous action.
Oozie can detect completion of computation/processing tasks by two different means, callbacks and polling.
@@ -514,19 +549,19 @@
the type of task cannot invoke the callback URL upon completion, Oozie has a mechanism to poll computation/processing
tasks for completion.
----+++++ 3.2.1.3 Actions Have 2 Transitions, =ok= and =error=
+##### 3.2.1.3 Actions Have 2 Transitions, `ok` and `error`
-If a computation/processing task -triggered by a workflow- completes successfully, it transitions to =ok=.
+If a computation/processing task -triggered by a workflow- completes successfully, it transitions to `ok`.
-If a computation/processing task -triggered by a workflow- fails to complete successfully, its transitions to =error=.
+If a computation/processing task -triggered by a workflow- fails to complete successfully, its transitions to `error`.
-If a computation/processing task exits in error, there computation/processing task must provide =error-code= and
- =error-message= information to Oozie. This information can be used from =decision= nodes to implement a fine grain
+If a computation/processing task exits in error, there computation/processing task must provide `error-code` and
+ `error-message` information to Oozie. This information can be used from `decision` nodes to implement a fine grain
error handling at workflow application level.
Each action type must clearly define all the error codes it can produce.
----+++++ 3.2.1.4 Action Recovery
+##### 3.2.1.4 Action Recovery
Oozie provides recovery capabilities when starting or ending actions.
@@ -554,13 +589,13 @@
If the failure is an error and a retry will not resolve the problem, Oozie will perform the error transition for the
action.
-#MapReduceAction
----++++ 3.2.2 Map-Reduce Action
+<a name="MapReduceAction"></a>
+#### 3.2.2 Map-Reduce Action
-The =map-reduce= action starts a Hadoop map/reduce job from a workflow. Hadoop jobs can be Java Map/Reduce jobs or
+The `map-reduce` action starts a Hadoop map/reduce job from a workflow. Hadoop jobs can be Java Map/Reduce jobs or
streaming jobs.
-A =map-reduce= action can be configured to perform file system cleanup and directory creation before starting the
+A `map-reduce` action can be configured to perform file system cleanup and directory creation before starting the
map reduce job. This capability enables Oozie to retry a Hadoop job in the situation of a transient failure (Hadoop
checks the non-existence of the job output directory and then creates it when the Hadoop job is starting, thus a retry
without cleanup of the job output directory would fail).
@@ -568,36 +603,37 @@
The workflow job will wait until the Hadoop map/reduce job completes before continuing to the next action in the
workflow execution path.
-The counters of the Hadoop job and job exit status (=FAILED=, =KILLED= or =SUCCEEDED=) must be available to the
+The counters of the Hadoop job and job exit status (`FAILED`, `KILLED` or `SUCCEEDED`) must be available to the
workflow job after the Hadoop jobs ends. This information can be used from within decision nodes and other actions
configurations.
-The =map-reduce= action has to be configured with all the necessary Hadoop JobConf properties to run the Hadoop
+The `map-reduce` action has to be configured with all the necessary Hadoop JobConf properties to run the Hadoop
map/reduce job.
-Hadoop JobConf properties can be specified as part of
- * the =config-default.xml= or
- * JobConf XML file bundled with the workflow application or
- * <global> tag in workflow definition or
- * Inline =map-reduce= action configuration or
- * An implementation of OozieActionConfigurator specified by the <config-class> tag in workflow definition.
+Hadoop JobConf properties can be specified as part of
-The configuration properties are loaded in the following above order i.e. =streaming=, =job-xml=, =configuration=,
-and =config-class=, and the precedence order is later values override earlier values.
+ * the `config-default.xml` or
+ * JobConf XML file bundled with the workflow application or
+ * \<global\> tag in workflow definition or
+ * Inline `map-reduce` action configuration or
+ * An implementation of OozieActionConfigurator specified by the \<config-class\> tag in workflow definition.
+
+The configuration properties are loaded in the following above order i.e. `streaming`, `job-xml`, `configuration`,
+and `config-class`, and the precedence order is later values override earlier values.
Streaming and inline property values can be parameterized (templatized) using EL expressions.
-The Hadoop =mapred.job.tracker= and =fs.default.name= properties must not be present in the job-xml and inline
+The Hadoop `mapred.job.tracker` and `fs.default.name` properties must not be present in the job-xml and inline
configuration.
-#FilesArchives
----+++++ 3.2.2.1 Adding Files and Archives for the Job
+<a name="FilesArchives"></a>
+##### 3.2.2.1 Adding Files and Archives for the Job
-The =file=, =archive= elements make available, to map-reduce jobs, files and archives. If the specified path is
+The `file`, `archive` elements make available, to map-reduce jobs, files and archives. If the specified path is
relative, it is assumed the file or archiver are within the application directory, in the corresponding sub-path.
If the path is absolute, the file or archive it is expected in the given absolute path.
-Files specified with the =file= element, will be symbolic links in the home directory of the task.
+Files specified with the `file` element, will be symbolic links in the home directory of the task.
If a file is a native library (an '.so' or a '.so.#' file), it will be symlinked as and '.so' file in the task running
directory, thus available to the task JVM.
@@ -607,27 +643,29 @@
Refer to Hadoop distributed cache documentation for details more details on files and archives.
----+++++ 3.2.2.2 Configuring the MapReduce action with Java code
+##### 3.2.2.2 Configuring the MapReduce action with Java code
Java code can be used to further configure the MapReduce action. This can be useful if you already have "driver" code for your
MapReduce action, if you're more familiar with MapReduce's Java API, if there's some configuration that requires logic, or some
configuration that's difficult to do in straight XML (e.g. Avro).
Create a class that implements the org.apache.oozie.action.hadoop.OozieActionConfigurator interface from the "oozie-sharelib-oozie"
-artifact. It contains a single method that receives a =JobConf= as an argument. Any configuration properties set on this =JobConf=
+artifact. It contains a single method that receives a `JobConf` as an argument. Any configuration properties set on this `JobConf`
will be used by the MapReduce action.
The OozieActionConfigurator has this signature:
-<verbatim>
+
+```
public interface OozieActionConfigurator {
public void configure(JobConf actionConf) throws OozieActionConfiguratorException;
}
-</verbatim>
-where =actionConf= is the =JobConf= you can update. If you need to throw an Exception, you can wrap it in
-an =OozieActionConfiguratorException=, also in the "oozie-sharelib-oozie" artifact.
+```
+where `actionConf` is the `JobConf` you can update. If you need to throw an Exception, you can wrap it in
+an `OozieActionConfiguratorException`, also in the "oozie-sharelib-oozie" artifact.
For example:
-<verbatim>
+
+```
package com.example;
import org.apache.hadoop.fs.Path;
@@ -653,11 +691,12 @@
...
}
}
-</verbatim>
+```
To use your config class in your MapReduce action, simply compile it into a jar, make the jar available to your action, and specify
-the class name in the =config-class= element (this requires at least schema 0.5):
-<verbatim>
+the class name in the `config-class` element (this requires at least schema 0.5):
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -679,51 +718,52 @@
</action>
...
</workflow-app>
-</verbatim>
+```
Another example of this can be found in the "map-reduce" example that comes with Oozie.
-A useful tip: The initial =JobConf= passed to the =configure= method includes all of the properties listed in the =configuration=
+A useful tip: The initial `JobConf` passed to the `configure` method includes all of the properties listed in the `configuration`
section of the MR action in a workflow. If you need to pass any information to your OozieActionConfigurator, you can simply put
them here.
-#StreamingMapReduceAction
----+++++ 3.2.2.3 Streaming
+<a name="StreamingMapReduceAction"></a>
+##### 3.2.2.3 Streaming
-Streaming information can be specified in the =streaming= element.
+Streaming information can be specified in the `streaming` element.
-The =mapper= and =reducer= elements are used to specify the executable/script to be used as mapper and reducer.
+The `mapper` and `reducer` elements are used to specify the executable/script to be used as mapper and reducer.
-User defined scripts must be bundled with the workflow application and they must be declared in the =files= element of
-the streaming configuration. If the are not declared in the =files= element of the configuration it is assumed they
+User defined scripts must be bundled with the workflow application and they must be declared in the `files` element of
+the streaming configuration. If the are not declared in the `files` element of the configuration it is assumed they
will be available (and in the command PATH) of the Hadoop slave machines.
Some streaming jobs require Files found on HDFS to be available to the mapper/reducer scripts. This is done using
-the =file= and =archive= elements described in the previous section.
+the `file` and `archive` elements described in the previous section.
-The Mapper/Reducer can be overridden by a =mapred.mapper.class= or =mapred.reducer.class= properties in the =job-xml=
-file or =configuration= elements.
+The Mapper/Reducer can be overridden by a `mapred.mapper.class` or `mapred.reducer.class` properties in the `job-xml`
+file or `configuration` elements.
-#PipesMapReduceAction
----+++++ 3.2.2.4 Pipes
+<a name="PipesMapReduceAction"></a>
+##### 3.2.2.4 Pipes
-Pipes information can be specified in the =pipes= element.
+Pipes information can be specified in the `pipes` element.
A subset of the command line options which can be used while using the Hadoop Pipes Submitter can be specified
-via elements - =map=, =reduce=, =inputformat=, =partitioner=, =writer=, =program=.
+via elements - `map`, `reduce`, `inputformat`, `partitioner`, `writer`, `program`.
-The =program= element is used to specify the executable/script to be used.
+The `program` element is used to specify the executable/script to be used.
User defined program must be bundled with the workflow application.
Some pipe jobs require Files found on HDFS to be available to the mapper/reducer scripts. This is done using
-the =file= and =archive= elements described in the previous section.
+the `file` and `archive` elements described in the previous section.
-Pipe properties can be overridden by specifying them in the =job-xml= file or =configuration= element.
+Pipe properties can be overridden by specifying them in the `job-xml` file or `configuration` element.
----+++++ 3.2.2.5 Syntax
+##### 3.2.2.5 Syntax
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -774,40 +814,41 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-The =prepare= element, if present, indicates a list of paths to delete before starting the job. This should be used
+The `prepare` element, if present, indicates a list of paths to delete before starting the job. This should be used
exclusively for directory cleanup or dropping of hcatalog table or table partitions for the job to be executed. The delete operation
-will be performed in the =fs.default.name= filesystem for hdfs URIs. The format for specifying hcatalog table URI is
+will be performed in the `fs.default.name` filesystem for hdfs URIs. The format for specifying hcatalog table URI is
hcat://[metastore server]:[port]/[database name]/[table name] and format to specify a hcatalog table partition URI is
-hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value].
-In case of a hcatalog URI, the hive-site.xml needs to be shipped using =file= tag and the hcatalog and hive jars
-need to be placed in workflow lib directory or specified using =archive= tag.
+`hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value]`.
+In case of a hcatalog URI, the hive-site.xml needs to be shipped using `file` tag and the hcatalog and hive jars
+need to be placed in workflow lib directory or specified using `archive` tag.
-The =job-xml= element, if present, must refer to a Hadoop JobConf =job.xml= file bundled in the workflow application.
-By default the =job.xml= file is taken from the workflow application namenode, regardless the namenode specified for the action.
-To specify a =job.xml= on another namenode use a fully qualified file path.
-The =job-xml= element is optional and as of schema 0.4, multiple =job-xml= elements are allowed in order to specify multiple Hadoop JobConf =job.xml= files.
+The `job-xml` element, if present, must refer to a Hadoop JobConf `job.xml` file bundled in the workflow application.
+By default the `job.xml` file is taken from the workflow application namenode, regardless the namenode specified for the action.
+To specify a `job.xml` on another namenode use a fully qualified file path.
+The `job-xml` element is optional and as of schema 0.4, multiple `job-xml` elements are allowed in order to specify multiple Hadoop JobConf `job.xml` files.
-The =configuration= element, if present, contains JobConf properties for the Hadoop job.
+The `configuration` element, if present, contains JobConf properties for the Hadoop job.
-Properties specified in the =configuration= element override properties specified in the file specified in the
- =job-xml= element.
+Properties specified in the `configuration` element override properties specified in the file specified in the
+ `job-xml` element.
-As of schema 0.5, the =config-class= element, if present, contains a class that implements OozieActionConfigurator that can be used
+As of schema 0.5, the `config-class` element, if present, contains a class that implements OozieActionConfigurator that can be used
to further configure the MapReduce job.
-Properties specified in the =config-class= class override properties specified in =configuration= element.
+Properties specified in the `config-class` class override properties specified in `configuration` element.
External Stats can be turned on/off by specifying the property _oozie.action.external.stats.write_ as _true_ or _false_ in the configuration element of workflow.xml. The default value for this property is _false_.
-The =file= element, if present, must specify the target symbolic link for binaries by separating the original file and target with a # (file#target-sym-link). This is not required for libraries.
+The `file` element, if present, must specify the target symbolic link for binaries by separating the original file and target with a # (file#target-sym-link). This is not required for libraries.
-The =mapper= and =reducer= process for streaming jobs, should specify the executable command with URL encoding. e.g. '%' should be replaced by '%25'.
+The `mapper` and `reducer` process for streaming jobs, should specify the executable command with URL encoding. e.g. '%' should be replaced by '%25'.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="foo-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="myfirstHadoopJob">
@@ -842,14 +883,15 @@
</action>
...
</workflow-app>
-</verbatim>
+```
In the above example, the number of Reducers to be used by the Map/Reduce job has to be specified as a parameter of
the workflow job configuration when creating the workflow job.
-*Streaming Example:*
+**Streaming Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="firstjob">
@@ -885,12 +927,13 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-*Pipes Example:*
+**Pipes Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="firstjob">
@@ -920,43 +963,45 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-#PigAction
----++++ 3.2.3 Pig Action
+<a name="PigAction"></a>
+#### 3.2.3 Pig Action
-The =pig= action starts a Pig job.
+The `pig` action starts a Pig job.
The workflow job will wait until the pig job completes before continuing to the next action.
-The =pig= action has to be configured with the resource-manager, name-node, pig script and the necessary parameters and
+The `pig` action has to be configured with the resource-manager, name-node, pig script and the necessary parameters and
configuration to run the Pig job.
-A =pig= action can be configured to perform HDFS files/directories cleanup or HCatalog partitions cleanup before
+A `pig` action can be configured to perform HDFS files/directories cleanup or HCatalog partitions cleanup before
starting the Pig job. This capability enables Oozie to retry a Pig job in the situation of a transient failure (Pig
creates temporary directories for intermediate data, thus a retry without cleanup would fail).
-Hadoop JobConf properties can be specified as part of
- * the =config-default.xml= or
- * JobConf XML file bundled with the workflow application or
- * <global> tag in workflow definition or
- * Inline =pig= action configuration.
+Hadoop JobConf properties can be specified as part of
-The configuration properties are loaded in the following above order i.e. =job-xml= and =configuration=, and
+ * the `config-default.xml` or
+ * JobConf XML file bundled with the workflow application or
+ * \<global\> tag in workflow definition or
+ * Inline `pig` action configuration.
+
+The configuration properties are loaded in the following above order i.e. `job-xml` and `configuration`, and
the precedence order is later values override earlier values.
Inline property values can be parameterized (templatized) using EL expressions.
-The YARN =yarn.resourcemanager.address= and HDFS =fs.default.name= properties must not be present in the job-xml and inline
+The YARN `yarn.resourcemanager.address` and HDFS `fs.default.name` properties must not be present in the job-xml and inline
configuration.
As with Hadoop map-reduce jobs, it is possible to add files and archives to be available to the Pig job, refer to
section [#FilesArchives][Adding Files and Archives for the Job].
-*Syntax for Pig actions in Oozie schema 1.0:*
-<verbatim>
+**Syntax for Pig actions in Oozie schema 1.0:**
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -994,10 +1039,11 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-*Syntax for Pig actions in Oozie schema 0.2:*
-<verbatim>
+**Syntax for Pig actions in Oozie schema 0.2:**
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:0.2">
...
<action name="[NODE-NAME]">
@@ -1035,11 +1081,12 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-*Syntax for Pig actions in Oozie schema 0.1:*
+**Syntax for Pig actions in Oozie schema 0.1:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:0.1">
...
<action name="[NODE-NAME]">
@@ -1074,46 +1121,47 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-The =prepare= element, if present, indicates a list of paths to delete before starting the job. This should be used
+The `prepare` element, if present, indicates a list of paths to delete before starting the job. This should be used
exclusively for directory cleanup or dropping of hcatalog table or table partitions for the job to be executed. The delete operation
-will be performed in the =fs.default.name= filesystem for hdfs URIs. The format for specifying hcatalog table URI is
+will be performed in the `fs.default.name` filesystem for hdfs URIs. The format for specifying hcatalog table URI is
hcat://[metastore server]:[port]/[database name]/[table name] and format to specify a hcatalog table partition URI is
-hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value].
-In case of a hcatalog URI, the hive-site.xml needs to be shipped using =file= tag and the hcatalog and hive jars
-need to be placed in workflow lib directory or specified using =archive= tag.
+`hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value]`.
+In case of a hcatalog URI, the hive-site.xml needs to be shipped using `file` tag and the hcatalog and hive jars
+need to be placed in workflow lib directory or specified using `archive` tag.
-The =job-xml= element, if present, must refer to a Hadoop JobConf =job.xml= file bundled in the workflow application.
-The =job-xml= element is optional and as of schema 0.4, multiple =job-xml= elements are allowed in order to specify multiple Hadoop JobConf =job.xml= files.
+The `job-xml` element, if present, must refer to a Hadoop JobConf `job.xml` file bundled in the workflow application.
+The `job-xml` element is optional and as of schema 0.4, multiple `job-xml` elements are allowed in order to specify multiple Hadoop JobConf `job.xml` files.
-The =configuration= element, if present, contains JobConf properties for the underlying Hadoop jobs.
+The `configuration` element, if present, contains JobConf properties for the underlying Hadoop jobs.
-Properties specified in the =configuration= element override properties specified in the file specified in the
- =job-xml= element.
+Properties specified in the `configuration` element override properties specified in the file specified in the
+ `job-xml` element.
External Stats can be turned on/off by specifying the property _oozie.action.external.stats.write_ as _true_ or _false_ in the configuration element of workflow.xml. The default value for this property is _false_.
The inline and job-xml configuration properties are passed to the Hadoop jobs submitted by Pig runtime.
-The =script= element contains the pig script to execute. The pig script can be templatized with variables of the
-form =${VARIABLE}=. The values of these variables can then be specified using the =params= element.
+The `script` element contains the pig script to execute. The pig script can be templatized with variables of the
+form `${VARIABLE}`. The values of these variables can then be specified using the `params` element.
NOTE: Oozie will perform the parameter substitution before firing the pig job. This is different from the
-[[http://wiki.apache.org/pig/ParameterSubstitution][parameter substitution mechanism provided by Pig]], which has a
+[parameter substitution mechanism provided by Pig](http://wiki.apache.org/pig/ParameterSubstitution), which has a
few limitations.
-The =params= element, if present, contains parameters to be passed to the pig script.
+The `params` element, if present, contains parameters to be passed to the pig script.
-*In Oozie schema 0.2:*
-The =arguments= element, if present, contains arguments to be passed to the pig script.
+**In Oozie schema 0.2:**
+The `arguments` element, if present, contains arguments to be passed to the pig script.
All the above elements can be parameterized (templatized) using EL expressions.
-*Example for Oozie schema 0.2:*
+**Example for Oozie schema 0.2:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:0.2">
...
<action name="myfirstpigjob">
@@ -1144,12 +1192,13 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-*Example for Oozie schema 0.1:*
+**Example for Oozie schema 0.1:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:0.1">
...
<action name="myfirstpigjob">
@@ -1174,35 +1223,36 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-#FsAction
----++++ 3.2.4 Fs (HDFS) action
+<a name="FsAction"></a>
+#### 3.2.4 Fs (HDFS) action
-The =fs= action allows to manipulate files and directories in HDFS from a workflow application. The supported commands
-are =move=, =delete=, =mkdir=, =chmod=, =touchz=, =setrep= and =chgrp=.
+The `fs` action allows to manipulate files and directories in HDFS from a workflow application. The supported commands
+are `move`, `delete`, `mkdir`, `chmod`, `touchz`, `setrep` and `chgrp`.
The FS commands are executed synchronously from within the FS action, the workflow job will wait until the specified
file commands are completed before continuing to the next action.
-Path names specified in the =fs= action can be parameterized (templatized) using EL expressions.
-Path name should be specified as a absolute path. In case of =move=, =delete=, =chmod= and =chgrp= commands, a glob pattern can also be specified instead of an absolute path.
-For =move=, glob pattern can only be specified for source path and not the target.
+Path names specified in the `fs` action can be parameterized (templatized) using EL expressions.
+Path name should be specified as a absolute path. In case of `move`, `delete`, `chmod` and `chgrp` commands, a glob pattern can also be specified instead of an absolute path.
+For `move`, glob pattern can only be specified for source path and not the target.
Each file path must specify the file system URI, for move operations, the target must not specify the system URI.
-*IMPORTANT:* For the purposes of copying files within a cluster it is recommended to refer to the =distcp= action
-instead. Refer to [[DG_DistCpActionExtension][=distcp=]] action to copy files within a cluster.
+**IMPORTANT:** For the purposes of copying files within a cluster it is recommended to refer to the `distcp` action
+instead. Refer to [`distcp`](DG_DistCpActionExtension.html) action to copy files within a cluster.
-*IMPORTANT:* All the commands within =fs= action do not happen atomically, if a =fs= action fails half way in the
-commands being executed, successfully executed commands are not rolled back. The =fs= action, before executing any
-command must check that source paths exist and target paths don't exist (constraint regarding target relaxed for the =move= action. See below for details), thus failing before executing any command.
-Therefore the validity of all paths specified in one =fs= action are evaluated before any of the file operation are
-executed. Thus there is less chance of an error occurring while the =fs= action executes.
+**IMPORTANT:** All the commands within `fs` action do not happen atomically, if a `fs` action fails half way in the
+commands being executed, successfully executed commands are not rolled back. The `fs` action, before executing any
+command must check that source paths exist and target paths don't exist (constraint regarding target relaxed for the `move` action. See below for details), thus failing before executing any command.
+Therefore the validity of all paths specified in one `fs` action are evaluated before any of the file operation are
+executed. Thus there is less chance of an error occurring while the `fs` action executes.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -1226,50 +1276,52 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-The =delete= command deletes the specified path, if it is a directory it deletes recursively all its content and then
+The `delete` command deletes the specified path, if it is a directory it deletes recursively all its content and then
deletes the directory. By default it does skip trash. It can be moved to trash by setting the value of skip-trash to
'false'. It can also be used to drop hcat tables/partitions. This is the only FS command which supports HCatalog URIs as well.
For eg:
-<verbatim>
+
+```
<delete path='hcat://[metastore server]:[port]/[database name]/[table name]'/>
OR
<delete path='hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value];...'/>
-</verbatim>
+```
-The =mkdir= command creates the specified directory, it creates all missing directories in the path. If the directory
+The `mkdir` command creates the specified directory, it creates all missing directories in the path. If the directory
already exist it does a no-op.
-In the =move= command the =source= path must exist. The following scenarios are addressed for a =move=:
+In the `move` command the `source` path must exist. The following scenarios are addressed for a `move`:
- * The file system URI(e.g. hdfs://{nameNode}) can be skipped in the =target= path. It is understood to be the same as that of the source. But if the target path does contain the system URI, it cannot be different than that of the source.
- * The parent directory of the =target= path must exist
- * For the =target= path, if it is a file, then it must not already exist.
- * However, if the =target= path is an already existing directory, the =move= action will place your =source= as a child of the =target= directory.
+ * The file system URI(e.g. `hdfs://{nameNode}`) can be skipped in the `target` path. It is understood to be the same as that of the source. But if the target path does contain the system URI, it cannot be different than that of the source.
+ * The parent directory of the `target` path must exist
+ * For the `target` path, if it is a file, then it must not already exist.
+ * However, if the `target` path is an already existing directory, the `move` action will place your `source` as a child of the `target` directory.
-The =chmod= command changes the permissions for the specified path. Permissions can be specified using the Unix Symbolic
+The `chmod` command changes the permissions for the specified path. Permissions can be specified using the Unix Symbolic
representation (e.g. -rwxrw-rw-) or an octal representation (755).
-When doing a =chmod= command on a directory, by default the command is applied to the directory and the files one level
-within the directory. To apply the =chmod= command to the directory, without affecting the files within it,
-the =dir-files= attribute must be set to =false=. To apply the =chmod= command
-recursively to all levels within a directory, put a =recursive= element inside the <chmod> element.
+When doing a `chmod` command on a directory, by default the command is applied to the directory and the files one level
+within the directory. To apply the `chmod` command to the directory, without affecting the files within it,
+the `dir-files` attribute must be set to `false`. To apply the `chmod` command
+recursively to all levels within a directory, put a `recursive` element inside the \<chmod\> element.
-The =touchz= command creates a zero length file in the specified path if none exists. If one already exists, then touchz will perform a touch operation.
+The `touchz` command creates a zero length file in the specified path if none exists. If one already exists, then touchz will perform a touch operation.
Touchz works only for absolute paths.
-The =chgrp= command changes the group for the specified path.
-When doing a =chgrp= command on a directory, by default the command is applied to the directory and the files one level
-within the directory. To apply the =chgrp= command to the directory, without affecting the files within it,
-the =dir-files= attribute must be set to =false=.
-To apply the =chgrp= command recursively to all levels within a directory, put a =recursive= element inside the <chgrp> element.
+The `chgrp` command changes the group for the specified path.
+When doing a `chgrp` command on a directory, by default the command is applied to the directory and the files one level
+within the directory. To apply the `chgrp` command to the directory, without affecting the files within it,
+the `dir-files` attribute must be set to `false`.
+To apply the `chgrp` command recursively to all levels within a directory, put a `recursive` element inside the \<chgrp\> element.
-The =setrep= command changes replication factor of an hdfs file(s). Changing RF of directories or symlinks is not
+The `setrep` command changes replication factor of an hdfs file(s). Changing RF of directories or symlinks is not
supported; this action requires an argument for RF.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="hdfscommands">
@@ -1286,25 +1338,26 @@
</action>
...
</workflow-app>
-</verbatim>
+```
In the above example, a directory named after the workflow job ID is created and the input of the job, passed as
workflow configuration parameter, is archived under the previously created directory.
-As of schema 0.4, if a =name-node= element is specified, then it is not necessary for any of the paths to start with the file system
-URI as it is taken from the =name-node= element. This is also true if the name-node is specified in the global section
-(see [[WorkflowFunctionalSpec#GlobalConfigurations][Global Configurations]])
+As of schema 0.4, if a `name-node` element is specified, then it is not necessary for any of the paths to start with the file system
+URI as it is taken from the `name-node` element. This is also true if the name-node is specified in the global section
+(see [Global Configurations](WorkflowFunctionalSpec.html#GlobalConfigurations))
-As of schema 0.4, zero or more =job-xml= elements can be specified; these must refer to Hadoop JobConf =job.xml= formatted files
+As of schema 0.4, zero or more `job-xml` elements can be specified; these must refer to Hadoop JobConf `job.xml` formatted files
bundled in the workflow application. They can be used to set additional properties for the FileSystem instance.
-As of schema 0.4, if a =configuration= element is specified, then it will also be used to set additional JobConf properties for the
-FileSystem instance. Properties specified in the =configuration= element override properties specified in the files specified
-by any =job-xml= elements.
+As of schema 0.4, if a `configuration` element is specified, then it will also be used to set additional JobConf properties for the
+FileSystem instance. Properties specified in the `configuration` element override properties specified in the files specified
+by any `job-xml` elements.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:0.4">
...
<action name="hdfscommands">
@@ -1324,21 +1377,22 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-#SubWorkflowAction
----++++ 3.2.5 Sub-workflow Action
+<a name="SubWorkflowAction"></a>
+#### 3.2.5 Sub-workflow Action
-The =sub-workflow= action runs a child workflow job, the child workflow job can be in the same Oozie system or in
+The `sub-workflow` action runs a child workflow job, the child workflow job can be in the same Oozie system or in
another Oozie system.
The parent workflow job will wait until the child workflow job has completed.
There can be several sub-workflows defined within a single workflow, each under its own action element.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -1358,22 +1412,23 @@
</action>
...
</workflow-app>
-</verbatim>
+```
The child workflow job runs in the same Oozie system instance where the parent workflow job is running.
-The =app-path= element specifies the path to the workflow application of the child workflow job.
+The `app-path` element specifies the path to the workflow application of the child workflow job.
-The =propagate-configuration= flag, if present, indicates that the workflow job configuration should be propagated to
+The `propagate-configuration` flag, if present, indicates that the workflow job configuration should be propagated to
the child workflow.
-The =configuration= section can be used to specify the job properties that are required to run the child workflow job.
+The `configuration` section can be used to specify the job properties that are required to run the child workflow job.
-The configuration of the =sub-workflow= action can be parameterized (templatized) using EL expressions.
+The configuration of the `sub-workflow` action can be parameterized (templatized) using EL expressions.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="a">
@@ -1391,94 +1446,97 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-In the above example, the workflow definition with the name =child-wf= will be run on the Oozie instance at
- =.http://myhost:11000/oozie=. The specified workflow application must be already deployed on the target Oozie instance.
+In the above example, the workflow definition with the name `child-wf` will be run on the Oozie instance at
+ `.http://myhost:11000/oozie`. The specified workflow application must be already deployed on the target Oozie instance.
-A configuration parameter =input.dir= is being passed as job property to the child workflow job.
+A configuration parameter `input.dir` is being passed as job property to the child workflow job.
-The subworkflow can inherit the lib jars from the parent workflow by setting =oozie.subworkflow.classpath.inheritance= to true
-in oozie-site.xml or on a per-job basis by setting =oozie.wf.subworkflow.classpath.inheritance= to true in a job.properties file.
-If both are specified, =oozie.wf.subworkflow.classpath.inheritance= has priority. If the subworkflow and the parent have
-conflicting jars, the subworkflow's jar has priority. By default, =oozie.wf.subworkflow.classpath.inheritance= is set to false.
+The subworkflow can inherit the lib jars from the parent workflow by setting `oozie.subworkflow.classpath.inheritance` to true
+in oozie-site.xml or on a per-job basis by setting `oozie.wf.subworkflow.classpath.inheritance` to true in a job.properties file.
+If both are specified, `oozie.wf.subworkflow.classpath.inheritance` has priority. If the subworkflow and the parent have
+conflicting jars, the subworkflow's jar has priority. By default, `oozie.wf.subworkflow.classpath.inheritance` is set to false.
-To prevent errant workflows from starting infinitely recursive subworkflows, =oozie.action.subworkflow.max.depth= can be specified
+To prevent errant workflows from starting infinitely recursive subworkflows, `oozie.action.subworkflow.max.depth` can be specified
in oozie-site.xml to set the maximum depth of subworkflow calls. For example, if set to 3, then a workflow can start subwf1, which
can start subwf2, which can start subwf3; but if subwf3 tries to start subwf4, then the action will fail. The default is 50.
-#JavaAction
----++++ 3.2.6 Java Action
+<a name="JavaAction"></a>
+#### 3.2.6 Java Action
-The =java= action will execute the =public static void main(String[] args)= method of the specified main Java class.
+The `java` action will execute the `public static void main(String[] args)` method of the specified main Java class.
Java applications are executed in the Hadoop cluster as map-reduce job with a single Mapper task.
The workflow job will wait until the java application completes its execution before continuing to the next action.
-The =java= action has to be configured with the resource-manager, name-node, main Java class, JVM options and arguments.
+The `java` action has to be configured with the resource-manager, name-node, main Java class, JVM options and arguments.
-To indicate an =ok= action transition, the main Java class must complete gracefully the =main= method invocation.
+To indicate an `ok` action transition, the main Java class must complete gracefully the `main` method invocation.
-To indicate an =error= action transition, the main Java class must throw an exception.
+To indicate an `error` action transition, the main Java class must throw an exception.
-The main Java class can call =System.exit(int n)=. Exit code zero is regarded as OK, while non-zero exit codes will
-cause the =java= action to do an =error= transition and exit.
+The main Java class can call `System.exit(int n)`. Exit code zero is regarded as OK, while non-zero exit codes will
+cause the `java` action to do an `error` transition and exit.
-A =java= action can be configured to perform HDFS files/directories cleanup or HCatalog partitions cleanup before
+A `java` action can be configured to perform HDFS files/directories cleanup or HCatalog partitions cleanup before
starting the Java application. This capability enables Oozie to retry a Java application in the situation of a transient
or non-transient failure (This can be used to cleanup any temporary data which may have been created by the Java
application in case of failure).
-A =java= action can create a Hadoop configuration for interacting with a cluster (e.g. launching a map-reduce job).
-Oozie prepares a Hadoop configuration file which includes the environments site configuration files (e.g. hdfs-site.xml,
-mapred-site.xml, etc) plus the properties added to the =<configuration>= section of the =java= action. The Hadoop configuration
-file is made available as a local file to the Java application in its running directory. It can be added to the =java= actions
-Hadoop configuration by referencing the system property: =oozie.action.conf.xml=. For example:
+A `java` action can create a Hadoop configuration for interacting with a cluster (e.g. launching a map-reduce job).
+Oozie prepares a Hadoop configuration file which includes the environments site configuration files (e.g. hdfs-site.xml,
+mapred-site.xml, etc) plus the properties added to the `<configuration>` section of the `java` action. The Hadoop configuration
+file is made available as a local file to the Java application in its running directory. It can be added to the `java` actions
+Hadoop configuration by referencing the system property: `oozie.action.conf.xml`. For example:
-<verbatim>
+
+```
// loading action conf prepared by Oozie
Configuration actionConf = new Configuration(false);
actionConf.addResource(new Path("file:///", System.getProperty("oozie.action.conf.xml")));
-</verbatim>
+```
-If =oozie.action.conf.xml= is not added then the job will pick up the mapred-default properties and this may result
+If `oozie.action.conf.xml` is not added then the job will pick up the mapred-default properties and this may result
in unexpected behaviour. For repeated configuration properties later values override earlier ones.
Inline property values can be parameterized (templatized) using EL expressions.
-The YARN =yarn.resourcemanager.address= (=resource-manager=) and HDFS =fs.default.name= (=name-node=) properties must not be present
-in the =job-xml= and in the inline configuration.
+The YARN `yarn.resourcemanager.address` (`resource-manager`) and HDFS `fs.default.name` (`name-node`) properties must not be present
+in the `job-xml` and in the inline configuration.
-As with =map-reduce= and =pig= actions, it is possible to add files and archives to be available to the Java
+As with `map-reduce` and `pig` actions, it is possible to add files and archives to be available to the Java
application. Refer to section [#FilesArchives][Adding Files and Archives for the Job].
-The =capture-output= element can be used to propagate values back into Oozie context, which can then be accessed via
-EL-functions. This needs to be written out as a java properties format file. The filename is obtained via a System
-property specified by the constant =oozie.action.output.properties=
+The `capture-output` element can be used to propagate values back into Oozie context, which can then be accessed via
+EL-functions. This needs to be written out as a java properties format file. The filename is obtained via a System
+property specified by the constant `oozie.action.output.properties`
-*IMPORTANT:* In order for a Java action to succeed on a secure cluster, it must propagate the Hadoop delegation token like in the
+**IMPORTANT:** In order for a Java action to succeed on a secure cluster, it must propagate the Hadoop delegation token like in the
following code snippet (this is benign on non-secure clusters):
-<verbatim>
+
+```
// propagate delegation related props from launcher job to MR job
if (System.getenv("HADOOP_TOKEN_FILE_LOCATION") != null) {
jobConf.set("mapreduce.job.credentials.binary", System.getenv("HADOOP_TOKEN_FILE_LOCATION"));
}
-</verbatim>
+```
-*IMPORTANT:* Because the Java application is run from within a Map-Reduce job, from Hadoop 0.20. onwards a queue must
+**IMPORTANT:** Because the Java application is run from within a Map-Reduce job, from Hadoop 0.20. onwards a queue must
be assigned to it. The queue name must be specified as a configuration property.
-*IMPORTANT:* The Java application from a Java action is executed in a single map task. If the task is abnormally terminated,
+**IMPORTANT:** The Java application from a Java action is executed in a single map task. If the task is abnormally terminated,
such as due to a TaskTracker restart (e.g. during cluster maintenance), the task will be retried via the normal Hadoop task
retry mechanism. To avoid workflow failure, the application should be written in a fashion that is resilient to such retries,
for example by detecting and deleting incomplete outputs or picking back up from complete outputs. Furthermore, if a Java action
spawns asynchronous activity outside the JVM of the action itself (such as by launching additional MapReduce jobs), the
application must consider the possibility of collisions with activity spawned by the new instance.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -1514,41 +1572,43 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-The =prepare= element, if present, indicates a list of paths to delete before starting the Java application. This should
+The `prepare` element, if present, indicates a list of paths to delete before starting the Java application. This should
be used exclusively for directory cleanup or dropping of hcatalog table or table partitions for the Java application to be executed.
-In case of =delete=, a glob pattern can be used to specify path.
+In case of `delete`, a glob pattern can be used to specify path.
The format for specifying hcatalog table URI is
hcat://[metastore server]:[port]/[database name]/[table name] and format to specify a hcatalog table partition URI is
-hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value].
-In case of a hcatalog URI, the hive-site.xml needs to be shipped using =file= tag and the hcatalog and hive jars
-need to be placed in workflow lib directory or specified using =archive= tag.
+`hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value]`.
+In case of a hcatalog URI, the hive-site.xml needs to be shipped using `file` tag and the hcatalog and hive jars
+need to be placed in workflow lib directory or specified using `archive` tag.
-The =java-opts= and =java-opt= elements, if present, contains the command line parameters which are to be used to start the JVM that
-will execute the Java application. Using this element is equivalent to using the =mapred.child.java.opts=
-or =mapreduce.map.java.opts= configuration properties, with the advantage that Oozie will append to these properties instead of
+The `java-opts` and `java-opt` elements, if present, contains the command line parameters which are to be used to start the JVM that
+will execute the Java application. Using this element is equivalent to using the `mapred.child.java.opts`
+or `mapreduce.map.java.opts` configuration properties, with the advantage that Oozie will append to these properties instead of
simply setting them (e.g. if you have one of these properties specified in mapred-site.xml, setting it again in
-the =configuration= element will override it, but using =java-opts= or =java-opt= will instead append to it, preserving the original
-value). You can have either one =java-opts=, multiple =java-opt=, or neither; you cannot use both at the same time. In any case,
-Oozie will set both =mapred.child.java.opts= and =mapreduce.map.java.opts= to the same value based on combining them. In other
+the `configuration` element will override it, but using `java-opts` or `java-opt` will instead append to it, preserving the original
+value). You can have either one `java-opts`, multiple `java-opt`, or neither; you cannot use both at the same time. In any case,
+Oozie will set both `mapred.child.java.opts` and `mapreduce.map.java.opts` to the same value based on combining them. In other
words, after Oozie is finished:
-<verbatim>
+
+```
mapred.child.java.opts <-- "<mapred.child.java.opts> <mapreduce.map.java.opts> <java-opt...|java-opts>"
mapreduce.map.java.opts <-- "<mapred.child.java.opts> <mapreduce.map.java.opts> <java-opt...|java-opts>"
-</verbatim>
-In the case that parameters are repeated, the latest instance of the parameter is used by Java. This means that =java-opt=
-(or =java-opts=) has the highest priority, followed by =mapreduce.map.java.opts=, and finally =mapred.child.java.opts=. When
-multiple =java-opt= are specified, they are included from top to bottom ordering, where the bottom has highest priority.
+```
+In the case that parameters are repeated, the latest instance of the parameter is used by Java. This means that `java-opt`
+(or `java-opts`) has the highest priority, followed by `mapreduce.map.java.opts`, and finally `mapred.child.java.opts`. When
+multiple `java-opt` are specified, they are included from top to bottom ordering, where the bottom has highest priority.
-The =arg= elements, if present, contains arguments for the main function. The value of each =arg= element is considered
-a single argument and they are passed to the =main= method in the same order.
+The `arg` elements, if present, contains arguments for the main function. The value of each `arg` element is considered
+a single argument and they are passed to the `main` method in the same order.
All the above elements can be parameterized (templatized) using EL expressions.
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:1.0">
...
<action name="myfirstjavajob">
@@ -1574,43 +1634,45 @@
</action>
...
</workflow-app>
-</verbatim>
+```
----+++++ 3.2.6.1 Overriding an action's Main class
+##### 3.2.6.1 Overriding an action's Main class
This feature is useful for developers to change the Main classes without having to recompile or redeploy Oozie.
-For most actions (not just the Java action), you can override the Main class it uses by specifying the following =configuration=
+For most actions (not just the Java action), you can override the Main class it uses by specifying the following `configuration`
property and making sure that your class is included in the workflow's classpath. If you specify this in the Java action,
the main-class element has priority.
-<verbatim>
+
+```
<property>
<name>oozie.launcher.action.main.class</name>
<value>org.my.CustomMain</value>
</property>
-</verbatim>
+```
-*Note:* Most actions typically pass information to their corresponding Main in specific ways; you should look at the action's
+**Note:** Most actions typically pass information to their corresponding Main in specific ways; you should look at the action's
existing Main to see how it works before creating your own. In fact, its probably simplest to just subclass the existing Main and
add/modify/overwrite any behavior you want to change.
-#GitAction
----++++ 3.2.7 Git action
+<a name="GitAction"></a>
+#### 3.2.7 Git action
-The =git= action allows one to clone a Git repository into HDFS. The supported options are =git-uri=, =branch=, =key-path=
-and =destination-uri=.
+The `git` action allows one to clone a Git repository into HDFS. The supported options are `git-uri`, `branch`, `key-path`
+and `destination-uri`.
-The =git clone= action is executed asynchronously by one of the YARN containers assigned to run on the cluster. If an SSH key is
+The `git clone` action is executed asynchronously by one of the YARN containers assigned to run on the cluster. If an SSH key is
specified it will be created on the file system in a YARN container's local directory, relying on YARN NodeManager to remove the
file after the action has run.
-Path names specified in the =git= action should be able to be parameterized (templatized) using EL expressions,
-e.g. =${wf:user()}= . Path name should be specified as an absolute path. Each file path must specify the file system URI.
+Path names specified in the `git` action should be able to be parameterized (templatized) using EL expressions,
+e.g. `${wf:user()}` . Path name should be specified as an absolute path. Each file path must specify the file system URI.
-*Syntax:*
+**Syntax:**
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -1628,11 +1690,12 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-*Example:*
+**Example:**
-<verbatim>
+
+```
<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:0.1">
...
<action name="clone_oozie">
@@ -1645,20 +1708,21 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-In the above example, a Git repository on e.g. GitHub.com is cloned to the HDFS directory =my_git_repo_directory= which should not
+In the above example, a Git repository on e.g. GitHub.com is cloned to the HDFS directory `my_git_repo_directory` which should not
exist previously on the filesystem. Note that repository addresses outside of GitHub.com but accessible to the YARN container
running the Git action may also be used.
-If a =name-node= element is specified, then it is not necessary for any of the paths to start with the file system URI as it is
-taken from the =name-node= element.
+If a `name-node` element is specified, then it is not necessary for any of the paths to start with the file system URI as it is
+taken from the `name-node` element.
-The =resource-manager= (Oozie 5.x) element has to be specified to name the YARN ResourceManager address.
+The `resource-manager` (Oozie 5.x) element has to be specified to name the YARN ResourceManager address.
If any of the paths need to be served from another HDFS namenode, its address has to be part of
that filesystem URI prefix:
-<verbatim>
+
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -1673,23 +1737,24 @@
</action>
...
</workflow-app>
-</verbatim>
+```
This is also true if the name-node is specified in the global section (see
-[[WorkflowFunctionalSpec#GlobalConfigurations][Global Configurations]]).
+[Global Configurations](WorkflowFunctionalSpec.html#GlobalConfigurations)).
-Be aware that =key-path= might point to a secure object store location other than the current =fs.defaultFS=. In that case,
+Be aware that `key-path` might point to a secure object store location other than the current `fs.defaultFS`. In that case,
appropriate file permissions are still necessary (readable by submitting user), credentials provided, etc.
-As of workflow schema 1.0, zero or more =job-xml= elements can be specified; these must refer to Hadoop JobConf =job.xml= formatted
-files bundled in the workflow application. They can be used to set additional properties for the =FileSystem= instance.
+As of workflow schema 1.0, zero or more `job-xml` elements can be specified; these must refer to Hadoop JobConf `job.xml` formatted
+files bundled in the workflow application. They can be used to set additional properties for the `FileSystem` instance.
-As of schema workflow schema 1.0, if a =configuration= element is specified, then it will also be used to set additional =JobConf=
-properties for the =FileSystem= instance. Properties specified in the =configuration= element are overridden by properties
-specified in the files specified by any =job-xml= elements.
+As of schema workflow schema 1.0, if a `configuration` element is specified, then it will also be used to set additional `JobConf`
+properties for the `FileSystem` instance. Properties specified in the `configuration` element are overridden by properties
+specified in the files specified by any `job-xml` elements.
-*Example:*
+**Example:**
+```
<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
...
<action name="[NODE-NAME]">
@@ -1708,49 +1773,50 @@
</action>
...
</workflow>
-</verbatim>
+```
-#WorkflowParameterization
----++ 4 Parameterization of Workflows
+<a name="WorkflowParameterization"></a>
+## 4 Parameterization of Workflows
Workflow definitions can be parameterized.
When workflow node is executed by Oozie all the ELs are resolved into concrete values.
The parameterization of workflow definitions it done using JSP Expression Language syntax from the
-[[http://jcp.org/aboutJava/communityprocess/final/jsr152/][JSP 2.0 Specification (JSP.2.3)]], allowing not only to
+[JSP 2.0 Specification (JSP.2.3)](http://jcp.org/aboutJava/communityprocess/final/jsr152/index.html), allowing not only to
support variables as parameters but also functions and complex expressions.
EL expressions can be used in the configuration values of action and decision nodes. They can be used in XML attribute
values and in XML element and attribute values.
They cannot be used in XML element and attribute names. They cannot be used in the name of a node and they cannot be
-used within the =transition= elements of a node.
+used within the `transition` elements of a node.
-#WorkflowProperties
----+++ 4.1 Workflow Job Properties (or Parameters)
+<a name="WorkflowProperties"></a>
+### 4.1 Workflow Job Properties (or Parameters)
When a workflow job is submitted to Oozie, the submitter may specify as many workflow job properties as required
(similar to Hadoop JobConf properties).
Workflow applications may define default values for the workflow job or action parameters. They must be defined in a
- =config-default.xml= file bundled with the workflow application archive (refer to section '7 Workflow
+ `config-default.xml` file bundled with the workflow application archive (refer to section '7 Workflow
Applications Packaging'). Job or action properties specified in the workflow definition have precedence over the default values.
-Properties that are a valid Java identifier, <code>[A-Za-z_][0-9A-Za-z_]*</code>, are available as '${NAME}'
+Properties that are a valid Java identifier, `[A-Za-z_][0-9A-Za-z_]*`, are available as '${NAME}'
variables within the workflow definition.
-Properties that are not valid Java Identifier, for example 'job.tracker', are available via the =String
-wf:conf(String name)= function. Valid identifier properties are available via this function as well.
+Properties that are not valid Java Identifier, for example 'job.tracker', are available via the `String
+wf:conf(String name)` function. Valid identifier properties are available via this function as well.
Using properties that are valid Java identifiers result in a more readable and compact definition.
By using properties
-*Example:*
+**Example:**
Parameterized Workflow definition:
-<verbatim>
+
+```
<workflow-app name='hello-wf' xmlns="uri:oozie:workflow:1.0">
...
<action name='firstjob'>
@@ -1781,24 +1847,25 @@
</action>
...
</workflow-app>
-</verbatim>
+```
When submitting a workflow job for the workflow definition above, 3 workflow job properties must be specified:
- * =resourceManager:=
- * =inputDir:=
- * =outputDir:=
+ * `resourceManager:`
+ * `inputDir:`
+ * `outputDir:`
-If the above 3 properties are not specified, the job will fail.
+If the above 3 properties are not specified, the job will fail.
-As of schema 0.4, a list of formal parameters can be provided which will allow Oozie to verify, at submission time, that said
-properties are actually specified (i.e. before the job is executed and fails). Default values can also be provided.
+As of schema 0.4, a list of formal parameters can be provided which will allow Oozie to verify, at submission time, that said
+properties are actually specified (i.e. before the job is executed and fails). Default values can also be provided.
-*Example:*
+**Example:**
The previous parameterized workflow definition with formal parameters:
-<verbatim>
+
+```
<workflow-app name='hello-wf' xmlns="uri:oozie:workflow:1.0">
<parameters>
<property>
@@ -1838,177 +1905,177 @@
</action>
...
</workflow-app>
-</verbatim>
+```
-In the above example, if =inputDir= is not specified, Oozie will print an error message instead of submitting the job. If
-=outputDir= is not specified, Oozie will use the default value, =out-dir=.
+In the above example, if `inputDir` is not specified, Oozie will print an error message instead of submitting the job. If
+`outputDir` is not specified, Oozie will use the default value, `out-dir`.
-#WorkflowELSupport
----+++ 4.2 Expression Language Functions
+<a name="WorkflowELSupport"></a>
+### 4.2 Expression Language Functions
Oozie, besides allowing the use of workflow job properties to parameterize workflow jobs, it provides a set of build
in EL functions that enable a more complex parameterization of workflow action nodes as well as the predicates in
decision nodes.
----++++ 4.2.1 Basic EL Constants
+#### 4.2.1 Basic EL Constants
- * *KB:* 1024, one kilobyte.
- * *MB:* 1024 * KB, one megabyte.
- * *GB:* 1024 * MB, one gigabyte.
- * *TB:* 1024 * GB, one terabyte.
- * *PB:* 1024 * TG, one petabyte.
+ * **KB:** 1024, one kilobyte.
+ * **MB:** 1024 *** KB, one megabyte.
+ * **GB:** 1024 *** MB, one gigabyte.
+ * **TB:** 1024 *** GB, one terabyte.
+ * **PB:** 1024 *** TG, one petabyte.
-All the above constants are of type =long=.
+All the above constants are of type `long`.
----++++ 4.2.2 Basic EL Functions
+#### 4.2.2 Basic EL Functions
-*String firstNotNull(String value1, String value2)*
+**String firstNotNull(String value1, String value2)**
-It returns the first not =null= value, or =null= if both are =null=.
+It returns the first not `null` value, or `null` if both are `null`.
-Note that if the output of this function is =null= and it is used as string, the EL library converts it to an empty
-string. This is the common behavior when using =firstNotNull()= in node configuration sections.
+Note that if the output of this function is `null` and it is used as string, the EL library converts it to an empty
+string. This is the common behavior when using `firstNotNull()` in node configuration sections.
-*String concat(String s1, String s2)*
+**String concat(String s1, String s2)**
-It returns the concatenation of 2 strings. A string with =null= value is considered as an empty string.
+It returns the concatenation of 2 strings. A string with `null` value is considered as an empty string.
-*String replaceAll(String src, String regex, String replacement)*
+**String replaceAll(String src, String regex, String replacement)**
-Replace each occurrence of regular expression match in the first string with the =replacement= string and return the replaced string. A 'regex' string with =null= value is considered as no change. A 'replacement' string with =null= value is consider as an empty string.
+Replace each occurrence of regular expression match in the first string with the `replacement` string and return the replaced string. A 'regex' string with `null` value is considered as no change. A 'replacement' string with `null` value is consider as an empty string.
-*String appendAll(String src, String append, String delimeter)*
+**String appendAll(String src, String append, String delimeter)**
- Add the =append= string into each splitted sub-strings of the first string(=src=). The split is performed into =src= string using the =delimiter=. E.g. =appendAll("/a/b/,/c/b/,/c/d/", "ADD", ",")= will return =/a/b/ADD,/c/b/ADD,/c/d/ADD=. A =append= string with =null= value is consider as an empty string. A =delimiter= string with value =null= is considered as no append in the string.
+ Add the `append` string into each splitted sub-strings of the first string(`src`). The split is performed into `src` string using the `delimiter`. E.g. `appendAll("/a/b/,/c/b/,/c/d/", "ADD", ",")` will return `/a/b/ADD,/c/b/ADD,/c/d/ADD`. A `append` string with `null` value is consider as an empty string. A `delimiter` string with value `null` is considered as no append in the string.
-*String trim(String s)*
+**String trim(String s)**
-It returns the trimmed value of the given string. A string with =null= value is considered as an empty string.
+It returns the trimmed value of the given string. A string with `null` value is considered as an empty string.
-*String urlEncode(String s)*
+**String urlEncode(String s)**
-It returns the URL UTF-8 encoded value of the given string. A string with =null= value is considered as an empty string.
+It returns the URL UTF-8 encoded value of the given string. A string with `null` value is considered as an empty string.
-*String timestamp()*
+**String timestamp()**
It returns the current datetime in ISO8601 format, down to minutes (yyyy-MM-ddTHH:mmZ), in the Oozie's processing timezone,
i.e. 1997-07-16T19:20Z
-*String toJsonStr(Map<String, String>)* (since Oozie 3.3)
+**String toJsonStr(Map<String, String>)** (since Oozie 3.3)
It returns an XML encoded JSON representation of a Map<String, String>. This function is useful to encode as
-a single property the complete action-data of an action, *wf:actionData(String actionName)*, in order to pass
+a single property the complete action-data of an action, **wf:actionData(String actionName)**, in order to pass
it in full to another action.
-*String toPropertiesStr(Map<String, String>)* (since Oozie 3.3)
+**String toPropertiesStr(Map<String, String>)** (since Oozie 3.3)
It returns an XML encoded Properties representation of a Map<String, String>. This function is useful to encode as
-a single property the complete action-data of an action, *wf:actionData(String actionName)*, in order to pass
+a single property the complete action-data of an action, **wf:actionData(String actionName)**, in order to pass
it in full to another action.
-*String toConfigurationStr(Map<String, String>)* (since Oozie 3.3)
+**String toConfigurationStr(Map<String, String>)** (since Oozie 3.3)
It returns an XML encoded Configuration representation of a Map<String, String>. This function is useful to encode as
-a single property the complete action-data of an action, *wf:actionData(String actionName)*, in order to pass
+a single property the complete action-data of an action, **wf:actionData(String actionName)**, in order to pass
it in full to another action.
----++++ 4.2.3 Workflow EL Functions
+#### 4.2.3 Workflow EL Functions
-*String wf:id()*
+**String wf:id()**
It returns the workflow job ID for the current workflow job.
-*String wf:name()*
+**String wf:name()**
It returns the workflow application name for the current workflow job.
-*String wf:appPath()*
+**String wf:appPath()**
It returns the workflow application path for the current workflow job.
-*String wf:conf(String name)*
+**String wf:conf(String name)**
It returns the value of the workflow job configuration property for the current workflow job, or an empty string if
undefined.
-*String wf:user()*
+**String wf:user()**
It returns the user name that started the current workflow job.
-*String wf:group()*
+**String wf:group()**
It returns the group/ACL for the current workflow job.
-*String wf:callback(String stateVar)*
+**String wf:callback(String stateVar)**
-It returns the callback URL for the current workflow action node, =stateVar= can be a valid exit state (=OK= or
- =ERROR=) for the action or a token to be replaced with the exit state by the remote system executing the task.
+It returns the callback URL for the current workflow action node, `stateVar` can be a valid exit state (`OK` or
+ `ERROR`) for the action or a token to be replaced with the exit state by the remote system executing the task.
-*String wf:transition(String node)*
+**String wf:transition(String node)**
It returns the transition taken by the specified workflow action node, or an empty string if the action has not being
executed or it has not completed yet.
-*String wf:lastErrorNode()*
+**String wf:lastErrorNode()**
-It returns the name of the last workflow action node that exit with an =ERROR= exit state, or an empty string if no
-action has exited with =ERROR= state in the current workflow job.
+It returns the name of the last workflow action node that exit with an `ERROR` exit state, or an empty string if no
+action has exited with `ERROR` state in the current workflow job.
-*String wf:errorCode(String node)*
+**String wf:errorCode(String node)**
It returns the error code for the specified action node, or an empty string if the action node has not exited
-with =ERROR= state.
+with `ERROR` state.
Each type of action node must define its complete error code list.
-*String wf:errorMessage(String message)*
+**String wf:errorMessage(String message)**
It returns the error message for the specified action node, or an empty string if no action node has not exited
-with =ERROR= state.
+with `ERROR` state.
The error message can be useful for debugging and notification purposes.
-*int wf:run()*
+**int wf:run()**
-It returns the run number for the current workflow job, normally =0= unless the workflow job is re-run, in which case
+It returns the run number for the current workflow job, normally `0` unless the workflow job is re-run, in which case
indicates the current run.
-*Map<String, String> wf:actionData(String node)*
+**Map<String, String> wf:actionData(String node)**
This function is only applicable to action nodes that produce output data on completion.
-The output data is in a Java Properties format and via this EL function it is available as a =Map<String, String>=.
+The output data is in a Java Properties format and via this EL function it is available as a `Map<String, String>`.
-*String wf:actionExternalId(String node)*
+**String wf:actionExternalId(String node)**
It returns the external Id for an action node, or an empty string if the action has not being executed or it has not
completed yet.
-*String wf:actionTrackerUri(String node)*
+**String wf:actionTrackerUri(String node)**
It returns the tracker URI for an action node, or an empty string if the action has not being executed or it has not
completed yet.
-*String wf:actionExternalStatus(String node)*
+**String wf:actionExternalStatus(String node)**
It returns the external status for an action node, or an empty string if the action has not being executed or it has
not completed yet.
----++++ 4.2.4 Hadoop EL Constants
+#### 4.2.4 Hadoop EL Constants
- * *RECORDS:* Hadoop record counters group name.
- * *MAP_IN:* Hadoop mapper input records counter name.
- * *MAP_OUT:* Hadoop mapper output records counter name.
- * *REDUCE_IN:* Hadoop reducer input records counter name.
- * *REDUCE_OUT:* Hadoop reducer input record counter name.
- * *GROUPS:* 1024 * Hadoop mapper/reducer record groups counter name.
+ * **RECORDS:** Hadoop record counters group name.
+ * **MAP_IN:** Hadoop mapper input records counter name.
+ * **MAP_OUT:** Hadoop mapper output records counter name.
+ * **REDUCE_IN:** Hadoop reducer input records counter name.
+ * **REDUCE_OUT:** Hadoop reducer input record counter name.
+ * **GROUPS:** 1024 *** Hadoop mapper/reducer record groups counter name.
----++++ 4.2.5 Hadoop EL Functions
+#### 4.2.5 Hadoop EL Functions
-#HadoopCountersEL
-*Map < String, Map < String, Long > > hadoop:counters(String node)*
+<a name="HadoopCountersEL"></a>
+**Map < String, Map < String, Long > > hadoop:counters(String node)**
-It returns the counters for a job submitted by a Hadoop action node. It returns =0= if the if the Hadoop job has not
+It returns the counters for a job submitted by a Hadoop action node. It returns `0` if the if the Hadoop job has not
started yet and for undefined counters.
The outer Map key is a counter group name. The inner Map value is a Map of counter names and counter values.
@@ -2017,8 +2084,9 @@
This function can also be used to access specific action statistics information. Examples of action stats and their usage through EL Functions (referenced in workflow xml) are given below.
-*Example of MR action stats:*
-<verbatim>
+**Example of MR action stats:**
+
+```
{
"ACTION_TYPE": "MAP_REDUCE",
"org.apache.hadoop.mapred.JobInProgress$Counter": {
@@ -2046,11 +2114,12 @@
"REDUCE_INPUT_RECORDS": 33
}
}
-</verbatim>
+```
Below is the workflow that describes how to access specific information using hadoop:counters() EL function from the MR stats.
-*Workflow xml:*
-<verbatim>
+**Workflow xml:**
+
+```
<workflow-app xmlns="uri:oozie:workflow:1.0" name="map-reduce-wf">
<start to="mr-node"/>
<action name="mr-node">
@@ -2116,11 +2185,12 @@
</kill>
<end name="end"/>
</workflow-app>
-</verbatim>
+```
-*Example of Pig action stats:*
-<verbatim>
+**Example of Pig action stats:**
+
+```
{
"ACTION_TYPE": "PIG",
"JOB_GRAPH": "job_201112191708_0008",
@@ -2165,11 +2235,12 @@
"NUMBER_MAPS": "1"
}
}
-</verbatim>
+```
Below is the workflow that describes how to access specific information using hadoop:counters() EL function from the Pig stats.
-*Workflow xml:*
-<verbatim>
+**Workflow xml:**
+
+```
<workflow-app xmlns="uri:oozie:workflow:1.0" name="pig-wf">
<start to="pig-node"/>
<action name="pig-node">
@@ -2222,10 +2293,10 @@
</kill>
<end name="end"/>
</workflow-app>
-</verbatim>
+```
-#Hadoop Configuration EL Function
-*String hadoop:conf(String hadoopConfHostPort, String propName)*
+<a name="Hadoop Configuration EL Function"></a>
+**String hadoop:conf(String hadoopConfHostPort, String propName)**
It returns the value of a property of Hadoop configuration.
@@ -2235,9 +2306,10 @@
and port. If hadoopConfHostPort could not be connected, Hadoop Conf EL Function will generate a default
hadoop configuration object directly, and then return the value of target property.
-*Example of usage of Hadoop Configuration EL Function:*
+**Example of usage of Hadoop Configuration EL Function:**
-<verbatim>
+
+```
<action name="mr-node">
<map-reduce>
<resource-manager>${resourceManager}</resource-manager>
@@ -2252,14 +2324,15 @@
<ok to="end"/>
<error to="fail"/>
</action>
-</verbatim>
+```
----++++ 4.2.6 Hadoop Jobs EL Function
+#### 4.2.6 Hadoop Jobs EL Function
The function _wf:actionData()_ can be used to access Hadoop ID's for actions such as Pig, by specifying the key as _hadoopJobs_.
An example is shown below.
-<verbatim>
+
+```
<workflow-app xmlns="uri:oozie:workflow:1.0" name="pig-wf">
<start to="pig-node"/>
<action name="pig-node">
@@ -2308,52 +2381,51 @@
</kill>
<end name="end"/>
</workflow-app>
-</verbatim>
-</verbatim>
+```
----++++ 4.2.7 HDFS EL Functions
+#### 4.2.7 HDFS EL Functions
-For all the functions in this section the path must include the FS URI. For example =hdfs://foo:8020/user/tucu=.
+For all the functions in this section the path must include the FS URI. For example `hdfs://foo:8020/user/tucu`.
-*boolean fs:exists(String path)*
+**boolean fs:exists(String path)**
-It returns =true= or =false= depending if the specified path URI exists or not. If a glob pattern is used for the URI, it returns true when there is at least one matching path.
+It returns `true` or `false` depending if the specified path URI exists or not. If a glob pattern is used for the URI, it returns true when there is at least one matching path.
-*boolean fs:isDir(String path)*
+**boolean fs:isDir(String path)**
-It returns =true= if the specified path URI exists and it is a directory, otherwise it returns =false=.
+It returns `true` if the specified path URI exists and it is a directory, otherwise it returns `false`.
-*long fs:dirSize(String path)*
+**long fs:dirSize(String path)**
It returns the size in bytes of all the files in the specified path. If the path is not a directory, or if it does not
exist it returns -1. It does not work recursively, only computes the size of the files under the specified path.
-*long fs:fileSize(String path)*
+**long fs:fileSize(String path)**
It returns the size in bytes of specified file. If the path is not a file, or if it does not exist it returns -1.
-*long fs:blockSize(String path)*
+**long fs:blockSize(String path)**
It returns the block size in bytes of specified file. If the path is not a file, or if it does not exist it returns -1.
----++++ 4.2.8 HCatalog EL Functions
+#### 4.2.8 HCatalog EL Functions
For all the functions in this section the URI must be a hcatalog URI identifying a table or set of partitions in a table.
The format for specifying hcatalog table URI is
hcat://[metastore server]:[port]/[database name]/[table name] and format to specify a hcatalog table partition URI is
-hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value]. For example:
+`hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value]`. For example:
<pre>
hcat://foo:8020/mydb/mytable
hcat://foo:8020/mydb/mytable/region=us;dt=20121212
</pre>
-*boolean hcat:exists(String uri)*
+**boolean hcat:exists(String uri)**
-It returns =true= or =false= based on if the partitions in the table exists or not.
+It returns `true` or `false` based on if the partitions in the table exists or not.
-#WorkflowNotifications
----++ 5 Workflow Notifications
+<a name="WorkflowNotifications"></a>
+## 5 Workflow Notifications
Workflow jobs can be configured to make an HTTP GET notification upon start and end of a workflow action node and upon
the completion of a workflow job.
@@ -2361,46 +2433,46 @@
Oozie will make a best effort to deliver the notifications, in case of failure it will retry the notification a
pre-configured number of times at a pre-configured interval before giving up.
-See also [[CoordinatorFunctionalSpec#CoordinatorNotifications][Coordinator Notifications]]
+See also [Coordinator Notifications](CoordinatorFunctionalSpec.html#CoordinatorNotifications)
----+++ 5.1 Workflow Job Status Notification
+### 5.1 Workflow Job Status Notification
-If the =oozie.wf.workflow.notification.url= property is present in the workflow job properties when submitting the job,
+If the `oozie.wf.workflow.notification.url` property is present in the workflow job properties when submitting the job,
Oozie will make a notification to the provided URL when the workflow job changes its status.
-=oozie.wf.workflow.notification.proxy= property can be used to configure either a http or socks proxy.
+`oozie.wf.workflow.notification.proxy` property can be used to configure either a http or socks proxy.
The format is proxyHostname:port or proxyType@proxyHostname:port. If proxy type is not specified, it defaults to http.
-For eg: myhttpproxyhost.mydomain.com:80 or socks@mysockshost.mydomain.com:1080.
+For eg: myhttpproxyhost.mydomain.com:80 or `socks@mysockshost.mydomain.com:1080`.
If the URL contains any of the following tokens, they will be replaced with the actual values by Oozie before making
the notification:
- * =$jobId= : The workflow job ID
- * =$status= : the workflow current state
- * =$parentId= : The parent ID of the workflow job. If there is no parent ID, it will be replaced with an empty string.
+ * `$jobId` : The workflow job ID
+ * `$status` : the workflow current state
+ * `$parentId` : The parent ID of the workflow job. If there is no parent ID, it will be replaced with an empty string.
----+++ 5.2 Node Start and End Notifications
+### 5.2 Node Start and End Notifications
-If the =oozie.wf.action.notification.url= property is present in the workflow job properties when submitting the job,
+If the `oozie.wf.action.notification.url` property is present in the workflow job properties when submitting the job,
Oozie will make a notification to the provided URL every time the workflow job enters and exits an action node. For
-decision nodes, Oozie will send a single notification with the name of the first evaluation that resolved to =true=.
-=oozie.wf.workflow.notification.proxy= property can be used to configure proxy, it should contain proxy host and port (xyz:4080).
+decision nodes, Oozie will send a single notification with the name of the first evaluation that resolved to `true`.
+`oozie.wf.workflow.notification.proxy` property can be used to configure proxy, it should contain proxy host and port (xyz:4080).
If the URL contains any of the following tokens, they will be replaced with the actual values by Oozie before making
the notification:
- * =$jobId= : The workflow job ID
- * =$nodeName= : The name of the workflow node
- * =$status= : If the action has not completed yet, it contains the action status 'S:<STATUS>'. If the action has ended, it contains the action transition 'T:<TRANSITION>'
+ * `$jobId` : The workflow job ID
+ * `$nodeName` : The name of the workflow node
+ * `$status` : If the action has not completed yet, it contains the action status `S:<STATUS>`. If the action has ended, it contains the action transition `T:<TRANSITION>`
-#UserPropagation
----++ 6 User Propagation
+<a name="UserPropagation"></a>
+## 6 User Propagation
-When submitting a workflow job, the configuration must contain a =user.name= property. If security is enabled, Oozie
-must ensure that the value of the =user.name= property in the configuration match the user credentials present in the
+When submitting a workflow job, the configuration must contain a `user.name` property. If security is enabled, Oozie
+must ensure that the value of the `user.name` property in the configuration match the user credentials present in the
protocol (web services) request.
-When submitting a workflow job, the configuration may contain the =oozie.job.acl= property (the =group.name= property
+When submitting a workflow job, the configuration may contain the `oozie.job.acl` property (the `group.name` property
has been deprecated). If authorization is enabled, this property is treated as as the ACL for the job, it can contain
user and group IDs separated by commas.
@@ -2410,8 +2482,8 @@
It is not allowed for map-reduce, pig and fs actions to override user/group information.
-#AppDeployment
----++ 7 Workflow Application Deployment
+<a name="AppDeployment"></a>
+## 7 Workflow Application Deployment
While Oozie encourages the use of self-contained applications (J2EE application model), it does not enforce it.
@@ -2420,31 +2492,33 @@
The layout of a workflow application directory is:
-<verbatim>
+
+```
- /workflow.xml
- /config-default.xml
|
- - /lib/ (*.jar;*.so)
-</verbatim>
+ - /lib/ (**.jar;**.so)
+```
-A workflow application must contain at least the workflow definition, the =workflow.xml= file.
+A workflow application must contain at least the workflow definition, the `workflow.xml` file.
All configuration files and scripts (Pig and shell) needed by the workflow action nodes should be under the application
HDFS directory.
All the JAR files and native libraries within the application 'lib/' directory are automatically added to the
-map-reduce and pig jobs =classpath= and =LD_PATH=.
+map-reduce and pig jobs `classpath` and `LD_PATH`.
Additional JAR files and native libraries not present in the application 'lib/' directory can be specified in
map-reduce and pig actions with the 'file' element (refer to the map-reduce and pig documentation).
For Map-Reduce jobs (not including streaming or pipes), additional jar files can also be included via an uber jar. An uber jar is a
jar file that contains additional jar files within a "lib" folder. To let Oozie know about an uber jar, simply specify it with
-the =oozie.mapreduce.uber.jar= configuration property and Oozie will tell Hadoop MapReduce that it is an uber jar. The ability to
-specify an uber jar is governed by the =oozie.action.mapreduce.uber.jar.enable= property in =oozie-site.xml=. See
-[[AG_Install#UberJar][Oozie Install]] for more information.
+the `oozie.mapreduce.uber.jar` configuration property and Oozie will tell Hadoop MapReduce that it is an uber jar. The ability to
+specify an uber jar is governed by the `oozie.action.mapreduce.uber.jar.enable` property in `oozie-site.xml`. See
+[Oozie Install](AG_Install.html#UberJar) for more information.
-<verbatim>
+
+```
<action name="mr-node">
<map-reduce>
<resource-manager>${resourceManager}</resource-manager>
@@ -2459,16 +2533,16 @@
<ok to="end"/>
<error to="fail"/>
</action>
-</verbatim>
+```
-The =config-default.xml= file defines, if any, default values for the workflow job or action parameters. This file must be in
-the Hadoop Configuration XML format. EL expressions are not supported and =user.name= property cannot be specified in
+The `config-default.xml` file defines, if any, default values for the workflow job or action parameters. This file must be in
+the Hadoop Configuration XML format. EL expressions are not supported and `user.name` property cannot be specified in
this file.
-Any other resources like =job.xml= files referenced from a workflow action node must be included under the
+Any other resources like `job.xml` files referenced from a workflow action node must be included under the
corresponding path, relative paths always start from the root of the workflow application.
----++ 8 External Data Assumptions
+## 8 External Data Assumptions
Oozie runs workflow jobs under the assumption all necessary data to execute an action is readily available at the
time the workflow job is about to executed the action.
@@ -2476,78 +2550,78 @@
In addition, it is assumed, but it is not the responsibility of Oozie, that all input data used by a workflow job is
immutable for the duration of the workflow job.
-#JobLifecycle
----++ 9 Workflow Jobs Lifecycle
+<a name="JobLifecycle"></a>
+## 9 Workflow Jobs Lifecycle
----+++ 9.1 Workflow Job Lifecycle
+### 9.1 Workflow Job Lifecycle
A workflow job can be in any of the following states:
- =PREP:= When a workflow job is first created it will be in =PREP= state. The workflow job is defined but it is not
+ `PREP:` When a workflow job is first created it will be in `PREP` state. The workflow job is defined but it is not
running.
- =RUNNING:= When a =CREATED= workflow job is started it goes into =RUNNING= state, it will remain in =RUNNING= state
+ `RUNNING:` When a `CREATED` workflow job is started it goes into `RUNNING` state, it will remain in `RUNNING` state
until it reaches its end state, ends in error or is suspended.
- =SUSPENDED:= A =RUNNING= workflow job can be suspended, it will remain in =SUSPENDED= state until the workflow job
+ `SUSPENDED:` A `RUNNING` workflow job can be suspended, it will remain in `SUSPENDED` state until the workflow job
is resumed or it is killed.
- =SUCCEEDED:= When a =RUNNING= workflow job reaches the =end= node it ends reaching the =SUCCEEDED= final state.
+ `SUCCEEDED:` When a `RUNNING` workflow job reaches the `end` node it ends reaching the `SUCCEEDED` final state.
- =KILLED:= When a =CREATED=, =RUNNING= or =SUSPENDED= workflow job is killed by an administrator or the owner via a
-request to Oozie the workflow job ends reaching the =KILLED= final state.
+ `KILLED:` When a `CREATED`, `RUNNING` or `SUSPENDED` workflow job is killed by an administrator or the owner via a
+request to Oozie the workflow job ends reaching the `KILLED` final state.
- =FAILED:= When a =RUNNING= workflow job fails due to an unexpected error it ends reaching the =FAILED= final state.
+ `FAILED:` When a `RUNNING` workflow job fails due to an unexpected error it ends reaching the `FAILED` final state.
-*Workflow job state valid transitions:*
+**Workflow job state valid transitions:**
- * --> =PREP=
- * =PREP= --> =RUNNING= | =KILLED=
- * =RUNNING= --> =SUSPENDED= | =SUCCEEDED= | =KILLED= | =FAILED=
- * =SUSPENDED= --> =RUNNING= | =KILLED=
+ * --> `PREP`
+ * `PREP` --> `RUNNING` | `KILLED`
+ * `RUNNING` --> `SUSPENDED` | `SUCCEEDED` | `KILLED` | `FAILED`
+ * `SUSPENDED` --> `RUNNING` | `KILLED`
----+++ 9.2 Workflow Action Lifecycle
+### 9.2 Workflow Action Lifecycle
-When a workflow action is created, it is in the =PREP= state. If an attempt to start it succeeds,
-it transitions to the =RUNNING= state; if the attempt fails in a way that Oozie deems to be transient, and a non-zero
-number of retries is configured, it enters the =START_RETRY= state and Oozie automatically retries the action until
+When a workflow action is created, it is in the `PREP` state. If an attempt to start it succeeds,
+it transitions to the `RUNNING` state; if the attempt fails in a way that Oozie deems to be transient, and a non-zero
+number of retries is configured, it enters the `START_RETRY` state and Oozie automatically retries the action until
it either succeeds or the configured number of retries is reached. If the error is not transient or still persists
-after the retries, the job transitions to the =START_MANUAL= state, where the user is expected to either kill the
+after the retries, the job transitions to the `START_MANUAL` state, where the user is expected to either kill the
action or manually resume it (after fixing any issues).
-From the =RUNNING= state, the action normally transitions to the =DONE= state. From =DONE=, it goes to =OK= if it ends
-successfully, otherwise to =ERROR= or =USER_RETRY=.
+From the `RUNNING` state, the action normally transitions to the `DONE` state. From `DONE`, it goes to `OK` if it ends
+successfully, otherwise to `ERROR` or `USER_RETRY`.
-If an error is encountered while Oozie is trying to end the action, the action transitions to the =END_RETRY= state if
-the error is transient and a non-zero number of retries is configured, or to the =END_MANUAL= state if it is not.
-In the =END_RETRY= state, Oozie automatically retries ending the action until it either succeeds or the configured
-number of retries is reached. If the error persists, the action goes to the =END_MANUAL= state, where the user is
+If an error is encountered while Oozie is trying to end the action, the action transitions to the `END_RETRY` state if
+the error is transient and a non-zero number of retries is configured, or to the `END_MANUAL` state if it is not.
+In the `END_RETRY` state, Oozie automatically retries ending the action until it either succeeds or the configured
+number of retries is reached. If the error persists, the action goes to the `END_MANUAL` state, where the user is
expected to either kill the action or manually resume it (after fixing any issues).
-The =USER_RETRY= state is used when retrying actions where the user has explicitly configured the number (and/or other
+The `USER_RETRY` state is used when retrying actions where the user has explicitly configured the number (and/or other
properties) of retries. For more information, see
-[[WorkflowFunctionalSpec#UserRetryWFActions][User-Retry for Workflow Actions]].
-From =USER_RETRY=, the action goes back to =RUNNING= and a retry is attempted. After the configured number of user
-retries, if the action is still failing, it goes to the =ERROR= state.
+[User-Retry for Workflow Actions](WorkflowFunctionalSpec.html#UserRetryWFActions).
+From `USER_RETRY`, the action goes back to `RUNNING` and a retry is attempted. After the configured number of user
+retries, if the action is still failing, it goes to the `ERROR` state.
-If an action is killed, it transitions to the =KILLED= state. If there is an error while attempting to kill the action,
-it goes to the =FAILED= state.
+If an action is killed, it transitions to the `KILLED` state. If there is an error while attempting to kill the action,
+it goes to the `FAILED` state.
-*Workflow action state valid transitions:*
+**Workflow action state valid transitions:**
- * --> =PREP=
- * =PREP= --> =START_RETRY= | =START_MANUAL= | =RUNNING= | =KILLED=
- * =START_RETRY= --> =START_MANUAL= | =RUNNING= | =KILLED=
- * =START_MANUAL= --> =RUNNING= | =KILLED=
- * =USER_RETRY= --> =RUNNING= | =DONE= | =KILLED=
- * =RUNNING= --> =DONE= | =KILLED=
- * =KILLED= --> =FAILED=
- * =DONE= --> =OK= | =ERROR= | =USER_RETRY= | =END_RETRY= | =END_MANUAL=
- * =END_RETRY= --> =END_MANUAL= | =KILLED= | =OK= | =ERROR=
- * =END_MANUAL= --> =KILLED= | =OK= | =ERROR=
+ * --> `PREP`
+ * `PREP` --> `START_RETRY` | `START_MANUAL` | `RUNNING` | `KILLED`
+ * `START_RETRY` --> `START_MANUAL` | `RUNNING` | `KILLED`
+ * `START_MANUAL` --> `RUNNING` | `KILLED`
+ * `USER_RETRY` --> `RUNNING` | `DONE` | `KILLED`
+ * `RUNNING` --> `DONE` | `KILLED`
+ * `KILLED` --> `FAILED`
+ * `DONE` --> `OK` | `ERROR` | `USER_RETRY` | `END_RETRY` | `END_MANUAL`
+ * `END_RETRY` --> `END_MANUAL` | `KILLED` | `OK` | `ERROR`
+ * `END_MANUAL` --> `KILLED` | `OK` | `ERROR`
-#JobReRun
----++ 10 Workflow Jobs Recovery (re-run)
+<a name="JobReRun"></a>
+## 10 Workflow Jobs Recovery (re-run)
Oozie must provide a mechanism by which a failed workflow job can be resubmitted and executed starting after any
action node that has completed its execution in the prior run. This is specially useful when the already executed
@@ -2557,15 +2631,15 @@
rerun will not fail due to not cleaned up data from the previous run.
When starting a workflow job in recovery mode, the user must indicate either what workflow nodes in the workflow should be
-skipped or whether job should be restarted from the failed node. At any rerun, only one option should be selected. The workflow nodes to skip must be specified in the =oozie.wf.rerun.skip.nodes= job configuration property,
-node names must be separated by commas. On the other hand, user needs to specify =oozie.wf.rerun.failnodes= to rerun from the failed node. The value is =true= or =false=. All workflow nodes indicated as skipped must have completed in the previous
-run. If a workflow node has not completed its execution in its previous run, and during the recovery submission is
+skipped or whether job should be restarted from the failed node. At any rerun, only one option should be selected. The workflow nodes to skip must be specified in the `oozie.wf.rerun.skip.nodes` job configuration property,
+node names must be separated by commas. On the other hand, user needs to specify `oozie.wf.rerun.failnodes` to rerun from the failed node. The value is `true` or `false`. All workflow nodes indicated as skipped must have completed in the previous
+run. If a workflow node has not completed its execution in its previous run, and during the recovery submission is
flagged as a node to be skipped, the recovery submission must fail.
The recovery workflow job will run under the same workflow job ID as the original workflow job.
-To submit a recovery workflow job the target workflow job to recover must be in an end state (=SUCCEEDED=, =FAILED=
-or =KILLED=).
+To submit a recovery workflow job the target workflow job to recover must be in an end state (`SUCCEEDED`, `FAILED`
+or `KILLED`).
A recovery run could be done using a new workflow application path under certain constraints (see next paragraph).
This is to allow users to do a one off patch for the workflow application without affecting other running jobs for the
@@ -2578,49 +2652,49 @@
for all executed nodes that will be skipped during recovery. This cannot be checked by Oozie, it is the responsibility
of the user to ensure this is the case.
-Oozie provides the =int wf:run()= EL function to returns the current run for a job, this function allows workflow
-applications to perform custom logic at workflow definition level (i.e. in a =decision= node) or at action node level
-(i.e. by passing the value of the =wf:run()= function as a parameter to the task).
+Oozie provides the `int wf:run()` EL function to returns the current run for a job, this function allows workflow
+applications to perform custom logic at workflow definition level (i.e. in a `decision` node) or at action node level
+(i.e. by passing the value of the `wf:run()` function as a parameter to the task).
-#OozieWSAPI
----++ 11 Oozie Web Services API
+<a name="OozieWSAPI"></a>
+## 11 Oozie Web Services API
-See the [[WebServicesAPI][Web Services API]] page.
+See the [Web Services API](WebServicesAPI.html) page.
----++ 12 Client API
+## 12 Client API
-Oozie provides a Java [[./apidocs/org/apache/oozie/client/package-summary.html][Client API]] that allows to
+Oozie provides a Java [Client API](./client/apidocs/org/apache/oozie/client/package-summary.html) that allows to
perform all common workflow job operations.
-The client API includes a [[./apidocs/org/apache/oozie/client/LocalOozie.html][LocalOozie class]] useful for testing
+The client API includes a [LocalOozie class](./core/apidocs/org/apache/oozie/local/LocalOozie.html) useful for testing
a workflow from within an IDE and for unit testing purposes.
The Client API is implemented as a client of the Web Services API.
----++ 13 Command Line Tools
+## 13 Command Line Tools
Oozie provides command line tool that allows to perform all common workflow job operations.
The command line tool is implemented as a client of the Web Services API.
----++ 14 Web UI Console
+## 14 Web UI Console
Oozie provides a read-only Web based console that allows to allow to monitor Oozie system status, workflow
applications status and workflow jobs status.
The Web base console is implemented as a client of the Web Services API.
----++ 15 Customizing Oozie with Extensions
+## 15 Customizing Oozie with Extensions
Out of the box Oozie provides support for a predefined set of action node types and Expression Language functions.
-Oozie provides a well defined API, [[./apidocs/org/apache/oozie/dag/action/package-summary.html][Action executor]]
+Oozie provides a well defined API, [Action executor](./core/apidocs/org/apache/oozie/action/package-summary.html)
API, to add support for additional action node types.
Extending Oozie should not require any code change to the Oozie codebase. It will require adding the JAR files
providing the new functionality and declaring them in Oozie system configuration.
----++ 16 Workflow Jobs Priority
+## 16 Workflow Jobs Priority
Oozie does not handle workflow jobs priority. As soon as a workflow job is ready to do a transition, Oozie will
trigger the transition. Workflow transitions and action triggering are assumed to be fast and lightweight operations.
@@ -2631,29 +2705,29 @@
Workflow applications can influence the remote systems priority via configuration if the remote systems support it.
-#ShareLib
----++ 17 HDFS Share Libraries for Workflow Applications (since Oozie 2.3)
+<a name="ShareLib"></a>
+## 17 HDFS Share Libraries for Workflow Applications (since Oozie 2.3)
Oozie supports job and system share libraries for workflow jobs.
Share libraries can simplify the deployment and management of common components across workflow applications.
For example, if a workflow job uses a share library with the Streaming, Pig & Har JARs files it does not have to
-bundled those JARs files in the workflow application =lib/= path.
+bundled those JARs files in the workflow application `lib/` path.
If workflow job uses a share library, Oozie will include all the JAR/SO files in the library in the
classpath/libpath for all its actions.
-A workflow job can specify a share library path using the job property =oozie.libpath=.
+A workflow job can specify a share library path using the job property `oozie.libpath`.
-A workflow job can use the system share library by setting the job property =oozie.use.system.libpath= to =true=.
-=oozie.use.system.libpath= can be also configured at action configuration.
-=oozie.use.system.libpath= defined at action level overrides job property.
+A workflow job can use the system share library by setting the job property `oozie.use.system.libpath` to `true`.
+`oozie.use.system.libpath` can be also configured at action configuration.
+`oozie.use.system.libpath` defined at action level overrides job property.
----+++ 17.1 Action Share Library Override (since Oozie 3.3)
+### 17.1 Action Share Library Override (since Oozie 3.3)
-Oozie share libraries are organized per action type, for example Pig action share library directory is =share/lib/pig/=
-and Mapreduce Streaming share library directory is =share/library/mapreduce-streaming/=.
+Oozie share libraries are organized per action type, for example Pig action share library directory is `share/lib/pig/`
+and Mapreduce Streaming share library directory is `share/library/mapreduce-streaming/`.
Oozie bundles a share library for specific versions of streaming, pig, hive, sqoop, distcp actions. These versions
of streaming, pig, hive, sqoop and distcp have been tested and verified to work correctly with the version of Oozie
@@ -2672,55 +2746,57 @@
* oozie.action.sharelib.for.#ACTIONTYPE# in the action configuration
* oozie.action.sharelib.for.#ACTIONTYPE# in the job configuration
* oozie.action.sharelib.for.#ACTIONTYPE# in the oozie server configuration
- * The action or custom action's =ActionExecutor getDefaultShareLibName()= method
+ * The action or custom action's `ActionExecutor getDefaultShareLibName()` method
More than one share library directory name can be specified for an action by using a comma separated list (since Oozie 4.x).
-For example: When using HCatLoader and HCatStorer in pig, =oozie.action.sharelib.for.pig= can be set to =pig,hcatalog= to include
+For example: When using HCatLoader and HCatStorer in pig, `oozie.action.sharelib.for.pig` can be set to `pig,hcatalog` to include
both pig and hcatalog jars.
-#UserRetryWFActions
----++ 18 User-Retry for Workflow Actions (since Oozie 3.1)
+<a name="UserRetryWFActions"></a>
+## 18 User-Retry for Workflow Actions (since Oozie 3.1)
-Oozie provides User-Retry capabilities when an action is in =ERROR= or =FAILED= state.
+Oozie provides User-Retry capabilities when an action is in `ERROR` or `FAILED` state.
Depending on the nature of the failure, Oozie can define what type of errors allowed for User-Retry. There are certain errors
-Oozie is allowing for user-retry in default, for example, file-exists-error =FS009, FS008= when using chmod in workflow =fs=
-action, output-directory-exists-error =JA018= in workflow =map-reduce= action, job-not-exists-error =JA017= in action executor,
-FileNotFoundException =JA008= in action executor, and IOException =JA009= in action executor.
+Oozie is allowing for user-retry in default, for example, file-exists-error `FS009, FS008` when using chmod in workflow `fs`
+action, output-directory-exists-error `JA018` in workflow `map-reduce` action, job-not-exists-error `JA017` in action executor,
+FileNotFoundException `JA008` in action executor, and IOException `JA009` in action executor.
User-Retry allows user to give certain number of reties (must not exceed system max retries), so user can find the causes of
-that problem and fix them when action is in =USER_RETRY= state. If failure or error does not go away after max retries,
-the action becomes =FAILED= or =ERROR= and Oozie marks workflow job to =FAILED= or =KILLED=.
+that problem and fix them when action is in `USER_RETRY` state. If failure or error does not go away after max retries,
+the action becomes `FAILED` or `ERROR` and Oozie marks workflow job to `FAILED` or `KILLED`.
Oozie administrator can allow more error codes to be handled for User-Retry. By adding this configuration
-=oozie.service.LiteWorkflowStoreService.user.retry.error.code.ext= to =oozie.site.xml=
+`oozie.service.LiteWorkflowStoreService.user.retry.error.code.ext` to `oozie.site.xml`
and error codes as value, these error codes will be considered as User-Retry after system restart.
-Since Oozie 4.3, User-retry allows user to mention retry policy. The value for policy can be =periodic=
-or =exponential=, =periodic= being the default. Oozie administrator can define user retry policy for all workflow
-actions by adding this configuration =oozie.service.LiteWorkflowStoreService.user.retry.policy= to =oozie.site.xml=.
+Since Oozie 4.3, User-retry allows user to mention retry policy. The value for policy can be `periodic`
+or `exponential`, `periodic` being the default. Oozie administrator can define user retry policy for all workflow
+actions by adding this configuration `oozie.service.LiteWorkflowStoreService.user.retry.policy` to `oozie.site.xml`.
This value will be considered as user retry policy after system restart. This value can be overridden while defining
-actions in workflow xml if needed. The =retry-interval= should be specified in minutes.
+actions in workflow xml if needed. The `retry-interval` should be specified in minutes.
Examples of User-Retry in a workflow action is :
-<verbatim>
+
+```
<workflow-app xmlns="uri:oozie:workflow:1.0" name="wf-name">
<action name="a" retry-max="2" retry-interval="1" retry-policy="exponential">
</action>
-</verbatim>
+```
-#GlobalConfigurations
----++ 19 Global Configurations
+<a name="GlobalConfigurations"></a>
+## 19 Global Configurations
Oozie allows a global section to reduce the redundant resource-manager and name-node declarations for each action. The user can
-define a =global= section in the beginning of the =workflow.xml=. The global section may contain the =job-xml=, =configuration=,
-=resource-manager=, or =name-node= that the user would like to set for every action. If a user then redefines one of these in a
+define a `global` section in the beginning of the `workflow.xml`. The global section may contain the `job-xml`, `configuration`,
+`resource-manager`, or `name-node` that the user would like to set for every action. If a user then redefines one of these in a
specific action node, Oozie will update use the specific declaration instead of the global one for that action.
Example of a global element:
-<verbatim>
+
+```
<workflow-app xmlns="uri:oozie:workflow:1.0" name="wf-name">
<global>
<resource-manager>${resourceManager}</resource-manager>
@@ -2734,31 +2810,33 @@
</configuration>
</global>
...
-</verbatim>
+```
----++ 20 Suspend On Nodes
+## 20 Suspend On Nodes
-Specifying =oozie.suspend.on.nodes= in a job.properties file lets users specify a list of actions that will cause Oozie to
+Specifying `oozie.suspend.on.nodes` in a job.properties file lets users specify a list of actions that will cause Oozie to
automatically suspend the workflow upon reaching any of those actions; like breakpoints in a debugger. To continue running the
workflow, the user simply uses the
-[[DG_CommandLineTool#Resuming_a_Workflow_Coordinator_or_Bundle_Job][-resume command from the Oozie CLI]]. Specifying a * will
+[-resume command from the Oozie CLI](DG_CommandLineTool.html#Resuming_a_Workflow_Coordinator_or_Bundle_Job). Specifying a * will
cause Oozie to suspend the workflow on all nodes.
For example:
-<verbatim>
+
+```
oozie.suspend.on.nodes=mr-node,my-pig-action,my-fork
-</verbatim>
+```
Specifying the above in a job.properties file will cause Oozie to suspend the workflow when any of those three actions are about
to be executed.
----++ Appendixes
+## Appendixes
-#OozieWFSchema
----+++ Appendix A, Oozie Workflow and Common XML Schemas
+<a name="OozieWFSchema"></a>
+### Appendix A, Oozie Workflow and Common XML Schemas
----++++ Oozie Workflow Schema Version 1.0
-<verbatim>
+#### Oozie Workflow Schema Version 1.0
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:workflow="uri:oozie:workflow:1.0"
elementFormDefault="qualified" targetNamespace="uri:oozie:workflow:1.0">
.
@@ -3043,10 +3121,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Oozie Common Schema Version 1.0
-<verbatim>
+#### Oozie Common Schema Version 1.0
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" elementFormDefault="qualified">
.
<xs:complexType name="LAUNCHER">
@@ -3122,10 +3201,11 @@
</xs:complexType>
.
</xs:schema>
-</verbatim>
+```
----++++ Oozie Workflow Schema Version 0.5
-<verbatim>
+#### Oozie Workflow Schema Version 0.5
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:workflow="uri:oozie:workflow:0.5"
elementFormDefault="qualified" targetNamespace="uri:oozie:workflow:0.5">
@@ -3450,10 +3530,11 @@
<xs:attribute name="type" type="xs:string" use="required"/>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie Workflow Schema Version 0.4.5
-<verbatim>
+#### Oozie Workflow Schema Version 0.4.5
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:workflow="uri:oozie:workflow:0.4.5"
elementFormDefault="qualified" targetNamespace="uri:oozie:workflow:0.4.5">
@@ -3775,10 +3856,11 @@
<xs:attribute name="type" type="xs:string" use="required"/>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie Workflow Schema Version 0.4
-<verbatim>
+#### Oozie Workflow Schema Version 0.4
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:workflow="uri:oozie:workflow:0.4"
elementFormDefault="qualified" targetNamespace="uri:oozie:workflow:0.4">
@@ -3969,7 +4051,7 @@
<xs:element name="name-node" type="xs:string" minOccurs="0" maxOccurs="1"/>
<xs:element name="job-xml" type="xs:string" minOccurs="0" maxOccurs="unbounded"/>
<xs:element name="configuration" type="workflow:CONFIGURATION" minOccurs="0" maxOccurs="1"/>
- <xs:choice minOccurs="0" maxOccurs="unbounded">
+ <xs:choice minOccurs="0" maxOccurs="unbounded">
<xs:element name="delete" type="workflow:DELETE"/>
<xs:element name="mkdir" type="workflow:MKDIR"/>
<xs:element name="move" type="workflow:MOVE"/>
@@ -4060,13 +4142,13 @@
<xs:attribute name="permissions" type="xs:string" use="required"/>
<xs:attribute name="dir-files" type="xs:string"/>
</xs:complexType>
-
- <xs:complexType name="CREDENTIALS">
+
+ <xs:complexType name="CREDENTIALS">
<xs:sequence minOccurs="0" maxOccurs="unbounded">
<xs:element name="credential" type="workflow:CREDENTIAL"/>
- </xs:sequence>
+ </xs:sequence>
</xs:complexType>
-
+
<xs:complexType name="CREDENTIAL">
<xs:sequence minOccurs="0" maxOccurs="unbounded" >
<xs:element name="property" minOccurs="1" maxOccurs="unbounded">
@@ -4083,10 +4165,11 @@
<xs:attribute name="type" type="xs:string" use="required"/>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie Workflow Schema Version 0.3
-<verbatim>
+#### Oozie Workflow Schema Version 0.3
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:workflow="uri:oozie:workflow:0.3"
elementFormDefault="qualified" targetNamespace="uri:oozie:workflow:0.3">
@@ -4334,13 +4417,13 @@
<xs:attribute name="permissions" type="xs:string" use="required"/>
<xs:attribute name="dir-files" type="xs:string"/>
</xs:complexType>
-
- <xs:complexType name="CREDENTIALS">
+
+ <xs:complexType name="CREDENTIALS">
<xs:sequence minOccurs="0" maxOccurs="unbounded">
<xs:element name="credential" type="workflow:CREDENTIAL"/>
- </xs:sequence>
+ </xs:sequence>
</xs:complexType>
-
+
<xs:complexType name="CREDENTIAL">
<xs:sequence minOccurs="0" maxOccurs="unbounded" >
<xs:element name="property" minOccurs="1" maxOccurs="unbounded">
@@ -4359,10 +4442,11 @@
</xs:schema>
-</verbatim>
+```
----++++ Oozie Workflow Schema Version 0.2.5
-<verbatim>
+#### Oozie Workflow Schema Version 0.2.5
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:workflow="uri:oozie:workflow:0.2.5"
elementFormDefault="qualified" targetNamespace="uri:oozie:workflow:0.2.5">
@@ -4608,13 +4692,13 @@
<xs:attribute name="permissions" type="xs:string" use="required"/>
<xs:attribute name="dir-files" type="xs:string"/>
</xs:complexType>
-
- <xs:complexType name="CREDENTIALS">
+
+ <xs:complexType name="CREDENTIALS">
<xs:sequence minOccurs="0" maxOccurs="unbounded">
<xs:element name="credential" type="workflow:CREDENTIAL"/>
- </xs:sequence>
+ </xs:sequence>
</xs:complexType>
-
+
<xs:complexType name="CREDENTIAL">
<xs:sequence minOccurs="0" maxOccurs="unbounded" >
<xs:element name="property" minOccurs="1" maxOccurs="unbounded">
@@ -4631,10 +4715,11 @@
<xs:attribute name="type" type="xs:string" use="required"/>
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie Workflow Schema Version 0.2
-<verbatim>
+#### Oozie Workflow Schema Version 0.2
+
+```
<?xml version="1.0" encoding="UTF-8"?>
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:workflow="uri:oozie:workflow:0.2"
elementFormDefault="qualified" targetNamespace="uri:oozie:workflow:0.2">
@@ -4881,13 +4966,14 @@
</xs:complexType>
</xs:schema>
-</verbatim>
+```
-#SLASchema
----++++ Oozie SLA Version 0.2
+<a name="SLASchema"></a>
+#### Oozie SLA Version 0.2
* Supported in Oozie workflow schema version 0.5
-<verbatim>
+
+```
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:sla="uri:oozie:sla:0.2" elementFormDefault="qualified"
targetNamespace="uri:oozie:sla:0.2">
@@ -4917,13 +5003,14 @@
</xs:complexType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie SLA Version 0.1
+#### Oozie SLA Version 0.1
* Oozie SLA schema is supported in Oozie workflow schema version 0.2 onwards
-<verbatim>
+
+```
<?xml version="1.0" encoding="UTF-8"?>
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:sla="uri:oozie:sla:0.1" elementFormDefault="qualified"
@@ -4985,10 +5072,11 @@
</xs:simpleType>
</xs:schema>
-</verbatim>
+```
----++++ Oozie Workflow Schema Version 0.1
-<verbatim>
+#### Oozie Workflow Schema Version 0.1
+
+```
<?xml version="1.0" encoding="UTF-8"?>
<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:workflow="uri:oozie:workflow:0.1"
elementFormDefault="qualified" targetNamespace="uri:oozie:workflow:0.1">
@@ -5208,25 +5296,27 @@
<xs:attribute name="permissions" type="xs:string" use="required" />
<xs:attribute name="dir-files" type="xs:string" />
</xs:complexType>
-</xs:schema></verbatim>
+</xs:schema>
+```
-#OozieWFExamples
----+++ Appendix B, Workflow Examples
+<a name="OozieWFExamples"></a>
+### Appendix B, Workflow Examples
----++++!! Fork and Join Example
+#### Fork and Join Example
The following workflow definition example executes 4 Map-Reduce jobs in 3 steps, 1 job, 2 jobs in parallel and 1 job.
The output of the jobs in the previous step are use as input for the next jobs.
-*Required workflow job parameters:*
+**Required workflow job parameters:**
- * =resourcemanager= : ResourceManager HOST:PORT
- * =namenode= : NameNode HOST:PORT
- * =input= : input directory
- * =output= : output directory
+ * `resourcemanager` : ResourceManager HOST:PORT
+ * `namenode` : NameNode HOST:PORT
+ * `input` : input directory
+ * `output` : output directory
-<verbatim>
+
+```
<workflow-app name='example-forkjoinwf' xmlns="uri:oozie:workflow:1.0">
<start to='firstjob' />
<action name="firstjob">
@@ -5360,8 +5450,8 @@
</kill>
<end name='end'/>
</workflow-app>
-</verbatim>
+```
-[[index][::Go back to Oozie Documentation Index::]]
+[::Go back to Oozie Documentation Index::](index.html)
-</noautolink>
+
diff --git a/docs/src/site/twiki/index.twiki b/docs/src/site/twiki/index.twiki
deleted file mode 100644
index 3003fa9..0000000
--- a/docs/src/site/twiki/index.twiki
+++ /dev/null
@@ -1,111 +0,0 @@
-<noautolink>
-
----+!! Oozie, Workflow Engine for Apache Hadoop
-
-Oozie v3 is a server based _Bundle Engine_ that provides a higher-level oozie abstraction that will batch a set of coordinator applications. The user will be able to start/stop/suspend/resume/rerun a set coordinator jobs in the bundle level resulting a better and easy operational control.
-
-Oozie v2 is a server based _Coordinator Engine_ specialized in running workflows based on time and data triggers.
-It can continuously run workflows based on time (e.g. run it every hour), and data availability
-(e.g. wait for my input data to exist before running my workflow).
-
-Oozie v1 is a server based _Workflow Engine_ specialized in running workflow jobs with actions that
-execute Hadoop Map/Reduce and Pig jobs.
-
-%TOC%
-
----++ Distribution Contents
-
-Oozie distribution consists of a single 'tar.gz' file containing:
-
- * Readme, license, notice & [[./release-log.txt][Release log]] files.
- * Oozie server: =oozie-server= directory.
- * Scripts: =bin/= directory, client and server scripts.
- * Binaries: =lib/= directory, client JAR files.
- * Configuration: =conf/= server configuration directory.
- * Archives:
- * =oozie-client-*.tar.gz=: Client tools.
- * =oozie.war=: Oozie WAR file.
- * =docs.zip=: Documentation.
- * =oozie-examples-*.tar.gz=: Examples.
- * =oozie-sharelib-*.tar.gz=: Share libraries (with Streaming, Pig JARs).
-
----++ Quick Start
-
-Enough reading already? Follow the steps in [[DG_QuickStart][Oozie Quick Start]] to get Oozie up and running.
-
----++ Developer Documentation
-
- * [[DG_Overview][Overview]]
- * [[DG_QuickStart][Oozie Quick Start]]
- * [[DG_Examples][Running the Examples]]
- * [[WorkflowFunctionalSpec][Workflow Functional Specification]]
- * [[CoordinatorFunctionalSpec][Coordinator Functional Specification]]
- * [[BundleFunctionalSpec][Bundle Functional Specification]]
- * [[https://docs.oracle.com/javaee/7/tutorial/jsf-el.htm][EL Expression Language Quick Reference]]
- * [[DG_CommandLineTool][Command Line Tool]]
- * [[DG_WorkflowReRun][Workflow Re-runs Explained]]
- * [[DG_HCatalogIntegration][HCatalog Integration Explained]]
-
- * [[./client/apidocs/index.html][Oozie Client Javadocs]]
- * [[./core/apidocs/index.html][Oozie Core Javadocs]]
- * [[WebServicesAPI][Oozie Web Services API]]
- * [[DG_ActionAuthentication][Action Authentication]]
-
----+++ Action Extensions
-
- * [[DG_EmailActionExtension][Email Action]]
- * [[DG_ShellActionExtension][Shell Action]]
- * [[DG_HiveActionExtension][Hive Action]]
- * [[DG_Hive2ActionExtension][Hive 2 Action]]
- * [[DG_SqoopActionExtension][Sqoop Action]]
- * [[DG_SshActionExtension][Ssh Action]]
- * [[DG_DistCpActionExtension][DistCp Action]]
- * [[DG_SparkActionExtension][Spark Action]]
- * [[DG_CustomActionExecutor][Writing a Custom Action Executor]]
-
----+++ Job Status and SLA Monitoring
-
- * [[DG_JMSNotifications][JMS Notifications for Job and SLA]]
- * [[DG_SLAMonitoring][Configuring and Monitoring SLA]]
-
----++ Administrator Documentation
-
- * [[AG_Install][Oozie Install]]
- * [[AG_OozieLogging][Oozie Logging]]
- * [[AG_HadoopConfiguration][Hadoop Configuration]]
- * [[AG_ActionConfiguration][Action Configuration]]
- * [[AG_Monitoring][Oozie Monitoring]]
- * [[DG_CommandLineTool][Command Line Tool]]
- * [[AG_OozieUpgrade][Oozie Upgrade]]
-
-#LicenseInfo
----++ Licensing Information
-
-Oozie is distributed under [[http://www.apache.org/licenses/LICENSE-2.0][Apache License 2.0]].
-
-For details on the license of the dependent components, refer to the
-[[./dependencies.html#Licenses][Dependencies Report, Licenses section]].
-
-Oozie bundles an embedded Jetty 9.x.
-
-Some of the components in the dependencies report don't mention their license in the published POM. They are:
-
- * JDOM: [[http://www.jdom.org/docs/faq.html#a0030][JDOM License]] (Apache style).
- * Oro: [[http://www.apache.org/licenses/LICENSE-2.0][Apache License 2.0]].
-
-Oozie uses a modified version of the Apache Doxia core and twiki plugins to generate Oozie documentation.
-
----++ Engineering Documentation
-
- * [[ENG_Building][Building Oozie]]
- * [[./dependencies.html][Dependencies Report]]
-
----++ MiniOozie Documentation
-
- * [[ENG_MiniOozie][Testing User Oozie Applications Using MiniOozie]]
-
----++ Oozie User Authentication Documentation
-
- * [[ENG_Custom_Authentication][Create Custom Oozie Authentication]]
-
-</noautolink>
diff --git a/pom.xml b/pom.xml
index 423d19d..d6440e4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1729,9 +1729,34 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-site-plugin</artifactId>
- <!-- Don't change version as doc generation may fail -->
- <!-- (using custom doxia for twiki pages generation) -->
- <version>2.0-beta-6</version>
+ <version>3.7</version>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.maven.doxia</groupId>
+ <artifactId>doxia-site-renderer</artifactId>
+ <version>1.8.1</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven.doxia</groupId>
+ <artifactId>doxia-sink-api</artifactId>
+ <version>1.8</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.velocity</groupId>
+ <artifactId>velocity</artifactId>
+ <version>1.5</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven.doxia</groupId>
+ <artifactId>doxia-module-markdown</artifactId>
+ <version>1.8</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven.doxia</groupId>
+ <artifactId>doxia-core</artifactId>
+ <version>1.7</version>
+ </dependency>
+ </dependencies>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
@@ -2011,7 +2036,7 @@
<exclude>work.log</exclude>
<exclude>SecurityAuth.audit</exclude>
<exclude>.gitignore</exclude>
- <exclude>docs/src/site/twiki/*.twiki</exclude>
+ <exclude>docs/src/site/markdown/*.md</exclude>
<exclude>examples/src/main/data/*</exclude>
<exclude>core/src/test/resources/test-ioutils.txt</exclude>
<exclude>core/src/test/resources/oldWorkFlowApp.serialized</exclude>
diff --git a/release-log.txt b/release-log.txt
index b835951..bf6c1c7 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
-- Oozie 5.1.0 release (trunk - unreleased)
+OOZIE-2734 [docs] Switch from TWiki to Markdown (asalamon74 via andras.piros, pbacsko, gezapeti)
OOZIE-3298 [MapReduce action] External ID is not filled properly and failing MR job is treated as SUCCEEDED (andras.piros via pbacsko, asasvari, gezapeti)
OOZIE-3317 amend [build] Fix false positive precommit reports (kmarton via andras.piros)
OOZIE-3160 PriorityDelayQueue put()/take() can cause significant CPU load due to busy waiting (pbacsko)