You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@falcon.apache.org by sr...@apache.org on 2014/09/25 08:04:14 UTC

svn commit: r1627461 [4/5] - in /incubator/falcon: site/ site/0.3-incubating/ site/0.4-incubating/ site/0.5-incubating/ site/restapi/ trunk/ trunk/general/ trunk/general/src/site/ trunk/releases/ trunk/releases/0.5-incubating/ trunk/releases/0.5-incuba...

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/HiveIntegration.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/HiveIntegration.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/HiveIntegration.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/HiveIntegration.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,372 @@
+---+ Hive Integration
+
+---++ Overview
+Falcon provides data management functions for feeds declaratively. It allows users to represent feed locations as
+time-based partition directories on HDFS containing files.
+
+Hive provides a simple and familiar database like tabular model of data management to its users,
+which are backed by HDFS. It supports two classes of tables, managed tables and external tables.
+
+Falcon allows users to represent feed location as Hive tables. Falcon supports both managed and external tables
+and provide data management services for tables such as replication, eviction, archival, etc. Falcon will notify
+HCatalog as a side effect of either acquiring, replicating or evicting a data set instance and adds the
+missing capability of HCatalog table replication.
+
+In the near future, Falcon will allow users to express pipeline processing in Hive scripts
+apart from Pig and Oozie workflows.
+
+
+---++ Assumptions
+   * Date is a mandatory first-level partition for Hive tables
+      * Data availability triggers are based on date pattern in Oozie
+   * Tables must be created in Hive prior to adding it as a Feed in Falcon.
+      * Duplicating this in Falcon will create confusion on the real source of truth. Also propagating schema changes
+    between systems is a hard problem.
+   * Falcon does not know about the encoding of the data and data should be in HCatalog supported format.
+
+---++ Configuration
+Falcon provides a system level option to enable Hive integration. Falcon must be configured with an implementation
+for the catalog registry. The default implementation for Hive is shipped with Falcon.
+
+<verbatim>
+catalog.service.impl=org.apache.falcon.catalog.HiveCatalogService
+</verbatim>
+
+
+---++ Incompatible changes
+Falcon depends heavily on data-availability triggers for scheduling Falcon workflows. Oozie must support
+data-availability triggers based on HCatalog partition availability. This is only available in oozie 4.x.
+
+Hence, Falcon for Hive support needs Oozie 4.x.
+
+
+---++ Oozie Shared Library setup
+Falcon post Hive integration depends heavily on the [[http://oozie.apache.org/docs/4.0.0/WorkflowFunctionalSpec.html#a17_HDFS_Share_Libraries_for_Workflow_Applications_since_Oozie_2.3][shared library feature of Oozie]].
+Since the sheer number of jars for HCatalog, Pig and Hive are in the many 10s in numbers, its quite daunting to
+redistribute the dependent jars from Falcon.
+
+[[http://oozie.apache.org/docs/4.0.0/DG_QuickStart.html#Oozie_Share_Lib_Installation][This is a one time effort in Oozie setup and is quite straightforward.]]
+
+
+---++ Approach
+
+---+++ Entity Changes
+
+   * Cluster DSL will have an additional registry-interface section, specifying the endpoint for the
+HCatalog server. If this is absent, no HCatalog publication will be done from Falcon for this cluster.
+      <verbatim>thrift://hcatalog-server:port</verbatim>
+   * Feed DSL will allow users to specify the URI (location) for HCatalog tables as:
+      <verbatim>catalog:database_name:table_name#partitions(key=value?)*</verbatim>
+   * Failure to publish to HCatalog will be retried (configurable # of retires) with back off. Permanent failures
+   after all the retries are exhausted will fail the Falcon workflow
+
+---+++ Eviction
+
+   * Falcon will construct DDL statements to filter candidate partitions eligible for eviction drop partitions
+   * Falcon will construct DDL statements to drop the eligible partitions
+   * Additionally, Falcon will nuke the data on HDFS for external tables
+
+
+---+++ Replication
+
+   * Falcon will use HCatalog (Hive) API to export the data for a given table and the partition,
+which will result in a data collection that includes metadata on the data's storage format, the schema,
+how the data is sorted, what table the data came from, and values of any partition keys from that table.
+   * Falcon will use discp tool to copy the exported data collection into the secondary cluster into a staging
+directory used by Falcon.
+   * Falcon will then import the data into HCatalog (Hive) using the HCatalog (Hive) API. If the specified table does
+not yet exist, Falcon will create it, using the information in the imported metadata to set defaults for the
+table such as schema, storage format, etc.
+   * The partition is not complete and hence not visible to users until all the data is committed on the secondary
+cluster, (no dirty reads)
+   * Data collection is staged by Falcon and retries for copy continues from where it left off.
+   * Failure to register with Hive will be retired. After all the attempts are exhausted,
+the data will be cleaned up by Falcon.
+
+
+---+++ Security
+The user owns all data managed by Falcon. Falcon runs as the user who submitted the feed. Falcon will authenticate
+with HCatalog as the end user who owns the entity and the data.
+
+For Hive managed tables, the table may be owned by the end user or “hive”. For “hive” owned tables,
+user will have to configure the feed as “hive”.
+
+
+---++ Load on HCatalog from Falcon
+It generally depends on the frequency of the feeds configured in Falcon and how often data is ingested, replicated,
+or processed.
+
+
+---++ User Impact
+   * There should not be any impact to user due to this integration
+   * Falcon will be fully backwards compatible 
+   * Users have a choice to either choose storage based on files on HDFS as they do today or use HCatalog for
+accessing the data in tables
+
+
+---++ Known Limitations
+
+---+++ Oozie
+
+   * Falcon with Hadoop 1.x requires copying guava jars manually to sharelib in oozie. Hadoop 2.x ships this.
+   * hcatalog-pig-adapter needs to be copied manually to oozie sharelib.
+<verbatim>
+bin/hadoop dfs -copyFromLocal $LFS/share/lib/hcatalog/hcatalog-pig-adapter-0.5.0-incubating.jar share/lib/hcatalog
+</verbatim>
+   * Oozie 4.x with Hadoop-2.x
+Replication jobs are submitted to oozie on the destination cluster. Oozie runs a table export job
+on RM on source cluster. Oozie server on the target cluster must be configured with source hadoop
+configs else jobs fail with errors on secure and non-secure clusters as below:
+<verbatim>
+org.apache.hadoop.security.token.SecretManager$InvalidToken: Password not found for ApplicationAttempt appattempt_1395965672651_0010_000002
+</verbatim>
+
+Make sure all oozie servers that falcon talks to has the hadoop configs configured in oozie-site.xml
+<verbatim>
+<property>
+      <name>oozie.service.HadoopAccessorService.hadoop.configurations</name>
+      <value>*=/etc/hadoop/conf,arpit-new-falcon-1.cs1cloud.internal:8020=/etc/hadoop-1,arpit-new-falcon-1.cs1cloud.internal:8032=/etc/hadoop-1,arpit-new-falcon-2.cs1cloud.internal:8020=/etc/hadoop-2,arpit-new-falcon-2.cs1cloud.internal:8032=/etc/hadoop-2,arpit-new-falcon-5.cs1cloud.internal:8020=/etc/hadoop-3,arpit-new-falcon-5.cs1cloud.internal:8032=/etc/hadoop-3</value>
+      <description>
+          Comma separated AUTHORITY=HADOOP_CONF_DIR, where AUTHORITY is the HOST:PORT of
+          the Hadoop service (JobTracker, HDFS). The wildcard '*' configuration is
+          used when there is no exact match for an authority. The HADOOP_CONF_DIR contains
+          the relevant Hadoop *-site.xml files. If the path is relative is looked within
+          the Oozie configuration directory; though the path can be absolute (i.e. to point
+          to Hadoop client conf/ directories in the local filesystem.
+      </description>
+    </property>
+</verbatim>
+
+---+++ Hive
+
+   * Dated Partitions
+Falcon does not work well when table partition contains multiple dated columns. Falcon only works
+with a single dated partition. This is being tracked in FALCON-357 which is a limitation in Oozie.
+<verbatim>
+catalog:default:table4#year=${YEAR};month=${MONTH};day=${DAY};hour=${HOUR};minute=${MINUTE}
+</verbatim>
+
+   * [[https://issues.apache.org/jira/browse/HIVE-5550][Hive table import fails for tables created with default text and sequence file formats using HCatalog API]]
+For some arcane reason, hive substitutes the output format for text and sequence to be prefixed with Hive.
+Hive table import fails since it compares against the input and output formats of the source table and they are
+different. Say, a table was created with out specifying the file format, it defaults to:
+<verbatim>
+fileFormat=TextFile, inputformat=org.apache.hadoop.mapred.TextInputFormat, outputformat=org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+</verbatim>
+
+But, when hive fetches the table from the metastore, it replaces the output format with org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+and the comparison between source and target table fails.
+<verbatim>
+org.apache.hadoop.hive.ql.parse.ImportSemanticAnalyzer#checkTable
+      // check IF/OF/Serde
+      String existingifc = table.getInputFormatClass().getName();
+      String importedifc = tableDesc.getInputFormat();
+      String existingofc = table.getOutputFormatClass().getName();
+      String importedofc = tableDesc.getOutputFormat();
+      if ((!existingifc.equals(importedifc))
+          || (!existingofc.equals(importedofc))) {
+        throw new SemanticException(
+            ErrorMsg.INCOMPATIBLE_SCHEMA
+                .getMsg(" Table inputformat/outputformats do not match"));
+      }
+</verbatim>
+The above is not an issue with Hive 0.13.
+
+---++ Hive Examples
+Following is an example entity configuration for lifecycle management functions for tables in Hive.
+
+---+++ Hive Table Lifecycle Management - Replication and Retention
+
+---++++ Primary Cluster
+
+<verbatim>
+<?xml version="1.0"?>
+<!--
+    Primary cluster configuration for demo vm
+  -->
+<cluster colo="west-coast" description="Primary Cluster"
+         name="primary-cluster"
+         xmlns="uri:falcon:cluster:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <interfaces>
+        <interface type="readonly" endpoint="hftp://localhost:10070"
+                   version="1.1.1" />
+        <interface type="write" endpoint="hdfs://localhost:10020"
+                   version="1.1.1" />
+        <interface type="execute" endpoint="localhost:10300"
+                   version="1.1.1" />
+        <interface type="workflow" endpoint="http://localhost:11010/oozie/"
+                   version="3.3.0" />
+        <interface type="registry" endpoint="thrift://localhost:19083"
+                   version="0.11.0" />
+        <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
+                   version="5.4.3" />
+    </interfaces>
+    <locations>
+        <location name="staging" path="/apps/falcon/staging" />
+        <location name="temp" path="/tmp" />
+        <location name="working" path="/apps/falcon/working" />
+    </locations>
+</cluster>
+</verbatim>
+
+---++++ BCP Cluster
+
+<verbatim>
+<?xml version="1.0"?>
+<!--
+    BCP cluster configuration for demo vm
+  -->
+<cluster colo="east-coast" description="BCP Cluster"
+         name="bcp-cluster"
+         xmlns="uri:falcon:cluster:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <interfaces>
+        <interface type="readonly" endpoint="hftp://localhost:20070"
+                   version="1.1.1" />
+        <interface type="write" endpoint="hdfs://localhost:20020"
+                   version="1.1.1" />
+        <interface type="execute" endpoint="localhost:20300"
+                   version="1.1.1" />
+        <interface type="workflow" endpoint="http://localhost:11020/oozie/"
+                   version="3.3.0" />
+        <interface type="registry" endpoint="thrift://localhost:29083"
+                   version="0.11.0" />
+        <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
+                   version="5.4.3" />
+    </interfaces>
+    <locations>
+        <location name="staging" path="/apps/falcon/staging" />
+        <location name="temp" path="/tmp" />
+        <location name="working" path="/apps/falcon/working" />
+    </locations>
+</cluster>
+</verbatim>
+
+---++++ Feed with replication and eviction policy
+
+<verbatim>
+<?xml version="1.0"?>
+<!--
+    Replicating Hourly customer table from primary to secondary cluster.
+  -->
+<feed description="Replicating customer table feed" name="customer-table-replicating-feed"
+      xmlns="uri:falcon:feed:0.1">
+    <frequency>hours(1)</frequency>
+    <timezone>UTC</timezone>
+
+    <clusters>
+        <cluster name="primary-cluster" type="source">
+            <validity start="2013-09-24T00:00Z" end="2013-10-26T00:00Z"/>
+            <retention limit="hours(2)" action="delete"/>
+        </cluster>
+        <cluster name="bcp-cluster" type="target">
+            <validity start="2013-09-24T00:00Z" end="2013-10-26T00:00Z"/>
+            <retention limit="days(30)" action="delete"/>
+
+            <table uri="catalog:tgt_demo_db:customer_bcp#ds=${YEAR}-${MONTH}-${DAY}-${HOUR}" />
+        </cluster>
+    </clusters>
+
+    <table uri="catalog:src_demo_db:customer_raw#ds=${YEAR}-${MONTH}-${DAY}-${HOUR}" />
+
+    <ACL owner="seetharam" group="users" permission="0755"/>
+    <schema location="" provider="hcatalog"/>
+</feed>
+</verbatim>
+
+
+---+++ Hive Table used in Processing Pipelines
+
+---++++ Primary Cluster
+The cluster definition from the lifecycle example can be used.
+
+---++++ Input Feed
+
+<verbatim>
+<?xml version="1.0"?>
+<feed description="clicks log table " name="input-table" xmlns="uri:falcon:feed:0.1">
+    <groups>online,bi</groups>
+    <frequency>hours(1)</frequency>
+    <timezone>UTC</timezone>
+
+    <clusters>
+        <cluster name="##cluster##" type="source">
+            <validity start="2010-01-01T00:00Z" end="2012-04-21T00:00Z"/>
+            <retention limit="hours(24)" action="delete"/>
+        </cluster>
+    </clusters>
+
+    <table uri="catalog:falcon_db:input_table#ds=${YEAR}-${MONTH}-${DAY}-${HOUR}" />
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
+</feed>
+</verbatim>
+
+
+---++++ Output Feed
+
+<verbatim>
+<?xml version="1.0"?>
+<feed description="clicks log identity table" name="output-table" xmlns="uri:falcon:feed:0.1">
+    <groups>online,bi</groups>
+    <frequency>hours(1)</frequency>
+    <timezone>UTC</timezone>
+
+    <clusters>
+        <cluster name="##cluster##" type="source">
+            <validity start="2010-01-01T00:00Z" end="2012-04-21T00:00Z"/>
+            <retention limit="hours(24)" action="delete"/>
+        </cluster>
+    </clusters>
+
+    <table uri="catalog:falcon_db:output_table#ds=${YEAR}-${MONTH}-${DAY}-${HOUR}" />
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
+</feed>
+</verbatim>
+
+
+---++++ Process
+
+<verbatim>
+<?xml version="1.0"?>
+<process name="##processName##" xmlns="uri:falcon:process:0.1">
+    <clusters>
+        <cluster name="##cluster##">
+            <validity end="2012-04-22T00:00Z" start="2012-04-21T00:00Z"/>
+        </cluster>
+    </clusters>
+
+    <parallel>1</parallel>
+    <order>FIFO</order>
+    <frequency>days(1)</frequency>
+    <timezone>UTC</timezone>
+
+    <inputs>
+        <input end="today(0,0)" start="today(0,0)" feed="input-table" name="input"/>
+    </inputs>
+
+    <outputs>
+        <output instance="now(0,0)" feed="output-table" name="output"/>
+    </outputs>
+
+    <properties>
+        <property name="blah" value="blah"/>
+    </properties>
+
+    <workflow engine="pig" path="/falcon/test/apps/pig/table-id.pig"/>
+
+    <retry policy="periodic" delay="minutes(10)" attempts="3"/>
+</process>
+</verbatim>
+
+
+---++++ Pig Script
+
+<verbatim>
+A = load '$input_database.$input_table' using org.apache.hcatalog.pig.HCatLoader();
+B = FILTER A BY $input_filter;
+C = foreach B generate id, value;
+store C into '$output_database.$output_table' USING org.apache.hcatalog.pig.HCatStorer('$output_dataout_partitions');
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/InstallationSteps.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/InstallationSteps.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/InstallationSteps.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/InstallationSteps.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,266 @@
+---++ Building & Installing Falcon
+
+
+---+++ Building Falcon
+
+<verbatim>
+git clone https://git-wip-us.apache.org/repos/asf/incubator-falcon.git falcon
+
+cd falcon
+
+export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=256m" && mvn clean install [For hadoop 1]
+export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=256m" && mvn clean install -Phadoop-2 [For hadoop 2]
+
+[optionally -Dhadoop.version=<<hadoop.version>> can be appended to build for a specific version of hadoop]
+[optionally -Doozie.version=<<oozie version>> can be appended to build with a specific version of oozie. Oozie versions >= 3.oozie-3.2.0-incubating are supported]
+
+</verbatim>
+
+Once the build successfully completes, artifacts can be packaged for deployment. The package can be built in embedded or distributed mode.
+
+*Embedded Mode*
+<verbatim>
+
+mvn clean assembly:assembly -DskipTests -DskipCheck=true [For hadoop 1]
+mvn clean assembly:assembly -DskipTests -DskipCheck=true -P hadoop-2 [For hadoop 2]
+
+</verbatim>
+
+Tar can be found in {project dir}/target/falcon-${project.version}-bin.tar.gz
+
+Tar is structured as follows
+
+<verbatim>
+
+|- bin
+   |- falcon
+   |- falcon-start
+   |- falcon-stop
+   |- falcon-config.sh
+   |- service-start.sh
+   |- service-stop.sh
+|- conf
+   |- startup.properties
+   |- runtime.properties
+   |- client.properties
+   |- log4j.xml
+   |- falcon-env.sh
+|- docs
+|- client
+   |- lib (client support libs)
+|- server
+   |- webapp
+      |- falcon.war
+|- hadooplibs
+|- README
+|- NOTICE.txt
+|- LICENSE.txt
+|- DISCLAIMER.txt
+|- CHANGES.txt
+</verbatim>
+
+*Distributed Mode*
+
+<verbatim>
+
+mvn clean assembly:assembly -DskipTests -DskipCheck=true -Pdistributed,hadoop-1 [For hadoop 1]
+mvn clean assembly:assembly -DskipTests -DskipCheck=true -Pdistributed,hadoop-2 [For hadoop 2]
+
+</verbatim>
+
+Tar can be found in {project dir}/target/falcon-distributed-${project.version}-server.tar.gz
+
+Tar is structured as follows
+
+<verbatim>
+
+|- bin
+   |- falcon
+   |- falcon-start
+   |- falcon-stop
+   |- falcon-config.sh
+   |- service-start.sh
+   |- service-stop.sh
+   |- prism-stop
+   |- prism-start
+|- conf
+   |- startup.properties
+   |- runtime.properties
+   |- client.properties
+   |- log4j.xml
+   |- falcon-env.sh
+|- docs
+|- client
+   |- lib (client support libs)
+|- server
+   |- webapp
+      |- falcon.war
+      |- prism.war
+|- hadooplibs
+|- README
+|- NOTICE.txt
+|- LICENSE.txt
+|- DISCLAIMER.txt
+|- CHANGES.txt
+</verbatim>
+
+---+++ Installing & running Falcon
+
+*Installing falcon*
+<verbatim>
+tar -xzvf {falcon package}
+cd falcon-distributed-${project.version} or falcon-${project.version}
+</verbatim>
+
+*Configuring Falcon*
+
+By default config directory used by falcon is {package dir}/conf. To override this set environment variable FALCON_CONF to the path of the conf dir.
+
+falcon-env.sh has been added to the falcon conf. This file can be used to set various environment variables that you need for you services.
+In addition you can set any other environment variables you might need. This file will be sourced by falcon scripts before any commands are executed. The following environment variables are available to set.
+
+<verbatim>
+# The java implementation to use. If JAVA_HOME is not found we expect java and jar to be in path
+#export JAVA_HOME=
+
+# any additional java opts you want to set. This will apply to both client and server operations
+#export FALCON_OPTS=
+
+# any additional java opts that you want to set for client only
+#export FALCON_CLIENT_OPTS=
+
+# java heap size we want to set for the client. Default is 1024MB
+#export FALCON_CLIENT_HEAP=
+
+# any additional opts you want to set for prism service.
+#export FALCON_PRISM_OPTS=
+
+# java heap size we want to set for the prism service. Default is 1024MB
+#export FALCON_PRISM_HEAP=
+
+# any additional opts you want to set for falcon service.
+#export FALCON_SERVER_OPTS=
+
+# java heap size we want to set for the falcon server. Default is 1024MB
+#export FALCON_SERVER_HEAP=
+
+# What is is considered as falcon home dir. Default is the base location of the installed software
+#export FALCON_HOME_DIR=
+
+# Where log files are stored. Default is logs directory under the base install location
+#export FALCON_LOG_DIR=
+
+# Where pid files are stored. Default is logs directory under the base install location
+#export FALCON_PID_DIR=
+
+# where the falcon active mq data is stored. Default is logs/data directory under the base install location
+#export FALCON_DATA_DIR=
+
+# Where do you want to expand the war file. By Default it is in /server/webapp dir under the base install dir.
+#export FALCON_EXPANDED_WEBAPP_DIR=
+</verbatim>
+
+*Starting Falcon Server*
+<verbatim>
+bin/falcon-start [-port <port>]
+</verbatim>
+
+By default, 
+* falcon server starts at port 15443 (https) by default . To change the port, use -port option
+   * falcon.enableTLS can be set to true or false explicitly to enable SSL, if not port that end with 443 will automatically put falcon on https://
+* falcon server starts embedded active mq. To control this behaviour, set the following system properties using -D option in environment variable FALCON_OPTS:
+   * falcon.embeddedmq=<true/false> - Should server start embedded active mq, default true
+   * falcon.embeddedmq.port=<port> - Port for embedded active mq, default 61616
+   * falcon.embeddedmq.data=<path> - Data path for embedded active mq, default {package dir}/logs/data
+* falcon server starts with conf from {package dir}/conf. To override this (to use the same conf with multiple falcon upgrades), set environment variable FALCON_CONF to the path of conf dir
+
+__Adding Extension Libraries__
+Library extensions allows users to add custom libraries to entity lifecycles such as feed retention, feed replication and process execution. This is useful for usecases such as adding filesystem extensions. To enable this, add the following configs to startup.properties:
+*.libext.paths=<paths to be added to all entity lifecycles>
+*.libext.feed.paths=<paths to be added to all feed lifecycles>
+*.libext.feed.retentions.paths=<paths to be added to feed retention workflow>
+*.libext.feed.replication.paths=<paths to be added to feed replication workflow>
+*.libext.process.paths=<paths to be added to process workflow>
+
+The configured jars are added to falcon classpath and the corresponding workflows
+
+
+*Starting Prism*
+<verbatim>
+bin/prism-start [-port <port>]
+</verbatim>
+
+By default, 
+* prism server starts at port 16443. To change the port, use -port option
+   * falcon.enableTLS can be set to true or false explicitly to enable SSL, if not port that end with 443 will automatically put prism on https://
+* prism starts with conf from {package dir}/conf. To override this (to use the same conf with multiple prism upgrades), set environment variable FALCON_CONF to the path of conf dir
+
+*Using Falcon*
+<verbatim>
+bin/falcon admin -version
+Falcon server build version: {Version:"0.3-SNAPSHOT-rd7e2be9afa2a5dc96acd1ec9e325f39c6b2f17f7",Mode:"embedded"}
+
+----
+
+bin/falcon help
+(for more details about falcon cli usage)
+</verbatim>
+
+*Dashboard*
+
+Once falcon / prism is started, you can view the status of falcon entities using the Web-based dashboard. The web UI works in both distributed and embedded mode. You can open your browser at the corresponding port to use the web UI.
+
+*Stopping Falcon Server*
+<verbatim>
+bin/falcon-stop
+</verbatim>
+
+*Stopping Prism*
+<verbatim>
+bin/prism-stop
+</verbatim>
+
+---+++ Preparing Oozie and Falcon packages for deployment
+<verbatim>
+cd <<project home>>
+src/bin/package.sh <<hadoop-version>> <<oozie-version>>
+
+>> ex. src/bin/package.sh 1.1.2 3.1.3-incubating or src/bin/package.sh 0.20.2-cdh3u5 4.0.0
+>> Falcon package is available in <<falcon home>>/target/falcon-<<version>>-bin.tar.gz
+>> Oozie package is available in <<falcon home>>/target/oozie-3.3.2-distro.tar.gz
+</verbatim>
+
+---+++ Running Examples using embedded package
+<verbatim>
+bin/falcon-start
+</verbatim>
+Make sure the hadoop and oozie endpoints are according to your setup in examples/entity/filesystem/standalone-cluster.xml
+<verbatim>
+bin/falcon entity -submit -type cluster -file examples/entity/filesystem/standalone-cluster.xml
+</verbatim>
+Submit input and output feeds:
+<verbatim>
+bin/falcon entity -submit -type feed -file examples/entity/filesystem/in-feed.xml
+bin/falcon entity -submit -type feed -file examples/entity/filesystem/out-feed.xml
+</verbatim>
+Set-up workflow for the process:
+<verbatim>
+hadoop fs -put examples/app /
+</verbatim>
+Submit and schedule the process:
+<verbatim>
+bin/falcon entity -submitAndSchedule -type process -file examples/entity/filesystem/oozie-mr-process.xml
+bin/falcon entity -submitAndSchedule -type process -file examples/entity/filesystem/pig-process.xml
+</verbatim>
+Generate input data:
+<verbatim>
+examples/data/generate.sh <<hdfs endpoint>>
+</verbatim>
+Get status of instances:
+<verbatim>
+bin/falcon instance -status -type process -name oozie-mr-process -start 2013-11-15T00:05Z -end 2013-11-15T01:00Z
+</verbatim>
+
+HCat based example entities are in examples/entity/hcat.
+
+

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/OnBoarding.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/OnBoarding.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/OnBoarding.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/OnBoarding.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,265 @@
+---++ Contents
+   * <a href="#Onboarding Steps">Onboarding Steps</a>
+   * <a href="#Sample Pipeline">Sample Pipeline</a>
+   * [[HiveIntegration][Hive Examples]]
+
+---+++ Onboarding Steps
+   * Create cluster definition for the cluster, specifying name node, job tracker, workflow engine endpoint, messaging endpoint. Refer to [[EntitySpecification][cluster definition]] for details.
+   * Create Feed definitions for each of the input and output specifying frequency, data path, ownership. Refer to [[EntitySpecification][feed definition]] for details.
+   * Create Process definition for your job. Process defines configuration for the workflow job. Important attributes are frequency, inputs/outputs and workflow path. Refer to [[EntitySpecification][process definition]] for process details.
+   * Define workflow for your job using the workflow engine(only oozie is supported as of now). Refer [[http://oozie.apache.org/docs/3.1.3-incubating/WorkflowFunctionalSpec.html][Oozie Workflow Specification]]. The libraries required for the workflow should be available in lib folder in workflow path.
+   * Set-up workflow definition, libraries and referenced scripts on hadoop. 
+   * Submit cluster definition
+   * Submit and schedule feed and process definitions
+   
+
+---+++ Sample Pipeline
+---++++ Cluster   
+Cluster definition that contains end points for name node, job tracker, oozie and jms server:
+<verbatim>
+<?xml version="1.0"?>
+<!--
+    Cluster configuration
+  -->
+<cluster colo="ua2" description="" name="corp" xmlns="uri:falcon:cluster:0.1"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">    
+    <interfaces>
+        <interface type="readonly" endpoint="hftp://name-node.com:50070" version="0.20.2-cdh3u0" />
+
+        <interface type="write" endpoint="hdfs://name-node.com:54310" version="0.20.2-cdh3u0" />
+
+        <interface type="execute" endpoint="job-tracker:54311" version="0.20.2-cdh3u0" />
+
+        <interface type="workflow" endpoint="http://oozie.com:11000/oozie/" version="3.1.4" />
+
+        <interface type="messaging" endpoint="tcp://jms-server.com:61616?daemon=true" version="5.1.6" />
+    </interfaces>
+
+    <locations>
+        <location name="staging" path="/projects/falcon/staging" />
+        <location name="temp" path="/tmp" />
+        <location name="working" path="/projects/falcon/working" />
+    </locations>
+</cluster>
+</verbatim>
+   
+---++++ Input Feed
+Hourly feed that defines feed path, frequency, ownership and validity:
+<verbatim>
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Hourly sample input data
+  -->
+
+<feed description="sample input data" name="SampleInput" xmlns="uri:falcon:feed:0.1"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <groups>group</groups>
+
+    <frequency>hours(1)</frequency>
+
+    <late-arrival cut-off="hours(6)" />
+
+    <clusters>
+        <cluster name="corp" type="source">
+            <validity start="2009-01-01T00:00Z" end="2099-12-31T00:00Z" timezone="UTC" />
+            <retention limit="months(24)" action="delete" />
+        </cluster>
+    </clusters>
+
+    <locations>
+        <location type="data" path="/projects/bootcamp/data/${YEAR}-${MONTH}-${DAY}-${HOUR}/SampleInput" />
+        <location type="stats" path="/projects/bootcamp/stats/SampleInput" />
+        <location type="meta" path="/projects/bootcamp/meta/SampleInput" />
+    </locations>
+
+    <ACL owner="suser" group="users" permission="0755" />
+
+    <schema location="/none" provider="none" />
+</feed>
+</verbatim>
+
+---++++ Output Feed
+Daily feed that defines feed path, frequency, ownership and validity:
+<verbatim>
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Daily sample output data
+  -->
+
+<feed description="sample output data" name="SampleOutput" xmlns="uri:falcon:feed:0.1"
+xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <groups>group</groups>
+
+    <frequency>days(1)</frequency>
+
+    <late-arrival cut-off="hours(6)" />
+
+    <clusters>
+        <cluster name="corp" type="source">
+            <validity start="2009-01-01T00:00Z" end="2099-12-31T00:00Z" timezone="UTC" />
+            <retention limit="months(24)" action="delete" />
+        </cluster>
+    </clusters>
+
+    <locations>
+        <location type="data" path="/projects/bootcamp/output/${YEAR}-${MONTH}-${DAY}/SampleOutput" />
+        <location type="stats" path="/projects/bootcamp/stats/SampleOutput" />
+        <location type="meta" path="/projects/bootcamp/meta/SampleOutput" />
+    </locations>
+
+    <ACL owner="suser" group="users" permission="0755" />
+
+    <schema location="/none" provider="none" />
+</feed>
+</verbatim>
+
+---++++ Process
+Sample process which runs daily at 6th hour on corp cluster. It takes one input - !SampleInput for the previous day(24 instances). It generates one output - !SampleOutput for previous day. The workflow is defined at /projects/bootcamp/workflow/workflow.xml. Any libraries available for the workflow should be at /projects/bootcamp/workflow/lib. The process also defines properties queueName, ssh.host, and fileTimestamp which are passed to the workflow. In addition, Falcon exposes the following properties to the workflow: nameNode, jobTracker(hadoop properties), input and output(Input/Output properties).
+
+<verbatim>
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Daily sample process. Runs at 6th hour every day. Input - last day's hourly data. Generates output for yesterday
+ -->
+<process name="SampleProcess">
+    <cluster name="corp" />
+
+    <frequency>days(1)</frequency>
+
+    <validity start="2012-04-03T06:00Z" end="2022-12-30T00:00Z" timezone="UTC" />
+
+    <inputs>
+        <input name="input" feed="SampleInput" start="yesterday(0,0)" end="today(-1,0)" />
+    </inputs>
+
+    <outputs>
+            <output name="output" feed="SampleOutput" instance="yesterday(0,0)" />
+    </outputs>
+
+    <properties>
+        <property name="queueName" value="reports" />
+        <property name="ssh.host" value="host.com" />
+        <property name="fileTimestamp" value="${coord:formatTime(coord:nominalTime(), 'yyyy-MM-dd')}" />
+    </properties>
+
+    <workflow engine="oozie" path="/projects/bootcamp/workflow" />
+
+    <retry policy="backoff" delay="minutes(5)" attempts="3" />
+    
+    <late-process policy="exp-backoff" delay="hours(1)">
+        <late-input input="input" workflow-path="/projects/bootcamp/workflow/lateinput" />
+    </late-process>
+</process>
+</verbatim>
+
+---++++ Oozie Workflow
+The sample user workflow contains 3 actions:
+   * Pig action - Executes pig script /projects/bootcamp/workflow/script.pig
+   * concatenator - Java action that concatenates part files and generates a single file
+   * file upload - ssh action that gets the concatenated file from hadoop and sends the file to a remote host
+   
+<verbatim>
+<workflow-app xmlns="uri:oozie:workflow:0.2" name="sample-wf">
+        <start to="pig" />
+
+        <action name="pig">
+                <pig>
+                        <job-tracker>${jobTracker}</job-tracker>
+                        <name-node>${nameNode}</name-node>
+                        <prepare>
+                                <delete path="${output}"/>
+                        </prepare>
+                        <configuration>
+                                <property>
+                                        <name>mapred.job.queue.name</name>
+                                        <value>${queueName}</value>
+                                </property>
+                                <property>
+                                        <name>mapreduce.fileoutputcommitter.marksuccessfuljobs</name>
+                                        <value>true</value>
+                                </property>
+                        </configuration>
+                        <script>${nameNode}/projects/bootcamp/workflow/script.pig</script>
+                        <param>input=${input}</param>
+                        <param>output=${output}</param>
+                        <file>lib/dependent.jar</file>
+                </pig>
+                <ok to="concatenator" />
+                <error to="fail" />
+        </action>
+
+        <action name="concatenator">
+                <java>
+                        <job-tracker>${jobTracker}</job-tracker>
+                        <name-node>${nameNode}</name-node>
+                        <prepare>
+                                <delete path="${nameNode}/projects/bootcamp/concat/data-${fileTimestamp}.csv"/>
+                        </prepare>
+                        <configuration>
+                                <property>
+                                        <name>mapred.job.queue.name</name>
+                                        <value>${queueName}</value>
+                                </property>
+                        </configuration>
+                        <main-class>com.wf.Concatenator</main-class>
+                        <arg>${output}</arg>
+                        <arg>${nameNode}/projects/bootcamp/concat/data-${fileTimestamp}.csv</arg>
+                </java>
+                <ok to="fileupload" />
+                <error to="fail"/>
+        </action>
+                        
+        <action name="fileupload">
+                <ssh>
+                        <host>localhost</host>
+                        <command>/tmp/fileupload.sh</command>
+                        <args>${nameNode}/projects/bootcamp/concat/data-${fileTimestamp}.csv</args>
+                        <args>${wf:conf("ssh.host")}</args>
+                        <capture-output/>
+                </ssh>
+                <ok to="fileUploadDecision" />
+                <error to="fail"/>
+        </action>
+
+        <decision name="fileUploadDecision">
+                <switch>
+                        <case to="end">
+                                ${wf:actionData('fileupload')['output'] == '0'}
+                        </case>
+                        <default to="fail"/>
+                </switch>
+        </decision>
+
+        <kill name="fail">
+                <message>Workflow failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
+        </kill>
+
+        <end name="end" />
+</workflow-app>
+</verbatim>
+
+---++++ File Upload Script
+The script gets the file from hadoop, rsyncs the file to /tmp on remote host and deletes the file from hadoop
+<verbatim>
+#!/bin/bash
+
+trap 'echo "output=$?"; exit $?' ERR INT TERM
+
+echo "Arguments: $@"
+SRCFILE=$1
+DESTHOST=$3
+
+FILENAME=`basename $SRCFILE`
+rm -f /tmp/$FILENAME
+hadoop fs -copyToLocal $SRCFILE /tmp/
+echo "Copied $SRCFILE to /tmp"
+
+rsync -ztv --rsh=ssh --stats /tmp/$FILENAME $DESTHOST:/tmp
+echo "rsynced $FILENAME to $DESTUSER@$DESTHOST:$DESTFILE"
+
+hadoop fs -rmr $SRCFILE
+echo "Deleted $SRCFILE"
+
+rm -f /tmp/$FILENAME
+echo "output=0"
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/Security.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/Security.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/Security.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/Security.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,193 @@
+---+ Securing Falcon
+
+---++ Overview
+
+Apache Falcon enforces authentication on protected resources. Once authentication has been established it sets a
+signed HTTP Cookie that contains an authentication token with the user name, user principal,
+authentication type and expiration time.
+
+It does so by using [[http://hadoop.apache .org/docs/current/hadoop-auth/index.html][Hadoop Auth]].
+Hadoop Auth is a Java library consisting of a client and a server components to enable Kerberos SPNEGO authentication
+for HTTP. Hadoop Auth also supports additional authentication mechanisms on the client and the server side via 2
+simple interfaces.
+
+
+---++ Authentication Methods
+
+It supports 2 authentication methods, simple and kerberos out of the box.
+
+---+++ Pseudo/Simple Authentication
+
+Falcon authenticates the user by simply trusting the value of the query string parameter 'user.name'. This is the
+default mode Falcon is configured with.
+
+---+++ Kerberos Authentication
+
+Falcon uses HTTP Kerberos SPNEGO to authenticate the user.
+
+---++ Server Side Configuration Setup
+
+---+++ Common Configuration Parameters
+
+<verbatim>
+# Authentication type must be specified: simple|kerberos
+*.falcon.authentication.type=kerberos
+</verbatim>
+
+---+++ Kerberos Configuration
+
+<verbatim>
+##### Service Configuration
+
+# Indicates the Kerberos principal to be used in Falcon Service.
+*.falcon.service.authentication.kerberos.principal=falcon/_HOST@EXAMPLE.COM
+
+# Location of the keytab file with the credentials for the Service principal.
+*.falcon.service.authentication.kerberos.keytab=/etc/security/keytabs/falcon.service.keytab
+
+# name node principal to talk to config store
+*.dfs.namenode.kerberos.principal=nn/_HOST@EXAMPLE.COM
+
+##### SPNEGO Configuration
+
+# Authentication type must be specified: simple|kerberos|<class>
+# org.apache.falcon.security.RemoteUserInHeaderBasedAuthenticationHandler can be used for backwards compatibility
+*.falcon.http.authentication.type=kerberos
+
+# Indicates how long (in seconds) an authentication token is valid before it has to be renewed.
+*.falcon.http.authentication.token.validity=36000
+
+# The signature secret for signing the authentication tokens.
+*.falcon.http.authentication.signature.secret=falcon
+
+# The domain to use for the HTTP cookie that stores the authentication token.
+*.falcon.http.authentication.cookie.domain=
+
+# Indicates if anonymous requests are allowed when using 'simple' authentication.
+*.falcon.http.authentication.simple.anonymous.allowed=true
+
+# Indicates the Kerberos principal to be used for HTTP endpoint.
+# The principal MUST start with 'HTTP/' as per Kerberos HTTP SPNEGO specification.
+*.falcon.http.authentication.kerberos.principal=HTTP/_HOST@EXAMPLE.COM
+
+# Location of the keytab file with the credentials for the HTTP principal.
+*.falcon.http.authentication.kerberos.keytab=/etc/security/keytabs/spnego.service.keytab
+
+# The kerberos names rules is to resolve kerberos principal names, refer to Hadoop's KerberosName for more details.
+*.falcon.http.authentication.kerberos.name.rules=DEFAULT
+
+# Comma separated list of black listed users
+*.falcon.http.authentication.blacklisted.users=
+</verbatim>
+
+---+++ Pseudo/Simple Configuration
+
+<verbatim>
+##### SPNEGO Configuration
+
+# Authentication type must be specified: simple|kerberos|<class>
+# org.apache.falcon.security.RemoteUserInHeaderBasedAuthenticationHandler can be used for backwards compatibility
+*.falcon.http.authentication.type=simple
+
+# Indicates how long (in seconds) an authentication token is valid before it has to be renewed.
+*.falcon.http.authentication.token.validity=36000
+
+# The signature secret for signing the authentication tokens.
+*.falcon.http.authentication.signature.secret=falcon
+
+# The domain to use for the HTTP cookie that stores the authentication token.
+*.falcon.http.authentication.cookie.domain=
+
+# Indicates if anonymous requests are allowed when using 'simple' authentication.
+*.falcon.http.authentication.simple.anonymous.allowed=true
+
+# Comma separated list of black listed users
+*.falcon.http.authentication.blacklisted.users=
+</verbatim>
+
+---+++ SSL Configuration
+
+<verbatim>
+*.falcon.enableTLS=true
+*.keystore.file=/path/to/keystore/file
+*.keystore.password=password
+</verbatim>
+
+---+++ Distributed Falcon Setup
+
+Falcon should be configured to communicate with Prism over TLS in secure mode. Its not enabled by default.
+
+
+---++ Changes to ownership and permissions of directories managed by Falcon
+
+| *Directory*             | *Location*                                                        | *Owner* | *Permissions* |
+| Configuration Store     | ${config.store.uri}                                               | falcon  | 750           |
+| Oozie coord/bundle XMLs | ${cluster.staging-location}/workflows/{entity}/{entity-name}      | falcon  | 644           |
+| Shared libs             | {cluster.working}/{lib,libext}                                    | falcon  | 755           |
+| App logs                | ${cluster.staging-location}/workflows/{entity}/{entity-name}/logs | falcon  | 777           |
+
+
+---++ Backwards compatibility
+
+---+++ Scheduled Entities
+
+Entities already scheduled with an earlier version of Falcon are not compatible with this version
+
+---+++ Falcon Clients
+
+Older Falcon clients are backwards compatible wrt Authentication and user information sent as part of the HTTP
+header, Remote-User is still honoured when the authentication type is configured as below:
+
+<verbatim>
+*.falcon.http.authentication.type=org.apache.falcon.security.RemoteUserInHeaderBasedAuthenticationHandler
+</verbatim>
+
+---+++ Blacklisted super users for authentication
+
+The blacklist users used to have the following super users: hdfs, mapreduce, oozie, and falcon.
+The list is externalized from code into Startup.properties file and is empty now and needs to be
+configured specifically in the file.
+
+
+---+++ Falcon Dashboard
+
+The dashboard assumes an anonymous user in Pseudo/Simple method and hence anonymous users must be enabled for it to
+work.
+<verbatim>
+# Indicates if anonymous requests are allowed when using 'simple' authentication.
+*.falcon.http.authentication.simple.anonymous.allowed=true
+</verbatim>
+
+In Kerberos method, the browser must support HTTP Kerberos SPNEGO.
+
+
+---++ Known Limitations
+
+   * ActiveMQ topics are not secure but will be in the near future
+   * Entities already scheduled with an earlier version of Falcon are not compatible with this version as new
+   workflow parameters are being passed back into Falcon such as the user are required
+   * Use of hftp as the scheme for read only interface in cluster entity [[https://issues.apache.org/jira/browse/HADOOP-10215][will not work in Oozie]]
+   The alternative is to use webhdfs scheme instead and its been tested with DistCp.
+
+
+---++ Examples
+
+---+++ Accessing the server using Falcon CLI (Java client)
+
+There is no change in the way the CLI is used. The CLI has been changed to work with the configured authentication
+method.
+
+---+++ Accessing the server using curl
+
+Try accessing protected resources using curl. The protected resources are:
+
+<verbatim>
+$ kinit
+Please enter the password for venkatesh@LOCALHOST:
+
+$ curl http://localhost:15000/api/admin/version
+
+$ curl http://localhost:15000/api/admin/version?user.name=venkatesh
+
+$ curl --negotiate -u foo -b ~/cookiejar.txt -c ~/cookiejar.txt curl http://localhost:15000/api/admin/version
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/index.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/index.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/index.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/index.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,17 @@
+---+++ Contents
+
+   * <a href="./InstallationSteps.html">Simple setup</a>
+
+   * <a href="./FalconArchitecture.html">Overview</a>
+
+   * <a href="./HiveIntegration.html">Hive Integration</a>
+
+   * <a href="./Security.html">Security</a>
+
+   * <a href="./OnBoarding.html">On boarding</a>
+
+   * <a href="./EntitySpecification.html">Entity specification</a>
+
+   * <a href="./FalconCLI.html">CLI</a>
+
+   * <a href="./restapi/ResourceList.html">Rest API</a>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdjacentVertices.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdjacentVertices.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdjacentVertices.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdjacentVertices.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,70 @@
+---++  GET api/graphs/lineage/vertices/:id/:direction
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Get a list of adjacent vertices or edges with a direction.
+
+---++ Parameters
+   * :id is the id of the vertex.
+   * :direction is the direction associated with the edges.
+
+   To get the adjacent out vertices of vertex pass direction as out, in to get adjacent in vertices
+   and both to get both in and out adjacent vertices. Similarly to get the out edges of vertex
+   pass outE, inE to get in edges and bothE to get the both in and out edges of vertex.
+
+      * out  : get the adjacent out vertices of vertex
+      * in   : get the adjacent in vertices of vertex
+      * both : get the both adjacent in and out vertices of vertex
+      * outCount  : get the number of out vertices of vertex
+      * inCount   : get the number of in vertices of vertex
+      * bothCount : get the number of adjacent in and out vertices of vertex
+      * outIds  : get the identifiers of out vertices of vertex
+      * inIds   : get the identifiers of in vertices of vertex
+      * bothIds : get the identifiers of adjacent in and out vertices of vertex
+
+---++ Results
+Adjacent vertices of the vertex for the specified direction.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/graphs/lineage/vertices/4/out
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "results": [
+        {
+            "timestamp":"2014-04-21T20:55Z",
+            "name":"sampleFeed",
+            "type":"feed-instance",
+            "_id":8,
+            "_type":"vertex"
+        }
+    ],
+    "totalSize":1}
+}
+</verbatim>
+
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/graphs/lineage/vertices/4/bothE
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "results":[
+        {
+            "_id":"Q5V-4-5g",
+            "_type":"edge",
+            "_outV":4,
+            "_inV":8,
+            "_label":"output"
+        }
+    ],
+    "totalSize":1
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminConfig.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminConfig.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminConfig.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminConfig.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,35 @@
+---++  GET /api/admin/config/:config-type
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Get configuration information of the falcon server.
+
+---++ Parameters
+   * :config-type can be build, deploy, startup or runtime
+
+---++ Results
+Configuration information of the server.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/admin/config/deploy
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "properties": [
+        {
+            "value": "embedded",
+            "key": "deploy.mode"
+        },
+        {
+            "value": "all",
+            "key": "domain"
+        }
+    ]
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminStack.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminStack.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminStack.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminStack.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,39 @@
+---++  GET /api/admin/stack
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Get stack trace of the falcon server.
+
+---++ Parameters
+None.
+---++ Results
+Stack trace of the server.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/admin/stack
+</verbatim>
+---+++ Result
+<verbatim>
+Reference Handler
+State: WAITING
+java.lang.Object.wait(Native Method)
+java.lang.Object.wait(Object.java:485)
+java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)Finalizer
+
+...
+
+State: TIMED_WAITING
+sun.misc.Unsafe.park(Native Method)
+java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
+java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
+java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
+java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
+java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:955)
+java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:917)
+java.lang.Thread.run(Thread.java:695)
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminVersion.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminVersion.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminVersion.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AdminVersion.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,34 @@
+---++  GET /api/admin/version
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Get version of the falcon server.
+
+---++ Parameters
+None.
+---++ Results
+Version of the server.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/admin/version
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "properties":[
+        {
+            "key":"Version",
+            "value":"0.4-incubating-SNAPSHOT-rb47788d1112fcf949c22a3860934167237b395b0"
+        },
+        {
+            "key":"Mode",
+            "value":"embedded"
+        }
+    ]
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AllEdges.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AllEdges.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AllEdges.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AllEdges.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,42 @@
+---++  GET pi/graphs/lineage//edges/all
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Get all edges.
+
+---++ Parameters
+None.
+
+---++ Results
+All edges in lineage graph.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/graphs/lineage/edges/all
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "results": [
+        {
+            "_id":"Q5V-4-5g",
+            "_type":"edge",
+            "_outV":4,
+            "_inV":8,
+            "_label":"output"
+        },
+        {
+            "_id":"Q6t-c-5g",
+            "_type":"edge",
+            "_outV":12,
+            "_inV":16,
+            "_label":"output"
+        }
+    ],
+    "totalSize": 2
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AllVertices.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AllVertices.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AllVertices.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/AllVertices.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,43 @@
+---++  GET api/graphs/lineage/vertices/all
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Get all vertices.
+
+---++ Parameters
+None.
+
+---++ Results
+All vertices in lineage graph.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/graphs/lineage/vertices/all
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "results": [
+        {
+            "timestamp":"2014-04-21T20:55Z",
+            "name":"sampleIngestProcess\/2014-03-01T10:00Z",
+            "type":"process-instance",
+            "version":"2.0.0",
+            "_id":4,
+            "_type":"vertex"
+        },
+        {
+            "timestamp":"2014-04-21T20:55Z",
+            "name":"rawEmailFeed\/2014-03-01T10:00Z",
+            "type":"feed-instance",
+            "_id":8,
+            "_type":"vertex"
+        }
+    ],
+    "totalSize": 2
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/Edge.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/Edge.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/Edge.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/Edge.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,33 @@
+---++  GET api/graphs/lineage/edges/:id
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Gets the edge with specified id.
+
+---++ Parameters
+   * :id is the unique id of the edge.
+
+---++ Results
+Edge with the specified id.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/graphs/lineage/edges/Q6t-c-5g
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "results":
+        {
+            "_id":"Q6t-c-5g",
+            "_type":"edge",
+            "_outV":12,
+            "_inV":16,
+            "_label":"output"
+        }
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDefinition.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDefinition.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDefinition.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDefinition.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,52 @@
+---++  GET /api/entities/definition/:entity-type/:entity-name
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Get definition of the entity.
+
+---++ Parameters
+   * :entity-type can be cluster, feed or process.
+   * :entity-name is name of the entity.
+
+---++ Results
+Definition of the entity.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/entities/definition/process/SampleProcess
+</verbatim>
+---+++ Result
+<verbatim>
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<process name="SampleProcess" xmlns="uri:falcon:process:0.1">
+  <clusters>
+    <cluster name="primary-cluster">
+      <validity start="2012-04-03T06:00Z" end="2022-12-30T00:00Z"/>
+    </cluster>
+  </clusters>
+  <parallel>1</parallel>
+  <order>FIFO</order>
+  <frequency>hours(1)</frequency>
+  <timezone>UTC</timezone>
+  <inputs>
+    <input name="input" feed="SampleInput" start="yesterday(0,0)" end="today(-1,0)"/>
+  </inputs>
+  <outputs>
+    <output name="output" feed="SampleOutput" instance="yesterday(0,0)"/>
+  </outputs>
+  <properties>
+    <property name="queueName" value="default"/>
+    <property name="ssh.host" value="localhost"/>
+    <property name="fileTimestamp" value="${coord:formatTime(coord:nominalTime(), 'yyyy-MM-dd')}"/>
+  </properties>
+  <workflow engine="oozie" path="/examples/apps/aggregator"/>
+  <retry policy="exp-backoff" delay="minutes(5)" attempts="3"/>
+  <late-process policy="exp-backoff" delay="hours(1)">
+    <late-input input="input" workflow-path="/projects/bootcamp/workflow/lateinput"/>
+  </late-process>
+</process>
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDelete.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDelete.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDelete.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDelete.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,30 @@
+---++  DELETE /api/entities/delete/:entity-type/:entity-name
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Delete the specified entity.
+
+---++ Parameters
+   * :entity-type can be feed or process.
+   * :entity-name is name of the feed or process.
+
+---++ Results
+Results of the delete operation.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+DELETE http://localhost:15000/api/entities/delete/cluster/SampleProcess
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "requestId": "falcon\/17ff6ca6-1c8a-459f-9ba8-8fec480e384a\n",
+    "message": "falcon\/SampleProcess(cluster) removed successfully\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>
+

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDependencies.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDependencies.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDependencies.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityDependencies.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,42 @@
+---++  GET /api/entities/dependencies/:entity-type/:entity-name
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Get dependencies of the entity.
+
+---++ Parameters
+   * :entity-type can be cluster, feed or process.
+   * :entity-name is name of the entity.
+
+---++ Results
+Dependenciess of the entity.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/entities/dependencies/process/SampleProcess
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "entity": [
+        {
+            "name": "SampleInput",
+            "type": "feed",
+            "tag": [Input]
+        },
+        {
+            "name": "SampleOutput",
+            "type": "feed"
+            "tag": [Output]
+        },
+        {
+            "name": "primary-cluster",
+            "type": "cluster"
+        }
+    ]
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityList.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityList.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityList.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityList.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,60 @@
+---++  GET /api/entities/list/:entity-type?fields=:fields
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Get list of the entities.
+
+---++ Parameters
+   * :entity-type can be cluster, feed or process.
+   * :fields (optional) additional fields that the client are interested in, separated by commas.
+     Currently falcon only support status as a valid field.
+
+---++ Results
+List of the entities.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/entities/list/feed
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "entity": [
+        {
+            "name": "SampleOutput",
+            "type": "feed"
+        },
+        {
+            "name": "SampleInput",
+            "type": "feed"
+        }
+    ]
+}
+</verbatim>
+
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/entities/list/feed?fields=status
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "entity": [
+        {
+            "name"  : "SampleOutput",
+            "type"  : "feed",
+            "status": "RUNNING"
+        },
+        {
+            "name": "SampleInput",
+            "type": "feed",
+            "status": "RUNNING"
+        }
+    ]
+}
+</verbatim>
+

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityResume.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityResume.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityResume.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityResume.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,29 @@
+---++  POST /api/entities/resume/:entity-type/:entity-name
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Resume a supended entity.
+
+---++ Parameters
+   * :entity-type can either be a feed or a process.
+   * :entity-name is name of the entity.
+
+---++ Results
+Result of the resume command.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/resume/process/SampleProcess
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "requestId": "default\/106582a9-130f-4903-8b8f-f95d7b286c30\n",
+    "message": "default\/SampleProcess(process) resumed successfully\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySchedule.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySchedule.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySchedule.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySchedule.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,29 @@
+---++  POST /api/entities/schedule/:entity-type/:entity-name
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Schedule an entity.
+
+---++ Parameters
+   * :entity-type can either be a feed or a process.
+   * :entity-name is name of the entity.
+
+---++ Results
+Result of the schedule command.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/schedule/process/SampleProcess
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "requestId": "default\/ee735c95-98bd-41b8-a705-2e78bcfcdcd9\n",
+    "message": "default\/SampleProcess(process) scheduled successfully\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityStatus.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityStatus.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityStatus.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityStatus.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,29 @@
+---++  GET /api/entities/status/:entity-type/:entity-name
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Get status of the entity.
+
+---++ Parameters
+   * :entity-type can be cluster, feed or process.
+   * :entity-name is name of the entity.
+
+---++ Results
+Status of the entity.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/entities/status/process/SampleProcess
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "requestId": "default\/4d35b382-852a-4bc7-9972-b9db3493322a\n",
+    "message": "default\/SUBMITTED\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySubmit.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySubmit.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySubmit.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySubmit.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,104 @@
+---++ POST  api/entities/submit/:entity-type
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Submit the given entity.
+
+---++ Parameters
+:entity-type can be cluster, feed or process.
+
+---++ Results
+Result of the submission.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/submit/feed
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Hourly sample input data -->
+
+<feed description="sample input data"
+      name="SampleInput" xmlns="uri:falcon:feed:0.1"
+      xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <groups>group</groups>
+    <frequency>hours(1)</frequency>
+    <late-arrival cut-off="hours(6)" />
+    <clusters>
+        <cluster name="primary-cluster" type="source">
+            <!--validity start="2009-01-01T00:00Z" end="2099-12-31T00:00Z" timezone="UTC" /-->
+            <validity start="2009-01-01T00:00Z" end="2099-12-31T00:00Z" />
+            <retention limit="months(24)" action="delete" />
+        </cluster>
+    </clusters>
+
+    <locations>
+        <location type="data" path="/projects/bootcamp/data/${YEAR}-${MONTH}-${DAY}-${HOUR}/SampleInput" />
+        <location type="stats" path="/projects/bootcamp/stats/SampleInput" />
+        <location type="meta" path="/projects/bootcamp/meta/SampleInput" />
+    </locations>
+
+    <ACL owner="suser" group="users" permission="0755" />
+
+    <schema location="/none" provider="none" />
+</feed>
+</verbatim>
+
+---+++ Result
+<verbatim>
+{
+    "requestId": "default\/d72a41f7-6420-487b-8199-62d66e492e35\n",
+    "message": "default\/Submit successful (feed) SampleInput\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>
+
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/submit/process
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Daily sample process. Runs at 6th hour every day. Input - last day's hourly data. Generates output for yesterday -->
+<process xmlns="uri:falcon:process:0.1" name="SampleProcess" >
+    <clusters>
+      <cluster name="primary-cluster">
+	<validity start="2012-04-03T06:00Z" end="2022-12-30T00:00Z" />
+      </cluster>
+    </clusters>
+
+    <parallel>1</parallel>
+    <order>FIFO</order>
+    <frequency>hours(1)</frequency>
+
+    <inputs>
+        <input name="input" feed="SampleInput" start="yesterday(0,0)" end="today(-1,0)" />
+    </inputs>
+
+    <outputs>
+        <output name="output" feed="SampleOutput" instance="yesterday(0,0)" />
+    </outputs>
+
+    <properties>
+        <property name="queueName" value="default" />
+        <property name="ssh.host" value="localhost" />
+        <property name="fileTimestamp" value="${coord:formatTime(coord:nominalTime(), 'yyyy-MM-dd')}" />
+    </properties>
+
+    <workflow engine="oozie" path="/examples/apps/aggregator" />
+    <retry policy="exp-backoff" delay="minutes(5)" attempts="3" />
+    
+    <late-process policy="exp-backoff" delay="hours(1)">
+        <late-input input="input" workflow-path="/projects/bootcamp/workflow/lateinput" />
+    </late-process>
+</process>
+</verbatim>
+
+---+++ Result
+<verbatim>
+{
+    "requestId": "default\/e5cc8230-f356-4566-9b65-536abdff8aa3\n",
+    "message": "default\/Submit successful (process) SampleProcess\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySubmitAndSchedule.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySubmitAndSchedule.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySubmitAndSchedule.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySubmitAndSchedule.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,62 @@
+---++  POST /api/entities/submitAndSchedule/:entity-type
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Submits and schedules an entity.
+
+---++ Parameters
+   * :entity-type can either be a feed or a process.
+
+---++ Results
+Result of the submit and schedule command.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/submitAndSchedule/process
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Daily sample process. Runs at 6th hour every day. Input - last day's hourly data. Generates output for yesterday -->
+<process xmlns="uri:falcon:process:0.1" name="SampleProcess" >
+    <clusters>
+      <cluster name="primary-cluster">
+	<validity start="2012-04-03T06:00Z" end="2022-12-30T00:00Z" />
+      </cluster>
+    </clusters>
+
+    <parallel>1</parallel>
+    <order>FIFO</order>
+    <frequency>hours(1)</frequency>
+
+    <inputs>
+        <input name="input" feed="SampleInput" start="yesterday(0,0)" end="today(-1,0)" />
+    </inputs>
+
+    <outputs>
+        <output name="output" feed="SampleOutput" instance="yesterday(0,0)" />
+    </outputs>
+
+    <properties>
+        <property name="queueName" value="default" />
+        <property name="ssh.host" value="localhost" />
+        <property name="fileTimestamp" value="${coord:formatTime(coord:nominalTime(), 'yyyy-MM-dd')}" />
+    </properties>
+
+    <workflow engine="oozie" path="/examples/apps/aggregator" />
+    <retry policy="exp-backoff" delay="minutes(5)" attempts="3" />
+    
+    <late-process policy="exp-backoff" delay="hours(1)">
+        <late-input input="input" workflow-path="/projects/bootcamp/workflow/lateinput" />
+    </late-process>
+</process>
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "requestId": "schedule\/default\/b5b40931-175b-4b15-8f2b-02ef2e66f06b\n\nsubmit\/default\/b5b40931-175b-4b15-8f2b-02ef2e66f06b\n\n",
+    "message": "schedule\/default\/SampleProcess(process) scheduled successfully\n\nsubmit\/default\/Submit successful (process) SampleProcess\n\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySuspend.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySuspend.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySuspend.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntitySuspend.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,29 @@
+---++  POST /api/entities/suspend/:entity-type/:entity-name
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Suspend an entity.
+
+---++ Parameters
+   * :entity-type can either be a feed or a process.
+   * :entity-name is name of the entity.
+
+---++ Results
+Status of the entity.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/suspend/process/SampleProcess
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "requestId": "default\/fe5f2b6c-1f2e-49fc-af3a-342079f0b46b\n",
+    "message": "default\/SampleProcess(process) suspended successfully\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityUpdate.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityUpdate.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityUpdate.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityUpdate.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,65 @@
+---++ POST  api/entities/update/:entity-type/:entity-name
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Updates the submitted entity.
+
+---++ Parameters
+   * :entity-type can be feed or process.
+   * :entity-name is name of the feed or process.
+   * :effective is optional effective time
+
+---++ Results
+Result of the validation.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/update/process/SampleProcess?effective=2014-01-01T00:00Z
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Daily sample process. Runs at 6th hour every day. Input - last day's hourly data. Generates output for yesterday -->
+<process xmlns="uri:falcon:process:0.1" name="SampleProcess" >
+    <clusters>
+      <cluster name="primary-cluster">
+	<validity start="2012-04-03T06:00Z" end="2022-12-30T00:00Z" />
+      </cluster>
+    </clusters>
+
+    <parallel>1</parallel>
+    <order>FIFO</order>
+    <frequency>hours(1)</frequency>
+
+    <inputs>
+        <input name="input" feed="SampleInput" start="yesterday(0,0)" end="today(-1,0)" />
+    </inputs>
+
+    <outputs>
+        <output name="output" feed="SampleOutput" instance="yesterday(0,0)" />
+    </outputs>
+
+    <properties>
+        <property name="queueName" value="default" />
+        <property name="ssh.host" value="localhost" />
+        <property name="fileTimestamp" value="${coord:formatTime(coord:nominalTime(), 'yyyy-MM-dd')}" />
+    </properties>
+
+    <workflow engine="oozie" path="/examples/apps/aggregator" />
+    <retry policy="exp-backoff" delay="minutes(5)" attempts="3" />
+    
+    <late-process policy="exp-backoff" delay="hours(1)">
+        <late-input input="input" workflow-path="/projects/bootcamp/workflow/lateinput" />
+    </late-process>
+</process>
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "requestId": "update\/default\/d6aaa328-6836-4818-a212-515bb43d8b86\n\n",
+    "message": "update\/default\/SampleProcess updated successfully with effective time [(local/2014-01-01T00:00Z)]\n\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>
+

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityValidate.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityValidate.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityValidate.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/EntityValidate.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,168 @@
+---++ POST  api/entities/validate/entity-type
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Validates the submitted entity.
+
+---++ Parameters
+:entity-type can be cluster, feed or process.
+
+---++ Results
+Result of the validation.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/validate/cluster
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<cluster xmlns="uri:falcon:cluster:0.1" name="primary-cluster" description="Primary Cluster" colo="west-coast">
+    <interfaces>
+        <interface type="readonly" endpoint="hftp://localhost:50070" version="1.1.1"/>
+        <interface type="write" endpoint="hdfs://localhost:9000" version="1.1.1"/>
+        <interface type="execute" endpoint="localhost:9001" version="1.1.1"/>
+        <interface type="workflow" endpoint="http://localhost:11000/oozie/" version="3.3.0"/>
+        <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true" version="5.4.3"/>
+    </interfaces>
+    <locations>
+        <location name="staging" path="/apps/falcon/staging"/>
+        <location name="temp" path="/tmp"/>
+        <location name="working" path="/apps/falcon/working"/>
+    </locations>
+</cluster>
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "requestId": "dd3f6c3a-a6f1-4c50-97fb-3f9a3f698e10",
+    "message": "Validated successfully (CLUSTER) primary-cluster",
+    "status": "SUCCEEDED"
+}
+</verbatim>
+
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/validate/feed
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Hourly sample input data -->
+
+<feed description="sample input data"
+      name="SampleInput" xmlns="uri:falcon:feed:0.1"
+      xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <groups>group</groups>
+    <frequency>hours(1)</frequency>
+    <late-arrival cut-off="hours(6)" />
+    <clusters>
+        <cluster name="primary-cluster" type="source">
+            <!--validity start="2009-01-01T00:00Z" end="2099-12-31T00:00Z" timezone="UTC" /-->
+            <validity start="2009-01-01T00:00Z" end="2099-12-31T00:00Z" />
+            <retention limit="months(24)" action="delete" />
+        </cluster>
+    </clusters>
+
+    <locations>
+        <location type="data" path="/projects/bootcamp/data/${YEAR}-${MONTH}-${DAY}-${HOUR}/SampleInput" />
+        <location type="stats" path="/projects/bootcamp/stats/SampleInput" />
+        <location type="meta" path="/projects/bootcamp/meta/SampleInput" />
+    </locations>
+
+    <ACL owner="suser" group="users" permission="0755" />
+
+    <schema location="/none" provider="none" />
+</feed>
+</verbatim>
+
+---+++ Result
+<verbatim>
+{
+    "requestId": "c85b190e-e653-493a-a863-d62de9c2e3b0",
+    "message": "Validated successfully (FEED) SampleInput",
+    "status": "SUCCEEDED"
+}
+</verbatim>
+
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/validate/feed
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Daily sample output data -->
+
+<feed description="sample output data" name="SampleOutput" xmlns="uri:falcon:feed:0.1"
+xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <groups>group</groups>
+    <frequency>hours(1)</frequency>
+    <late-arrival cut-off="hours(6)" />
+    <clusters>
+        <cluster name="primary-cluster" type="source">
+            <!--validity start="2009-01-01T00:00Z" end="2099-12-31T00:00Z" timezone="UTC" /-->
+            <validity start="2009-01-01T00:00Z" end="2099-12-31T00:00Z" />
+            <retention limit="months(24)" action="delete" />
+        </cluster>
+    </clusters>
+    <locations>
+        <location type="data" path="/projects/bootcamp/output/${YEAR}-${MONTH}-${DAY}-${HOUR}/SampleOutput" />
+        <location type="stats" path="/projects/bootcamp/stats/SampleOutput" />
+        <location type="meta" path="/projects/bootcamp/meta/SampleOutput" />
+    </locations>
+    <ACL owner="suser" group="users" permission="0755" />
+    <schema location="/none" provider="none" />
+</feed>
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "requestId": "60781732-460e-4c6c-ba86-a75fae574b05",
+    "message": "Validated successfully (FEED) SampleOutput",
+    "status": "SUCCEEDED"
+}
+</verbatim>
+
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/entities/validate/process
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Daily sample process. Runs at 6th hour every day. Input - last day's hourly data. Generates output for yesterday -->
+<process xmlns="uri:falcon:process:0.1" name="SampleProcess" >
+    <clusters>
+      <cluster name="primary-cluster">
+	<validity start="2012-04-03T06:00Z" end="2022-12-30T00:00Z" />
+      </cluster>
+    </clusters>
+
+    <parallel>1</parallel>
+    <order>FIFO</order>
+    <frequency>hours(1)</frequency>
+
+    <inputs>
+        <input name="input" feed="SampleInput" start="yesterday(0,0)" end="today(-1,0)" />
+    </inputs>
+
+    <outputs>
+        <output name="output" feed="SampleOutput" instance="yesterday(0,0)" />
+    </outputs>
+
+    <properties>
+        <property name="queueName" value="default" />
+        <property name="ssh.host" value="localhost" />
+        <property name="fileTimestamp" value="${coord:formatTime(coord:nominalTime(), 'yyyy-MM-dd')}" />
+    </properties>
+
+    <workflow engine="oozie" path="/examples/apps/aggregator" />
+    <retry policy="exp-backoff" delay="minutes(5)" attempts="3" />
+    
+    <late-process policy="exp-backoff" delay="hours(1)">
+        <late-input input="input" workflow-path="/projects/bootcamp/workflow/lateinput" />
+    </late-process>
+</process>
+</verbatim>
+
+---+++ Result
+<verbatim>
+{
+    "requestId": "e4a965c6-c7a2-41d9-ba08-2e77f1c43f57",
+    "message": "Validated successfully (PROCESS) SampleProcess",
+    "status": "SUCCEEDED"
+}
+</verbatim>

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/Graph.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/Graph.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/Graph.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/Graph.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,22 @@
+---++  GET api/graphs/lineage/serialize
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Dump the graph.
+
+---++ Parameters
+None.
+
+---++ Results
+Serialize graph to a file configured using *.falcon.graph.serialize.path in Custom startup.properties.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+GET http://localhost:15000/api/graphs/lineage/serialize
+</verbatim>
+---+++ Result
+None.

Added: incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/InstanceKill.twiki
URL: http://svn.apache.org/viewvc/incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/InstanceKill.twiki?rev=1627461&view=auto
==============================================================================
--- incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/InstanceKill.twiki (added)
+++ incubator/falcon/trunk/releases/0.5-incubating/src/site/twiki/restapi/InstanceKill.twiki Thu Sep 25 06:04:11 2014
@@ -0,0 +1,41 @@
+---++  POST /api/instance/kill/:entity-type/:entity-name
+   * <a href="#Description">Description</a>
+   * <a href="#Parameters">Parameters</a>
+   * <a href="#Results">Results</a>
+   * <a href="#Examples">Examples</a>
+
+---++ Description
+Kill a currently running instance.
+
+---++ Parameters
+   * :entity-type can either be a feed or a process.
+   * :entity-name is name of the entity.
+   * start start time of the entity.
+
+---++ Results
+Result of the kill operation.
+
+---++ Examples
+---+++ Rest Call
+<verbatim>
+POST http://localhost:15000/api/instance/kill/process/SampleProcess?colo=*&start=2012-04-03T07:00Z
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "instances": [
+        {
+            "details": "",
+            "endTime": "2013-10-21T15:26:59-07:00",
+            "startTime": "2013-10-21T15:19:57-07:00",
+            "cluster": "primary-cluster",
+            "logFile": "http:\/\/localhost:11000\/oozie?job=0000070-131021115933395-oozie-rgau-W",
+            "status": "KILLED",
+            "instance": "2012-04-03T07:00Z"
+        }
+    ],
+    "requestId": "default\/23b3cfee-ee22-40c0-825d-39c322587d5f\n",
+    "message": "default\/KILL\n",
+    "status": "SUCCEEDED"
+}
+</verbatim>