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 2013/11/12 12:05:26 UTC

[09/12] FALCON-85 Hive (HCatalog) integration. Contributed by Venkatesh Seetharam FALCON-163 Merge FALCON-85 branch into main line. Contributed by Venkatesh Seetharam

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/17f901a6/docs/src/site/twiki/EntitySpecification.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/EntitySpecification.twiki b/docs/src/site/twiki/EntitySpecification.twiki
index ef86055..f2cac4a 100644
--- a/docs/src/site/twiki/EntitySpecification.twiki
+++ b/docs/src/site/twiki/EntitySpecification.twiki
@@ -44,6 +44,14 @@ A workflow interface specifies the interface for workflow engine, example of its
 Falcon uses this interface to schedule the processes referencing this cluster on workflow engine defined here.
 
 <verbatim>
+<interface type="registry" endpoint="thrift://localhost:9083" version="0.11.0" />
+</verbatim>
+A registry interface specifies the interface for metadata catalog, such as Hive Metastore (or HCatalog).
+Falcon uses this interface to register/de-register partitions for a given database and table. Also,
+uses this information to schedule data availability events based on partitions in the workflow engine.
+Although Hive metastore supports both RPC and HTTP, Falcon comes with an implementation for RPC over thrift.
+
+<verbatim>
 <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true" version="5.4.6" />
 </verbatim>
 A messaging interface specifies the interface for sending feed availability messages, it's endpoint is broker url with tcp address.
@@ -76,41 +84,20 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
 </verbatim>
 A feed should have a unique name and this name is referenced by processes as input or output feed.
 
-<verbatim>
-   <partitions>
-        <partition name="country" />
-        <partition name="cluster" />
-    </partitions>
-</verbatim>
-A feed can define multiple partitions, if a referenced cluster defines partitions then the number of partitions in feed has to be equal to or more than the cluster partitions.
+---+++ Storage
+Falcon introduces a new abstraction to encapsulate the storage for a given feed which can either be
+expressed as a path on the file system, File System Storage or a table in a catalog such as Hive, Catalog Storage.
 
 <verbatim>
-    <groups>online,bi</groups>
+    <xs:choice minOccurs="1" maxOccurs="1">
+        <xs:element type="locations" name="locations"/>
+        <xs:element type="catalog-table" name="table"/>
+    </xs:choice>
 </verbatim>
-A feed specifies a list of comma separated groups, a group is a logical grouping of feeds and a group is said to be
-available if all the feeds belonging to a group are available. The frequency of all the feed which belong to the same group
-must be same.
 
-<verbatim>
-    <availabilityFlag>_SUCCESS</availabilityFlag>
-</verbatim>
-An availabilityFlag specifies the name of a file which when present/created in a feeds data directory, 
-the feed is termed as available. ex: _SUCCESS, if this element is ignored then Falcon would consider the presence of feed's
-data directory as feed availability.
+Feed should contain one of the two storage options. Locations on File System or Table in a Catalog.
 
-<verbatim>
-    <frequency>minutes(20)</frequency>
-</verbatim>
-A feed has a frequency which specifies the frequency by which this feed is generated. 
-ex: it can be generated every hour, every 5 minutes, daily, weekly etc.
-valid frequency type for a feed are minutes, hours, days, months. The values can be negative, zero or positive.
-
-<verbatim>
-    <late-arrival cut-off="hours(6)" />
-</verbatim>
-A late-arrival specifies the cut-off period till which the feed is expected to arrive late and should be honored be processes referring to it as input feed by rerunning the instances in case the data arrives late with in a cut-off period.
-The cut-off period is specified by expression frequency(times), ex: if the feed can arrive late
-upto 8 hours then late-arrival's cut-off="hours(8)"
+---++++ File System Storage
 
 <verbatim>
         <clusters>
@@ -131,7 +118,7 @@ Validity of a feed on cluster specifies duration for which this feed is valid on
 Retention specifies how long the feed is retained on this cluster and the action to be taken on the feed after the expiry of retention period.
 The retention limit is specified by expression frequency(times), ex: if feed should be retained for at least 6 hours then retention's limit="hours(6)".
 The field partitionExp contains partition tags. Number of partition tags has to be equal to number of partitions specified in feed schema. A partition tag can be a wildcard(*), a static string or an expression. Atleast one of the strings has to be an expression.
-Location specifies where the feed is available on this cluster. This is an optional parameter and path can be same or different from the global locations tag value ( it is mentioned outside the clusters tag ) . This tag provides the user to flexibility to have feed at different locations on different clusters. If this attribute is missing then the default global location is picked from the feed definition. Also the individual location tags data, stats, meta are optional. 
+Location specifies where the feed is available on this cluster. This is an optional parameter and path can be same or different from the global locations tag value ( it is mentioned outside the clusters tag ) . This tag provides the user to flexibility to have feed at different locations on different clusters. If this attribute is missing then the default global location is picked from the feed definition. Also the individual location tags data, stats, meta are optional.
 
 <verbatim>
  <location type="data" path="/projects/falcon/clicks" />
@@ -145,6 +132,93 @@ The granularity of date pattern in the path should be atleast that of a frequenc
 Other location type which are supported are stats and meta paths, if a process references a feed then the meta and stats
 paths are available as a property in a process.
 
+---++++ Catalog Storage (Table)
+
+A table tag specifies the table URI in the catalog registry as:
+<verbatim>
+catalog:$database-name:$table-name#partition-key=partition-value);partition-key=partition-value);*
+</verbatim>
+
+This is modeled as a URI (similar to an ISBN URI). It does not have any reference to Hive or HCatalog. Its quite
+generic so it can be tied to other implementations of a catalog registry. The catalog implementation specified
+in the startup config provides implementation for the catalog URI.
+
+Top-level partition has to be a dated pattern and the granularity of date pattern should be at least that
+of a frequency of a feed.
+
+<verbatim>
+    <xs:complexType name="catalog-table">
+        <xs:annotation>
+            <xs:documentation>
+                catalog specifies the uri of a Hive table along with the partition spec.
+                uri="catalog:$database:$table#(partition-key=partition-value);+"
+                Example: catalog:logs-db:clicks#ds=${YEAR}-${MONTH}-${DAY}
+            </xs:documentation>
+        </xs:annotation>
+        <xs:attribute type="xs:string" name="uri" use="required"/>
+    </xs:complexType>
+</verbatim>
+
+Examples:
+<verbatim>
+<table uri="catalog:default:clicks#ds=${YEAR}-${MONTH}-${DAY}-${HOUR};region=${region}" />
+<table uri="catalog:src_demo_db:customer_raw#ds=${YEAR}-${MONTH}-${DAY}-${HOUR}" />
+<table uri="catalog:tgt_demo_db:customer_bcp#ds=${YEAR}-${MONTH}-${DAY}-${HOUR}" />
+</verbatim>
+
+---+++ Partitions
+
+<verbatim>
+   <partitions>
+        <partition name="country" />
+        <partition name="cluster" />
+    </partitions>
+</verbatim>
+A feed can define multiple partitions, if a referenced cluster defines partitions then the number of partitions in feed has to be equal to or more than the cluster partitions.
+
+*Note:* This will only apply for FileSystem storage but not Table storage as partitions are defined and maintained in
+Hive (Hcatalog) registry.
+
+---+++ Groups
+
+<verbatim>
+    <groups>online,bi</groups>
+</verbatim>
+A feed specifies a list of comma separated groups, a group is a logical grouping of feeds and a group is said to be
+available if all the feeds belonging to a group are available. The frequency of all the feed which belong to the same group
+must be same.
+
+---+++ Availability Flags
+
+<verbatim>
+    <availabilityFlag>_SUCCESS</availabilityFlag>
+</verbatim>
+An availabilityFlag specifies the name of a file which when present/created in a feeds data directory, 
+the feed is termed as available. ex: _SUCCESS, if this element is ignored then Falcon would consider the presence of feed's
+data directory as feed availability.
+
+---+++ Frequency
+
+<verbatim>
+    <frequency>minutes(20)</frequency>
+</verbatim>
+A feed has a frequency which specifies the frequency by which this feed is generated. 
+ex: it can be generated every hour, every 5 minutes, daily, weekly etc.
+valid frequency type for a feed are minutes, hours, days, months. The values can be negative, zero or positive.
+
+---+++ Late Arrival
+
+<verbatim>
+    <late-arrival cut-off="hours(6)" />
+</verbatim>
+A late-arrival specifies the cut-off period till which the feed is expected to arrive late and should be honored be processes referring to it as input feed by rerunning the instances in case the data arrives late with in a cut-off period.
+The cut-off period is specified by expression frequency(times), ex: if the feed can arrive late
+upto 8 hours then late-arrival's cut-off="hours(8)"
+
+*Note:* This will only apply for FileSystem storage but not Table storage until a future time.
+
+---++++ Custom Properties
+
 <verbatim>
     <properties>
         <property name="tmpFeedPath" value="tmpFeedPathValue" />
@@ -155,7 +229,10 @@ paths are available as a property in a process.
         <property name="parallel" value="3"/>
     </properties>
 </verbatim>
-A key-value pair, which are propagated to the workflow engine. "queueName" and "jobPriority" are special properties available to user to specify the hadoop job queue and priority, the same value is used by Falcons launcher job. "timeout" and "parallel" are other special properties which decides replication instance's timeout value while waiting for the feed instance and parallel decides the concurrent replication instances that can run at any given time.
+A key-value pair, which are propagated to the workflow engine. "queueName" and "jobPriority" are special properties
+available to user to specify the hadoop job queue and priority, the same value is used by Falcons launcher job.
+"timeout" and "parallel" are other special properties which decides replication instance's timeout value while
+waiting for the feed instance and parallel decides the concurrent replication instances that can run at any given time.
  
 ---++ Process Specification
 A process defines configuration for a workflow. A workflow is a directed acyclic graph(DAG) which defines the job for the workflow engine. A process definition defines  the configurations required to run the workflow job. For example, process defines the frequency at which the workflow should run, the clusters on which the workflow should run, the inputs and outputs for the workflow, how the workflow failures should be handled, how the late inputs should be handled and so on.  
@@ -306,11 +383,70 @@ Example:
 ...
 </process>
 </verbatim>
-The input for the workflow is a hourly feed and takes 0th and 1st hour data of today(the day when the workflow runs). If the workflow is running for 2012-03-01T06:40Z, the inputs are /projects/bootcamp/feed1/2012-03-01-00/*/US and /projects/bootcamp/feed1/2012-03-01-01/*/US. The property for this input is
+The input for the workflow is a hourly feed and takes 0th and 1st hour data of today(the day when the workflow runs).
+If the workflow is running for 2012-03-01T06:40Z, the inputs are /projects/bootcamp/feed1/2012-03-01-00/*/US and
+/projects/bootcamp/feed1/2012-03-01-01/*/US. The property for this input is
 input1=/projects/bootcamp/feed1/2012-03-01-00/*/US,/projects/bootcamp/feed1/2012-03-01-01/*/US
 
+Also, feeds with Hive table storage can be used as inputs to a process. Several parameters from inputs are passed as
+params to the user workflow or pig script.
+
+<verbatim>
+    ${wf:conf('falcon_input_database')} - database name associated with the feed for a given input
+    ${wf:conf('falcon_input_table')} - table name associated with the feed for a given input
+    ${wf:conf('falcon_input_catalog_url')} - Hive metastore URI for this input feed
+    ${wf:conf('falcon_input_partition_filter_pig')} - value of ${coord:dataInPartitionFilter('$input', 'pig')}
+    ${wf:conf('falcon_input_partition_filter_hive')} - value of ${coord:dataInPartitionFilter('$input', 'hive')}
+    ${wf:conf('falcon_input_partition_filter_java')} - value of ${coord:dataInPartitionFilter('$input', 'java')}
+</verbatim>
+
+*NOTE:* input is the name of the input configured in the process, which is input.getName().
+<verbatim><input name="input" feed="clicks-raw-table" start="yesterday(0,0)" end="yesterday(20,0)"/></verbatim>
+
+Example workflow configuration:
+
+<verbatim>
+<configuration>
+  <property>
+    <name>falcon_input_database</name>
+    <value>falcon_db</value>
+  </property>
+  <property>
+    <name>falcon_input_table</name>
+    <value>input_table</value>
+  </property>
+  <property>
+    <name>falcon_input_catalog_url</name>
+    <value>thrift://localhost:29083</value>
+  </property>
+  <property>
+    <name>falcon_input_storage_type</name>
+    <value>TABLE</value>
+  </property>
+  <property>
+    <name>feedInstancePaths</name>
+    <value>hcat://localhost:29083/falcon_db/output_table/ds=2012-04-21-00</value>
+  </property>
+  <property>
+    <name>falcon_input_partition_filter_java</name>
+    <value>(ds='2012-04-21-00')</value>
+  </property>
+  <property>
+    <name>falcon_input_partition_filter_hive</name>
+    <value>(ds='2012-04-21-00')</value>
+  </property>
+  <property>
+    <name>falcon_input_partition_filter_pig</name>
+    <value>(ds=='2012-04-21-00')</value>
+  </property>
+  ...
+</configuration>
+</verbatim>
+
+
 ---++++ Optional Inputs
-User can metion one or more inputs as optional inputs. In such cases the job does not wait on those inputs which are mentioned as optional. If they are present it considers them otherwise continue with the comlpulsury ones. 
+User can mention one or more inputs as optional inputs. In such cases the job does not wait on those inputs which are
+mentioned as optional. If they are present it considers them otherwise continue with the compulsory ones.
 Example:
 <verbatim>
 <feed name="feed1">
@@ -335,6 +471,8 @@ Example:
 </process>
 </verbatim>
 
+*Note:* This is only supported for FileSystem storage but not Table storage at this point.
+
 
 ---++++ Outputs
 Outputs define the output data that is generated by the workflow. A process can define 0 or more outputs. Each output is mapped to a feed and the output path is picked up from feed definition. The output instance that should be generated is specified in terms of [[FalconDocumentation][EL expression]].
@@ -372,11 +510,57 @@ Example:
 ...
 </process>
 </verbatim>
-The output of the workflow is feed instance for today. If the workflow is running for 2012-03-01T06:40Z, the workflow generates output /projects/bootcamp/feed2/2012-03-01. The property for this output that is available for workflow is:
-output1=/projects/bootcamp/feed2/2012-03-01
+The output of the workflow is feed instance for today. If the workflow is running for 2012-03-01T06:40Z,
+the workflow generates output /projects/bootcamp/feed2/2012-03-01. The property for this output that is available
+for workflow is: output1=/projects/bootcamp/feed2/2012-03-01
+
+Also, feeds with Hive table storage can be used as outputs to a process. Several parameters from outputs are passed as
+params to the user workflow or pig script.
+<verbatim>
+    ${wf:conf('falcon_output_database')} - database name associated with the feed for a given output
+    ${wf:conf('falcon_output_table')} - table name associated with the feed for a given output
+    ${wf:conf('falcon_output_catalog_url')} - Hive metastore URI for the given output feed
+    ${wf:conf('falcon_output_dataout_partitions')} - value of ${coord:dataOutPartitions('$output')}
+</verbatim>
+
+*NOTE:* output is the name of the output configured in the process, which is output.getName().
+<verbatim><output name="output" feed="clicks-summary-table" instance="today(0,0)"/></verbatim>
+
+Example workflow configuration:
+
+<verbatim>
+<configuration>
+  <property>
+    <name>falcon_output_database</name>
+    <value>falcon_db</value>
+  </property>
+  <property>
+    <name>falcon_output_table</name>
+    <value>output_table</value>
+  </property>
+  <property>
+    <name>falcon_output_catalog_url</name>
+    <value>thrift://localhost:29083</value>
+  </property>
+  <property>
+    <name>falcon_output_storage_type</name>
+    <value>TABLE</value>
+  </property>
+  <property>
+    <name>feedInstancePaths</name>
+    <value>hcat://localhost:29083/falcon_db/output_table/ds=2012-04-21-00</value>
+  </property>
+  <property>
+    <name>falcon_output_dataout_partitions</name>
+    <value>'ds=2012-04-21-00'</value>
+  </property>
+  ....
+</configuration>
+</verbatim>
 
 ---++++ Properties
-The properties are key value pairs that are passed to the workflow. These properties are optional and can be used in workflow to parameterize the workflow.
+The properties are key value pairs that are passed to the workflow. These properties are optional and can be used
+in workflow to parameterize the workflow.
 Synatx:
 <verbatim>
 <process name="[process name]">
@@ -450,6 +634,29 @@ Example:
 This defines the workflow engine to be pig and the pig script is defined at
 /projects/bootcamp/pig.script.
 
+Feeds with Hive table storage will send one more parameter apart from the general ones:
+<verbatim>$input_filter</verbatim>
+
+---+++++ Hive
+
+Falcon also adds the Hive engine as part of Hive Integration which enables users to embed a Hive script as a process.
+This would enable users to create materialized queries in a declarative way.
+
+Example:
+<verbatim>
+<process name="sample-process">
+...
+    <workflow engine="hive" path="/projects/bootcamp/hive-script.hql"/>
+...
+</process>
+</verbatim>
+
+This defines the workflow engine to be hive and the hive script is defined at
+/projects/bootcamp/hive-script.hql.
+
+Feeds with Hive table storage will send one more parameter apart from the general ones:
+<verbatim>$input_filter</verbatim>
+
 ---++++ Retry
 Retry policy defines how the workflow failures should be handled. Two retry policies are defined: backoff and exp-backoff(exponential backoff). Depending on the delay and number of attempts, the workflow is re-tried after specific intervals.
 Syntax:
@@ -512,3 +719,5 @@ Example:
 </process>
 </verbatim>
 This late handling specifies that late data detection should run at feed's late cut-off which is 6 hours in this case. If there is late data, Falcon should run the workflow specified at /projects/bootcamp/workflow/lateinput1/workflow.xml
+
+*Note:* This is only supported for FileSystem storage but not Table storage at this point.

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/17f901a6/docs/src/site/twiki/FalconDocumentation.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/FalconDocumentation.twiki b/docs/src/site/twiki/FalconDocumentation.twiki
index e8fe7ef..71affa0 100644
--- a/docs/src/site/twiki/FalconDocumentation.twiki
+++ b/docs/src/site/twiki/FalconDocumentation.twiki
@@ -85,6 +85,55 @@ individual operations performed are recorded into a transaction journal. This jo
 the overall user action. In some cases, it is not possible to undo the action. In such cases, Falcon attempts
 to keep the system in an consistent state.
 
+---+++ Storage
+Falcon introduces a new abstraction to encapsulate the storage for a given feed which can either be
+expressed as a path on the file system, File System Storage or a table in a catalog such as Hive, Catalog Storage.
+
+<verbatim>
+    <xs:choice minOccurs="1" maxOccurs="1">
+        <xs:element type="locations" name="locations"/>
+        <xs:element type="catalog-table" name="table"/>
+    </xs:choice>
+</verbatim>
+
+Feed should contain one of the two storage options. Locations on File System or Table in a Catalog.
+
+---++++ File System Storage
+
+This is expressed as a location on the file system. Location specifies where the feed is available on this cluster.
+A location tag specifies the type of location like data, meta, stats and the corresponding paths for them.
+A feed should at least define the location for type data, which specifies the HDFS path pattern where the feed is
+generated periodically. ex: type="data" path="/projects/TrafficHourly/${YEAR}-${MONTH}-${DAY}/traffic"
+The granularity of date pattern in the path should be at least that of a frequency of a feed.
+
+<verbatim>
+ <location type="data" path="/projects/falcon/clicks" />
+ <location type="stats" path="/projects/falcon/clicksStats" />
+ <location type="meta" path="/projects/falcon/clicksMetaData" />
+</verbatim>
+
+---++++ Catalog Storage (Table)
+
+A table tag specifies the table URI in the catalog registry as:
+<verbatim>
+catalog:$database-name:$table-name#partition-key=partition-value);partition-key=partition-value);*
+</verbatim>
+
+This is modeled as a URI (similar to an ISBN URI). It does not have any reference to Hive or HCatalog. Its quite
+generic so it can be tied to other implementations of a catalog registry. The catalog implementation specified
+in the startup config provides implementation for the catalog URI.
+
+Top-level partition has to be a dated pattern and the granularity of date pattern should be at least that
+of a frequency of a feed.
+
+Examples:
+<verbatim>
+<table uri="catalog:default:clicks#ds=${YEAR}-${MONTH}-${DAY}-${HOUR};region=${region}" />
+<table uri="catalog:src_demo_db:customer_raw#ds=${YEAR}-${MONTH}-${DAY}-${HOUR}" />
+<table uri="catalog:tgt_demo_db:customer_bcp#ds=${YEAR}-${MONTH}-${DAY}-${HOUR}" />
+</verbatim>
+
+
 ---++ Entity Management actions
 All the following operation can also be done using [[restapi/ResourceList][Falcon's RESTful API]].
 
@@ -171,7 +220,7 @@ In all the cases where your request is syntactically correct but logically not,
 
 ---++ Retention
 In coherence with it's feed lifecycle management philosophy, Falcon allows the user to retain data in the system
-for a specific period of time for a scheduled feed. The user can specify the retention period in the respective 
+for a specific period of time for a scheduled feed. The user can specify the retention period in the respective
 feed/data xml in the following manner for each cluster the feed can belong to :
 <verbatim>
 <clusters>
@@ -187,6 +236,8 @@ The 'limit' attribute can be specified in units of minutes/hours/days/months, an
 be attached to it. It essentially instructs the system to retain data spanning from the current moment to the time specified
 in the attribute spanning backwards in time. Any data beyond the limit (past/future) is erased from the system.
 
+With the integration of Hive, Falcon also provides retention for tables in Hive catalog.
+
 ---+++ Example:
 If retention period is 10 hours, and the policy kicks in at time 't', the data retained by system is essentially the
 one falling in between [t-10h,t]. Any data in the boundaries [-�,t-10h) and (t,�] is removed from the system.
@@ -229,18 +280,27 @@ Ideally, the feeds data path should have the same granularity as that for freque
 </verbatim>
 
 If more than 1 source cluster is defined, then partition expression is compulsory, a partition can also have a constant.
-The expression is required to avoid copying data from different source location to the same target location, also only the data in the partition is considered for replication if it is present. The partitions defined in the cluster should be less than or equal to the number of partition declared in the feed definition.
+The expression is required to avoid copying data from different source location to the same target location,
+also only the data in the partition is considered for replication if it is present. The partitions defined in the
+cluster should be less than or equal to the number of partition declared in the feed definition.
 
-Falcon uses pull based replication mechanism, meaning in every target cluster, for a given source cluster, a coordinator is scheduled which pulls the data using distcp from source cluster. So in the above example, 2 coordinators are scheduled in backupCluster, one which pulls the data from sourceCluster1 and another from sourceCluster2.
-Also, for every feed instance which is replicated Falcon sends a JMS message on success or failure of replication instance.
+Falcon uses pull based replication mechanism, meaning in every target cluster, for a given source cluster,
+a coordinator is scheduled which pulls the data using distcp from source cluster. So in the above example,
+2 coordinators are scheduled in backupCluster, one which pulls the data from sourceCluster1 and another
+from sourceCluster2. Also, for every feed instance which is replicated Falcon sends a JMS message on success or
+failure of replication instance.
 
-Replication can be scheduled with the past date, the time frame considered for replication is the minimum overlapping window of start and end time of source and target cluster, ex: if s1 and e1 is the start and end time of source cluster respectively,
-and s2 and e2 of target cluster, then the coordinator is scheduled in target cluster with start time max(s1,s2) and min(e1,e2).
+Replication can be scheduled with the past date, the time frame considered for replication is the minimum
+overlapping window of start and end time of source and target cluster, ex: if s1 and e1 is the start and end time
+of source cluster respectively, and s2 and e2 of target cluster, then the coordinator is scheduled in
+target cluster with start time max(s1,s2) and min(e1,e2).
 
-A feed can also optionally specify the delay for replication instance in the cluster tag, the delay governs the replication instance delays. If the frequency of the feed is hours(2) and delay is hours(1), then the replication instance will run every 2 hours and replicates data with an offset of 1 hour, i.e. at
-09:00 UTC, feed instance which is eligible for replication is 08:00; and 11:00 UTC, feed instance of 10:00 UTC is eligible and so on.
+A feed can also optionally specify the delay for replication instance in the cluster tag, the delay governs the
+replication instance delays. If the frequency of the feed is hours(2) and delay is hours(1), then the replication
+instance will run every 2 hours and replicates data with an offset of 1 hour, i.e. at 09:00 UTC, feed instance which
+is eligible for replication is 08:00; and 11:00 UTC, feed instance of 10:00 UTC is eligible and so on.
 
----+++ Where is the feed path defined?
+---+++ Where is the feed path defined for File System Storage?
 
 It's defined in the feed xml within the location tag.
 
@@ -274,6 +334,22 @@ may have a different feed path.
     </clusters>
 </verbatim>
 
+---+++ Hive Table Replication
+
+With the integration of Hive, Falcon adds table replication of Hive catalog tables. Replication will be triggered
+for a partition when the partition is complete at the source.
+
+   * 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 DistCp 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)
+
 
 ---+++ Relation between feed's retention limit and feed's late arrival cut off period:
 
@@ -455,6 +531,9 @@ explicitly set the feed names in late-input which needs to be checked for late d
    </late-process>
 </verbatim>
 
+*NOTE:* Feeds configured with table storage does not support late input data handling at this point. This will be
+made available in the near future.
+
 ---++ Idempotency
 All the operations in Falcon are Idempotent. That is if you make same request to the falcon server / prism again you will get a SUCCESSFUL return if it was SUCCESSFUL in the first attempt. For example, you submit a new process / feed and get SUCCESSFUL message return. Now if you run the same command / api request on same entity you will again get a SUCCESSFUL message. Same is true for other operations like schedule, kill, suspend and resume.
 Idempotency also by takes care of the condition when request is sent through prism and fails on one or more servers. For example prism is configured to send request to 3 servers. First user sends a request to SUBMIT a process on all 3 of them, and receives a response SUCCESSFUL from all of them. Then due to some issue one of the servers goes down, and user send a request to schedule the submitted process. This time he will receive a response with PARTIAL status and a FAILURE message from the server that has gone down. If the users check he will find the process would have been started and running on the 2 SUCCESSFUL servers. Now the issue with server is figured out and it is brought up. Sending the SCHEDULE request again through prism will result in a SUCCESSFUL response from prism as well as other three servers, but this time PROCESS will be SCHEDULED only on the server which had failed earlier and other two will keep running as before. 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/17f901a6/docs/src/site/twiki/HiveIntegration.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/HiveIntegration.twiki b/docs/src/site/twiki/HiveIntegration.twiki
new file mode 100644
index 0000000..0ee571e
--- /dev/null
+++ b/docs/src/site/twiki/HiveIntegration.twiki
@@ -0,0 +1,342 @@
+---+ 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 DistCp 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>
+
+---+++ Hive
+
+   * [[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>
+
+
+---++ 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>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/17f901a6/docs/src/site/twiki/InstallationSteps.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/InstallationSteps.twiki b/docs/src/site/twiki/InstallationSteps.twiki
index ab23d49..223242b 100644
--- a/docs/src/site/twiki/InstallationSteps.twiki
+++ b/docs/src/site/twiki/InstallationSteps.twiki
@@ -35,22 +35,27 @@ Tar is structured as follows
    |- falcon
    |- falcon-start
    |- falcon-stop
+   |- falcon-config.sh
+   |- service-start.sh
+   |- service-stop.sh
 |- conf
    |- startup.properties
    |- runtime.properties
    |- client.properties
    |- log4j.xml
-|- src
+   |- falcon-env.sh
 |- docs
-|- apidocs
 |- client
    |- lib (client support libs)
 |- server
    |- webapp
       |- falcon.war
-|- logs (application log files & temp data files)
-   |- falcon.pid
-   
+|- hadooplibs
+|- README
+|- NOTICE.txt
+|- LICENSE.txt
+|- DISCLAIMER.txt
+|- CHANGES.txt
 </verbatim>
 
 *Distributed Mode*
@@ -62,20 +67,40 @@ mvn clean assembly:assembly -DskipTests -DskipCheck=true -P distributed -Dhadoop
 
 </verbatim>
 
-This generates 3 tars - {project dir}/target/falcon-${project.version}-prism.tar.gz, {project dir}/target/falcon-${project.version}-server.tar.gz and {project dir}/target/falcon-${project.version}-client.tar.gz
+Tar can be found in {project dir}/target/falcon-distributed-${project.version}-server.tar.gz
+
+Tar is structured as follows
 
-Package structure of prism and server tars is same as that of the embedded package with one difference that prism package contains prism war instead of falcon war. Client package 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
@@ -83,7 +108,54 @@ Package structure of prism and server tars is same as that of the embedded packa
 *Installing falcon*
 <verbatim>
 tar -xzvf {falcon package}
-cd falcon-server-${project.version} or cd falcon-prism-${project.version}
+cd falcon-distributed-${project.version} or falcon-${project.version}
+</verbatim>
+
+*Configuring Falcon*
+
+By default config directory used by faclon is {package dir}/conf. To override this set environemnt 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 enviornment 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 exectuted. The following enviornment 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 prisim service.
+#export FALCON_PRISM_OPTS=
+
+# java heap size we want to set for the prisim 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 locaion of the installed software
+#export FALCON_HOME_DIR=
+
+# Where log files are stored. Defatult is logs directory under the base install location
+#export FALCON_LOG_DIR=
+
+# Where pid files are stored. Defatult is logs directory under the base install location
+#export FALCON_PID_DIR=
+
+# where the falcon active mq data is stored. Defatult 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*
@@ -147,5 +219,5 @@ mkdir target/package
 src/bin/pacakge.sh <<hadoop-version>>
 
 >> ex. src/bin/pacakge.sh 1.1.2 or src/bin/pacakge.sh 0.20.2-cdh3u5
->> oozie bundle available in target/package/oozie-3.2.0-incubating/distro/target/oozie-3.2.2-distro.tar.gz
-</verbatim>
\ No newline at end of file
+>> oozie bundle available in target/package/oozie-4.0.0/distro/target/oozie-4.0.0-distro.tar.gz
+</verbatim>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/17f901a6/docs/src/site/twiki/OnBoarding.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/OnBoarding.twiki b/docs/src/site/twiki/OnBoarding.twiki
index 639d741..4fa5893 100644
--- a/docs/src/site/twiki/OnBoarding.twiki
+++ b/docs/src/site/twiki/OnBoarding.twiki
@@ -1,6 +1,7 @@
 ---++ 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.

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/17f901a6/docs/src/site/twiki/index.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/index.twiki b/docs/src/site/twiki/index.twiki
index 2fcd4d8..81c4c3e 100644
--- a/docs/src/site/twiki/index.twiki
+++ b/docs/src/site/twiki/index.twiki
@@ -8,11 +8,11 @@ management on hadoop clusters.
 
    * Establishes relationship between various data and processing elements on a Hadoop environment
 
-   * Feed management services such as feed retention, replications across clusters, archival etc 
+   * Feed management services such as feed retention, replications across clusters, archival etc.
 
    * Easy to onboard new workflows/pipelines, with support for late data handling, retry policies
 
-   * Integration with metastore/catalog
+   * Integration with metastore/catalog such as Hive/HCatalog
 
    * Provide notification to end customer based on availability of feed groups
      (logical group of related feeds, which are likely to be used together)
@@ -22,11 +22,16 @@ management on hadoop clusters.
 ---+ Getting Started
 
 Start with these simple steps to install an falcon instance [[InstallationSteps][Simple setup]]. Also refer
-to Falcon architecture and documentation in [[FalconDocumentation][Documentation]]. [[OnBoarding][On boarding]] describes steps to on-board a pipeline to Falcon. It also gives a sample pipeline for reference. [[EntitySpecification][Entity Specification]] gives complete details of all Falcon entities.
+to Falcon architecture and documentation in [[FalconDocumentation][Documentation]]. [[OnBoarding][On boarding]]
+describes steps to on-board a pipeline to Falcon. It also gives a sample pipeline for reference.
+[[EntitySpecification][Entity Specification]] gives complete details of all Falcon entities.
 
 [[FalconCLI][Falcon CLI]] implements [[restapi/ResourceList][Falcon's RESTful API]] and
 describes various options for the command line utility provided by Falcon.
 
+Falcon provides OOTB [[HiveIntegration][lifecycle management for Tables in Hive (HCatalog)]]
+such as table replication for BCP and table eviction.
+
 #LicenseInfo
 ---+ Licensing Information