You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by vi...@apache.org on 2013/02/25 22:42:09 UTC

svn commit: r1449911 [8/8] - in /oozie/trunk: ./ client/src/main/java/org/apache/oozie/cli/ client/src/main/java/org/apache/oozie/client/ client/src/main/java/org/apache/oozie/client/rest/ client/src/test/java/org/apache/oozie/client/rest/ core/ core/s...

Modified: oozie/trunk/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
URL: http://svn.apache.org/viewvc/oozie/trunk/docs/src/site/twiki/WorkflowFunctionalSpec.twiki?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/docs/src/site/twiki/WorkflowFunctionalSpec.twiki (original)
+++ oozie/trunk/docs/src/site/twiki/WorkflowFunctionalSpec.twiki Mon Feb 25 21:42:07 2013
@@ -132,6 +132,11 @@ schema 0.4
 
    * Opened [[https://issues.apache.org/jira/browse/HADOOP-5303][JIRA HADOOP-5303]]
 
+---+++!! 27/DEC/2012:
+
+   * Added information on dropping hcatalog table partitions in prepare block
+   * Added hcatalog EL functions section
+
 ---++ 0 Definitions
 
 *Action:* An execution/computation task (Map-Reduce job, Pig job, a shell command). It can also be referred as task or
@@ -668,9 +673,12 @@ Pipe properties can be overridden by spe
 </workflow-app>
 </verbatim>
 
-The =prepare= element, if present, indicates a list of path do delete before starting the job. This should be used
-exclusively for directory cleanup for the job to be executed. The delete operation will be performed in the
- =fs.default.name= filesystem.
+The =prepare= element, if present, indicates a list of paths to delete before starting the job. This should be used
+exclusively for directory cleanup or dropping of hcatalog table partitions for the job to be executed. The delete operation
+will be performed in the =fs.default.name= filesystem for hdfs URIs. The format to specify a hcatalog table partition URI is
+hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value].
+In case of a hcatalog URI, the hive-site.xml needs to be shipped using =file= tag and the hcatalog and hive jars
+need to be placed in workflow lib directory or specified using =archive= tag.
 
 The =job-xml= element, if present, must refer to a Hadoop JobConf =job.xml= file bundled in the workflow application.
 The =job-xml= element is optional and as of schema 0.4, multiple =job-xml= elements are allowed in order to specify multiple Hadoop JobConf =job.xml= files.
@@ -814,9 +822,9 @@ The workflow job will wait until the pig
 The =pig= action has to be configured with the job-tracker, name-node, pig script and the necessary parameters and
 configuration to run the Pig job.
 
-A =pig= action can be configured to perform HDFS files/directories cleanup before starting the Pig job. This capability
-enables Oozie to retry a Pig job in the situation of a transient failure (Pig creates temporary directories for
-intermediate data, thus a retry without cleanup would fail).
+A =pig= action can be configured to perform HDFS files/directories cleanup or HCatalog partitions cleanup before
+starting the Pig job. This capability enables Oozie to retry a Pig job in the situation of a transient failure (Pig
+creates temporary directories for intermediate data, thus a retry without cleanup would fail).
 
 Hadoop JobConf properties can be specified in a JobConf XML file bundled with the workflow application or they can be
 indicated inline in the =pig= action configuration.
@@ -913,8 +921,12 @@ section [#FilesAchives][Adding Files and
 </workflow-app>
 </verbatim>
 
-The =prepare= element, if present, indicates a list of path do delete before starting the job. This should be used
-exclusively for directory cleanup for the job to be executed.
+The =prepare= element, if present, indicates a list of paths to delete before starting the job. This should be used
+exclusively for directory cleanup or dropping of hcatalog table partitions for the job to be executed.
+The format to specify a hcatalog table partition URI is
+hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value].
+In case of a hcatalog URI, the hive-site.xml needs to be shipped using =file= tag and the hcatalog and hive jars
+need to be placed in workflow lib directory or specified using =archive= tag.
 
 The =job-xml= element, if present, must refer to a Hadoop JobConf =job.xml= file bundled in the workflow application.
 The =job-xml= element is optional and as of schema 0.4, multiple =job-xml= elements are allowed in order to specify multiple Hadoop JobConf =job.xml= files.
@@ -1304,9 +1316,10 @@ To indicate an =error= action transition
 The main Java class must not call =System.exit(int n)= as this will make the =java= action to do an =error= transition
 regardless of the used exit code.
 
-A =java= action can be configured to perform HDFS files/directories cleanup before starting the Java application. This
-capability enables Oozie to retry a Java application in the situation of a transient or non-transient failure (This can
-be used to cleanup any temporary data which may have been created by the Java application in case of failure).
+A =java= action can be configured to perform HDFS files/directories cleanup or HCatalog partitions cleanup before
+starting the Java application. This capability enables Oozie to retry a Java application in the situation of a transient
+or non-transient failure (This can be used to cleanup any temporary data which may have been created by the Java
+application in case of failure).
 
 A =java= action can create a Hadoop configuration. The Hadoop configuration is made available as a local file to the
 Java application in its running directory, the file name is =oozie-action.conf.xml=. Similar to =map-reduce= and
@@ -1377,8 +1390,12 @@ be assigned to it. The queue name must b
 </workflow-app>
 </verbatim>
 
-The =prepare= element, if present, indicates a list of path do delete before starting the Java application. This should
-be used exclusively for directory cleanup for the Java application to be executed.
+The =prepare= element, if present, indicates a list of paths to delete before starting the Java application. This should
+be used exclusively for directory cleanup or dropping of hcatalog table partitions for the Java application to be executed.
+The format to specify a hcatalog table partition URI is
+hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value].
+In case of a hcatalog URI, the hive-site.xml needs to be shipped using =file= tag and the hcatalog and hive jars
+need to be placed in workflow lib directory or specified using =archive= tag.
 
 The =java-opts= element, if present, contains the command line parameters which are to be used to start the JVM that
 will execute the Java application. Using this element is equivalent to use the =mapred.child.java.opts= configuration
@@ -2015,6 +2032,16 @@ It returns the size in bytes of specifie
 
 It returns the block size in bytes of specified file. If the path is not a file, or if it does not exist it returns -1.
 
+---++++ 4.2.8 HCatalog EL Functions
+
+For all the functions in this section the URI must be a hcatalog URI identifying a set of partitions in a table.
+The format to specify a hcatalog table partition URI is
+hcat://[metastore server]:[port]/[database name]/[table name]/[partkey1]=[value];[partkey2]=[value]. For example: <pre>hcat://foo:8020/mydb/mytable/region=us;dt=20121212</pre>
+
+*boolean hcat:exists(String uri)*
+
+It returns =true= or =false= based on if the partitions in the table exists or not.
+
 #WorkflowNotifications
 ---++ 5 Oozie Notifications
 
@@ -2268,7 +2295,8 @@ and Mapreduce Streaming share library di
 
 Oozie bundles a share library for specific versions of streaming, pig, hive, sqoop, distcp actions. These versions
 of streaming, pig, hive, sqoop and distcp have been tested and verified to work correctly with the version of Oozie
-that includes them.
+that includes them. Oozie also bundles a separate share library for hcatalog, which can be used with pig, hive and java
+actions.
 
 In addition, Oozie provides a mechanism to override the action share library JARs to allow using an alternate version
 of of the action JARs.
@@ -2276,14 +2304,18 @@ of of the action JARs.
 This mechanism enables Oozie administrators to patch share library JARs, to include alternate versios of the share
 libraries, to provide acess to more than one version at the same time.
 
-The sharelibrary override is supported at server level and at job level. The share library name is resolved using
-the following precedence order:
+The share library override is supported at server level and at job level. The share library directory names are resolved
+using the following precedence order:
 
    *  action.sharelib.for.#ACTIONTYPE# in the action configuration
    *  action.sharelib.for.#ACTIONTYPE# in the job configuration
    *  action.sharelib.for.#ACTIONTYPE# in the oozie server configuration
    *  action's =ActionExecutor getDefaultShareLibName()= method
 
+More than one share library directory name can be specified for an action by using a comma separated list. For example:
+When using HCatLoader and HCatStorer in pig, =action.sharelib.for.pig= can be set to =pig,hcatalog= to include both pig
+and hcatalog jars.
+
 ---++ 18 User-Retry for Workflow Actions (since Oozie 3.1)
 
 Oozie provides User-Retry capabilities when an action is in =ERROR= or =FAILED= state.

Added: oozie/trunk/examples/src/main/apps/hcatalog/README
URL: http://svn.apache.org/viewvc/oozie/trunk/examples/src/main/apps/hcatalog/README?rev=1449911&view=auto
==============================================================================
--- oozie/trunk/examples/src/main/apps/hcatalog/README (added)
+++ oozie/trunk/examples/src/main/apps/hcatalog/README Mon Feb 25 21:42:07 2013
@@ -0,0 +1,22 @@
+Running the coordinator example with HCatalog (End-to-end test)
+
+1. Setup services - Hive with JMS (e.g. ActiveMQ Server), HCatalog, database e.g. MySQL, and of course Hadoop
+2. This example points to hive.metastore.uris=thrift://localhost:11002. Change it in job.properties if required
+3. Create 2 tables 'invites' (input) and 'oozie' (output) with this structure: "create table invites (foo INT, bar INT) partitioned by (ds STRING, region STRING)"
+4. Oozie distro should be built this way $> bin/mkdistro.sh -Dhcatalog.version=0.4.1 -DskipTests
+5. The 'libext' dir used by oozie-setup should contain the following list JARS
+   hcatalog-core.jar
+   webhcat-java-client.jar
+   jackson-mapper-asl-1.8.8.jar
+   jackson-core-asl-1.8.8.jar
+   hive-common.jar
+   hive-metastore.jar
+   hive-exec.jar
+   hive-serde.jar
+   hive-shims.jar
+   libfb303.jar
+   (Note) hcatalog JARs will be automatically injected
+6. Upload this application directory to HDFS
+7. Run Oozie job using the job.properties. Coordinator actions will be in WAITING
+8. Make input dependencies available throught HCat client by "alter table invites add partition (ds='2010-01-01', region='usa')". This event will start the workflows with pig action
+9. First workflow will SUCCEED as expected, however second one will fail due to 'partition already exists' error. Disregard this. The example demonstrates working as expected.

Added: oozie/trunk/examples/src/main/apps/hcatalog/coordinator.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/examples/src/main/apps/hcatalog/coordinator.xml?rev=1449911&view=auto
==============================================================================
--- oozie/trunk/examples/src/main/apps/hcatalog/coordinator.xml (added)
+++ oozie/trunk/examples/src/main/apps/hcatalog/coordinator.xml Mon Feb 25 21:42:07 2013
@@ -0,0 +1,73 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<coordinator-app name="cron-coord" frequency="${coord:minutes(10)}" start="${start}" end="${end}" timezone="UTC"
+                 xmlns="uri:oozie:coordinator:0.2">
+    <datasets>
+        <dataset name="raw-logs" frequency="${coord:minutes(20)}" initial-instance="2010-01-01T00:00Z" timezone="UTC">
+            <uri-template>${hcatNode}/${db}/${table}/ds=${YEAR}-${MONTH}-${DAY};region=${region}</uri-template>
+        <done-flag></done-flag>
+        </dataset>
+        <dataset name="processed-logs" frequency="${coord:minutes(20)}" initial-instance="2010-01-01T00:00Z" timezone="UTC">
+            <uri-template>${hcatNode}/${db}/${outputtable}/ds=${dataOut};region=${region}</uri-template>
+        <done-flag></done-flag>
+        </dataset>
+    </datasets>
+
+    <input-events>
+        <data-in name="input" dataset="raw-logs">
+            <instance>${coord:current(0)}</instance>
+        </data-in>
+    </input-events>
+    <output-events>
+        <data-out name="output" dataset="processed-logs">
+            <instance>${coord:current(0)}</instance>
+        </data-out>
+    </output-events>
+
+        <action>
+        <workflow>
+            <app-path>${workflowAppUri}</app-path>
+            <configuration>
+                <property>
+                    <name>DB</name>
+                    <value>${coord:databaseIn('input')}</value>
+                </property>
+                <property>
+                    <name>TABLE</name>
+                    <value>${coord:tableIn('input')}</value>
+                </property>
+                <property>
+                    <name>FILTER</name>
+                    <value>${coord:dataInPartitionFilter('input', 'pig')}</value>
+                </property>
+                <property>
+                    <name>OUTPUT_PARTITION</name>
+                    <value>${coord:dataOutPartitions('output')}</value>
+                </property>
+                <property>
+                    <name>OUTPUT_DB</name>
+                    <value>${coord:databaseOut('output')}</value>
+                </property>
+                <property>
+                    <name>OUTPUT_TABLE</name>
+                    <value>${coord:tableOut('output')}</value>
+                </property>
+            </configuration>
+        </workflow>
+    </action>
+</coordinator-app>

Added: oozie/trunk/examples/src/main/apps/hcatalog/id.pig
URL: http://svn.apache.org/viewvc/oozie/trunk/examples/src/main/apps/hcatalog/id.pig?rev=1449911&view=auto
==============================================================================
--- oozie/trunk/examples/src/main/apps/hcatalog/id.pig (added)
+++ oozie/trunk/examples/src/main/apps/hcatalog/id.pig Mon Feb 25 21:42:07 2013
@@ -0,0 +1,21 @@
+--
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+--
+A = load '$DB.$TABLE' using org.apache.hcatalog.pig.HCatLoader();
+B = FILTER A BY $FILTER;
+C = foreach B generate foo, bar;
+store C into '$OUTPUT_DB.$OUTPUT_TABLE' USING org.apache.hcatalog.pig.HCatStorer('$OUTPUT_PARTITION');

Added: oozie/trunk/examples/src/main/apps/hcatalog/job.properties
URL: http://svn.apache.org/viewvc/oozie/trunk/examples/src/main/apps/hcatalog/job.properties?rev=1449911&view=auto
==============================================================================
--- oozie/trunk/examples/src/main/apps/hcatalog/job.properties (added)
+++ oozie/trunk/examples/src/main/apps/hcatalog/job.properties Mon Feb 25 21:42:07 2013
@@ -0,0 +1,33 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+nameNode=hdfs://localhost:8020
+jobTracker=localhost:8021
+queueName=default
+examplesRoot=examples
+
+oozie.coord.application.path=${nameNode}/user/${user.name}/${examplesRoot}/apps/hcatalog
+hcatNode=hcat://localhost:11002
+db=default
+table=invites
+start=2010-01-01T01:00Z
+end=2010-01-01T01:20Z
+workflowAppUri=${nameNode}/user/${user.name}/${examplesRoot}/apps/hcatalog
+dataOut=2011-01-01
+outputtable=oozie
+region=usa

Added: oozie/trunk/examples/src/main/apps/hcatalog/workflow.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/examples/src/main/apps/hcatalog/workflow.xml?rev=1449911&view=auto
==============================================================================
--- oozie/trunk/examples/src/main/apps/hcatalog/workflow.xml (added)
+++ oozie/trunk/examples/src/main/apps/hcatalog/workflow.xml Mon Feb 25 21:42:07 2013
@@ -0,0 +1,50 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<workflow-app xmlns="uri:oozie:workflow:0.2" name="pig-wf">
+    <start to="pig-node"/>
+    <action name="pig-node">
+        <pig>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapred.compress.map.output</name>
+                    <value>true</value>
+                </property>
+            </configuration>
+            <script>id.pig</script>
+        <param>DB=${DB}</param>
+        <param>TABLE=${TABLE}</param>
+        <param>FILTER=${FILTER}</param>
+            <param>OUTPUT_DB=${OUTPUT_DB}</param>
+            <param>OUTPUT_TABLE=${OUTPUT_TABLE}</param>
+            <param>OUTPUT_PARTITION=${OUTPUT_PARTITION}</param>
+        <file>lib/hive-site.xml</file>
+        </pig>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
+    <kill name="fail">
+        <message>Pig failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
+    </kill>
+    <end name="end"/>
+</workflow-app>

Modified: oozie/trunk/examples/src/test/java/org/apache/oozie/example/TestLocalOozieExample.java
URL: http://svn.apache.org/viewvc/oozie/trunk/examples/src/test/java/org/apache/oozie/example/TestLocalOozieExample.java?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/examples/src/test/java/org/apache/oozie/example/TestLocalOozieExample.java (original)
+++ oozie/trunk/examples/src/test/java/org/apache/oozie/example/TestLocalOozieExample.java Mon Feb 25 21:42:07 2013
@@ -17,8 +17,6 @@
  */
 package org.apache.oozie.example;
 
-import junit.framework.TestCase;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.test.XFsTestCase;
@@ -35,6 +33,7 @@ public class TestLocalOozieExample exten
 
     @Override
     protected void setUp() throws Exception {
+        System.setProperty("oozie.test.metastore.server", "false");
         super.setUp();
         oozieLocalLog = System.getProperty("oozielocal.log");
         System.setProperty("oozielocal.log", getTestCaseDir()+"/oozielocal.log");

Added: oozie/trunk/hcataloglibs/hcatalog-0.6/pom.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/hcataloglibs/hcatalog-0.6/pom.xml?rev=1449911&view=auto
==============================================================================
--- oozie/trunk/hcataloglibs/hcatalog-0.6/pom.xml (added)
+++ oozie/trunk/hcataloglibs/hcatalog-0.6/pom.xml Mon Feb 25 21:42:07 2013
@@ -0,0 +1,215 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.oozie</groupId>
+        <artifactId>oozie-main</artifactId>
+        <version>4.0.0-SNAPSHOT</version>
+        <relativePath>../../pom.xml</relativePath>
+    </parent>
+    <groupId>org.apache.oozie</groupId>
+    <artifactId>oozie-hcatalog</artifactId>
+    <version>0.6.0.oozie-4.0.0-SNAPSHOT</version>
+    <description>Apache Oozie HCatalog ${project.version}</description>
+    <name>Apache Oozie HCatalog ${project.version}</name>
+    <packaging>jar</packaging>
+
+    <!-- src/main/assemblies/hcataloglib.xml is configured with useTransitiveDependencies as false
+         as the required jars are very less and too many dependencies to exclude -->
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.hcatalog</groupId>
+            <artifactId>hcatalog-server-extensions</artifactId>
+            <version>0.6.0-SNAPSHOT</version>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.activemq</groupId>
+                    <artifactId>activemq-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.activemq</groupId>
+                    <artifactId>kahadb</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.activemq</groupId>
+                    <artifactId>activeio-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.activemq.protobuf</groupId>
+                    <artifactId>activemq-protobuf</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.geronimo.specs</groupId>
+                    <artifactId>geronimo-jms_1.1_spec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.geronimo.specs</groupId>
+                    <artifactId>geronimo-j2ee-management_1.1_spec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.osgi</groupId>
+                    <artifactId>org.osgi.core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.jms</groupId>
+                    <artifactId>jms</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hcatalog</groupId>
+            <artifactId>hcatalog-core</artifactId>
+            <version>0.6.0-SNAPSHOT</version>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-service</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-cli</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-builtins</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>jline</groupId>
+                    <artifactId>jline</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.google.code.findbugs</groupId>
+                    <artifactId>jsr305</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hcatalog</groupId>
+            <artifactId>webhcat-java-client</artifactId>
+            <version>0.6.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-common</artifactId>
+            <version>${hive.version}</version>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-metastore</artifactId>
+            <version>${hive.version}</version>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-exec</artifactId>
+            <version>${hive.version}</version>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-serde</artifactId>
+            <version>${hive.version}</version>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libfb303</artifactId>
+            <version>0.7.0</version>
+            <scope>compile</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.codehaus.jackson</groupId>
+            <artifactId>jackson-core-asl</artifactId>
+            <version>1.8.8</version>
+            <scope>compile</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.codehaus.jackson</groupId>
+            <artifactId>jackson-mapper-asl</artifactId>
+            <version>1.8.8</version>
+            <scope>compile</scope>
+        </dependency>
+
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <descriptors>
+                        <descriptor>../../src/main/assemblies/hcataloglib.xml</descriptor>
+                    </descriptors>
+                    <finalName>hcataloglibs</finalName>
+                    <appendAssemblyId>false</appendAssemblyId>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
+

Added: oozie/trunk/hcataloglibs/pom.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/hcataloglibs/pom.xml?rev=1449911&view=auto
==============================================================================
--- oozie/trunk/hcataloglibs/pom.xml (added)
+++ oozie/trunk/hcataloglibs/pom.xml Mon Feb 25 21:42:07 2013
@@ -0,0 +1,61 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.oozie</groupId>
+        <artifactId>oozie-main</artifactId>
+        <version>4.0.0-SNAPSHOT</version>
+    </parent>
+    <groupId>org.apache.oozie</groupId>
+    <artifactId>oozie-hcataloglibs</artifactId>
+    <version>4.0.0-SNAPSHOT</version>
+    <description>Apache Oozie HCatalog Libs</description>
+    <name>Apache Oozie HCatalog Libs</name>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>hcatalog-0.6</module>
+    </modules>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <descriptors>
+                        <descriptor>../src/main/assemblies/hcataloglibs.xml</descriptor>
+                    </descriptors>
+                    <finalName>oozie-${project.version}</finalName>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
+

Modified: oozie/trunk/login/src/main/java/org/apache/oozie/servlet/login/LoginServlet.java
URL: http://svn.apache.org/viewvc/oozie/trunk/login/src/main/java/org/apache/oozie/servlet/login/LoginServlet.java?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/login/src/main/java/org/apache/oozie/servlet/login/LoginServlet.java (original)
+++ oozie/trunk/login/src/main/java/org/apache/oozie/servlet/login/LoginServlet.java Mon Feb 25 21:42:07 2013
@@ -44,16 +44,6 @@ public class LoginServlet extends HttpSe
     private static final String LOGIN_PAGE_TEMPLATE_DEFAULT = "login-page-template.html";
     private String loginPageTemplate;
 
-    /**
-     * Constant for the configuration property that indicates the expiration time (or max age) of the "oozie.web.login.auth" cookie.
-     * It is given in seconds.  A positive value indicates that the cookie will expire after that many seconds have passed; make
-     * sure this value is high enough to allow the user to be forwarded to the backurl before the cookie expires.  A negative value
-     * indicates that the cookie will be deleted when the browser exits.
-     */
-    public static final String LOGIN_AUTH_COOKIE_EXPIRE_TIME = "login.auth.cookie.expire.time";
-    private static final int LOGIN_AUTH_COOKIE_EXPIRE_TIME_DEFAULT = 180;   // 3 minutes
-    private int loginAuthCookieExpireTime;
-
     private static final String USERNAME = "username";
     private static final String PASSWORD = "password";
     private static final String BACKURL = "backurl";
@@ -82,20 +72,6 @@ public class LoginServlet extends HttpSe
         } catch (IOException ex) {
             throw new ServletException("Could not read resource [" + loginPageTemplateName + "]");
         }
-
-        // Read in the cookie expiration time
-        String cookieExpireTime = getInitParameter(LOGIN_AUTH_COOKIE_EXPIRE_TIME);
-        if (cookieExpireTime == null) {
-            loginAuthCookieExpireTime = LOGIN_AUTH_COOKIE_EXPIRE_TIME_DEFAULT;
-        }
-        else {
-            try {
-                loginAuthCookieExpireTime = Integer.parseInt(cookieExpireTime);
-            }
-            catch (NumberFormatException nfe) {
-                throw new ServletException(LOGIN_AUTH_COOKIE_EXPIRE_TIME + " must be a valid integer", nfe);
-            }
-        }
     }
 
     protected void renderLoginPage(String message, String username, String backUrl, HttpServletResponse resp)
@@ -174,7 +150,6 @@ public class LoginServlet extends HttpSe
     protected void writeCookie(HttpServletResponse resp, String username) throws UnsupportedEncodingException {
         Cookie cookie = new Cookie("oozie.web.login.auth", URLEncoder.encode(username, "UTF-8"));
         cookie.setPath("/");
-        cookie.setMaxAge(loginAuthCookieExpireTime);
         resp.addCookie(cookie);
     }
 }

Modified: oozie/trunk/login/src/main/webapp/WEB-INF/web.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/login/src/main/webapp/WEB-INF/web.xml?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/login/src/main/webapp/WEB-INF/web.xml (original)
+++ oozie/trunk/login/src/main/webapp/WEB-INF/web.xml Mon Feb 25 21:42:07 2013
@@ -39,17 +39,6 @@
                 console).
             </description>
         </init-param>
-        <init-param>
-            <param-name>login.auth.cookie.expire.time</param-name>
-            <param-value>180</param-value>
-            <description>
-                Constant for the configuration property that indicates the expiration time (or max age) of the
-                "oozie.web.login.auth" cookie.  It is given in seconds.  A positive value indicates that the cookie will expire
-                after that many seconds have passed; make sure this value is high enough to allow the user to be forwarded to the
-                backurl before the cookie expires.  A negative value indicates that the cookie will be deleted when the browser
-                exits.
-            </description>
-        </init-param>
         -->
         <!--
         <init-param>

Modified: oozie/trunk/minitest/src/test/java/org/apache/oozie/test/WorkflowTest.java
URL: http://svn.apache.org/viewvc/oozie/trunk/minitest/src/test/java/org/apache/oozie/test/WorkflowTest.java?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/minitest/src/test/java/org/apache/oozie/test/WorkflowTest.java (original)
+++ oozie/trunk/minitest/src/test/java/org/apache/oozie/test/WorkflowTest.java Mon Feb 25 21:42:07 2013
@@ -41,6 +41,7 @@ public class WorkflowTest extends MiniOo
 
     @Override
     protected void setUp() throws Exception {
+        System.setProperty("oozie.test.metastore.server", "false");
         System.setProperty(XLogService.LOG4J_FILE, "oozie-log4j.properties");
         super.setUp();
     }

Modified: oozie/trunk/pom.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/pom.xml?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/pom.xml (original)
+++ oozie/trunk/pom.xml Mon Feb 25 21:42:07 2013
@@ -45,7 +45,7 @@
         <!-- to be able to run a single test case from the main project -->
         <failIfNoTests>false</failIfNoTests>
 
-        <test.timeout>3600</test.timeout>
+        <test.timeout>5400</test.timeout>
 
         <!-- platform encoding override -->
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
@@ -68,9 +68,11 @@
 
         <hadoop.version>1.1.1</hadoop.version>
         <hbase.version>0.94.2</hbase.version>
+        <hcatalog.version>0.6.0</hcatalog.version>
 
         <hadooplib.version>${hadoop.version}.oozie-${project.version}</hadooplib.version>
         <hbaselib.version>${hbase.version}.oozie-${project.version}</hbaselib.version>
+        <hcataloglib.version>${hcatalog.version}.oozie-${project.version}</hcataloglib.version>
 
         <clover.license>/home/jenkins/tools/clover/latest/lib/clover.license</clover.license>
         <!--
@@ -94,6 +96,7 @@
         <module>client</module>
         <module>hadooplibs</module>
         <module>hbaselibs</module>
+        <module>hcataloglibs</module>
         <module>core</module>
         <module>tests</module>
         <module>webapp</module>
@@ -267,11 +270,17 @@
           <dependency>
               <groupId>org.slf4j</groupId>
               <artifactId>slf4j-simple</artifactId>
-              <version>1.5.8</version>
+              <version>1.6.6</version>
           </dependency>
 
             <!-- core -->
-             <dependency>
+            <dependency>
+                <groupId>org.apache.oozie</groupId>
+                <artifactId>oozie-hcatalog</artifactId>
+                <version>${hcataloglib.version}</version>
+            </dependency>
+
+            <dependency>
                 <groupId>org.apache.oozie</groupId>
                 <artifactId>oozie-hbase</artifactId>
                 <version>${hbaselib.version}</version>
@@ -332,6 +341,12 @@
             </dependency>
 
             <dependency>
+                <groupId>net.sf.ehcache</groupId>
+                <artifactId>ehcache-core</artifactId>
+                <version>2.6.3</version>
+            </dependency>
+
+            <dependency>
                 <groupId>org.apache.hadoop</groupId>
                 <artifactId>hadoop-streaming</artifactId>
                 <version>${streaming.version}</version>
@@ -451,19 +466,38 @@
             <dependency>
                 <groupId>org.slf4j</groupId>
                 <artifactId>slf4j-api</artifactId>
-                <version>1.5.8</version>
+                <version>1.6.6</version>
             </dependency>
             <dependency>
                 <groupId>org.slf4j</groupId>
                 <artifactId>slf4j-log4j12</artifactId>
-                <version>1.5.8</version>
+                <version>1.6.6</version>
             </dependency>
+
             <dependency>
                 <groupId>org.apache.hive</groupId>
                 <artifactId>hive-builtins</artifactId>
                 <version>${hive.version}</version>
             </dependency>
 
+           <dependency>
+                <groupId>org.apache.activemq</groupId>
+                <artifactId>activemq-client</artifactId>
+                <version>5.8.0</version>
+           </dependency>
+
+           <dependency>
+                <groupId>org.apache.activemq</groupId>
+                <artifactId>activemq-broker</artifactId>
+                <version>5.8.0</version>
+           </dependency>
+
+           <dependency>
+                <groupId>org.apache.activemq</groupId>
+                <artifactId>activemq-kahadb-store</artifactId>
+                <version>5.8.0</version>
+           </dependency>
+
             <dependency>
                 <groupId>commons-logging</groupId>
                 <artifactId>commons-logging</artifactId>
@@ -850,6 +884,7 @@
                         -->
                         <exclude>**/TestSsh*.java</exclude>
 
+
                         <!-- See 'testSqoop' profile in core/pom.xml and the Building doc-->
                         <exclude>**/TestSqoop*.java</exclude>
 

Modified: oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/oozie/trunk/release-log.txt?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/release-log.txt (original)
+++ oozie/trunk/release-log.txt Mon Feb 25 21:42:07 2013
@@ -1,8 +1,50 @@
 -- Oozie 4.0.0 release (trunk - unreleased)
 
 OOZIE-1237 Bump up trunk to 4.0.0-SNAPSHOT (virag)
+OOZIE-561 Integrate Oozie with HCatalog
+    OOZIE-1181 Dependency cache with configurations for eviction, ttl and max elements in memory (rohini via virag)
+    OOZIE-1217 Address review comments in OOZIE-1210 (rohini via virag)
+    OOZIE-1197 Create a hcat sharelib which can be included in pig, hive and java actions (mona,rohini via virag)
+    OOZIE-1196 HCat EL functions for database and table should be modified (mona)
+    OOZIE-1210 Rework uri handling for Prepare actions and jms server mapping (rohini via virag)
+    OOZIE-1179 coord action in WAITING when no definition of dataset in coord job xml (mona)
+    OOZIE-1185 Retry jms connections on failure (rohini via virag)
+    OOZIE-1158 Add hcataloglib sub-module (mona)
+    OOZIE-1180 Separate the connection context details from JMS Accessor service (virag)
+    OOZIE-1157 EL function hcat:exists for decision making (rohini via mona)
+    OOZIE-1167 Fix and rework PartitionDependency Management (rohini via virag)
+    OOZIE-1156 Make all the latest/future instances as pull dependences (virag)
+    OOZIE-1145 Modify Recovery Service to handle push missing dependencies (virag)
+    OOZIE-1135 Display missing partition dependencies via job -info command on CLI (mona)
+    OOZIE-1125 Prepare actions for hcat (rohini via virag)
+    OOZIE-1123 EL Functions for hcatalog (mona)
+    OOZIE-1138 Provide rule based mechanism to allow multiple hcatalog servers to connect to JMS server (virag)
+    OOZIE-1111 change HCatURI to specify partitions in path instead of query parameter (rohini,ryota via virag)
+    OOZIE-1108 Fix JMS message consumer to maintain single session per topic registration (mona)
+    OOZIE-1075 Create general scheme handler (rohini via virag)
+    OOZIE-1107 Change default done-flag from _SUCCESS to empty for Hcat (mohammad)
+    OOZIE-1095 Add HCatalog jar as resource for building (mona)
+    OOZIE-1105 Resolve issues found during integration(mohammad)
+    OOZIE-1086 Command to check the missing partitions directly against HCatalog server (mohammad)
+    OOZIE-1050 Implement logic to update dependencies via push JMS message(mona via mohammad)
+    OOZIE-1068 Metadata Accessor service for HCatalog(mohammad)
+    OOZIE-1069 Update dataIn and dataOut EL functions to support partitions (mohammad)
+    OOZIE-1043 Add logic to register to Missing Dependency Structure in coord action materialization (ryota via mohammad)
+    OOZIE-1061 Add new EL functions to retrieve HCatalog server, DB and table name(mohammad)
+    OOZIE-1056 Command to update push-based dependency (mohammad)
+    OOZIE-1059 Add static method to create URI String in HCatURI(ryota via mohammad)
+    OOZIE-1039 Implement the Missing Dependency structure for HCat partitions (mona via mohammad)
+    OOZIE-1042 Coordinator action table schema change. (Mohammad)
+    OOZIE-1036 Utility class to parse HCat URI (Ryota via Mohammad)
+    OOZIE-1033 Generic utility class to register/unregister a JMS message handler(Mohammad)
+    OOZIE-1032 Create JMSService used for any JMS compliant product (Mohammad)
+OOZIE-1045 Parameterize <unresolved-instances> tag currently hardcoded (egashira via mona)
+OOZIE-1189 add filter option to specify JobID and AppName in SLA CLI command (egashira via mona)
+OOZIE-1054 Create script to properly upload sharelib to HDFS (bowenzhangusa via tucu)
 OOZIE-1096 Update wfgen README.txt to have the TLP mailing list (jun aoki via rkanter)
+OOZIE-1015 HadoopAccessorService jobtracker validation should not have hardcoded conf key (mona)
 OOZIE-1078 Help -> Documentation and Help -> Online Help should link to oozie.apache.org/ (jun via mohammad)
+OOZIE-669 Deprecate oozie-start.sh, oozie-stop.sh & oozie-run.sh scripts (rkanter via tucu)
 OOZIE-809 MySQL TEXT columns should be MEDIUMTEXT (rkanter via tucu)
 OOZIE-979 bump up trunk version to 3.4.0-SNAPSHOT (tucu)
 OOZIE-1007 Add license headers to all files don't have them  (egashira via tucu)
@@ -11,12 +53,6 @@ OOZIE-944 Implement Workflow Generator U
 
 -- Oozie 3.3.2 (unreleased)
 
-OOZIE-1189 add filter option to specify JobID and AppName in SLA CLI command (egashira via mona)
-OOZIE-1054 Create script to properly upload sharelib to HDFS (bowenzhangusa via tucu)
-OOZIE-1015 HadoopAccessorService jobtracker validation should not have hardcoded conf key (mona)
-OOZIE-669 Deprecate oozie-start.sh, oozie-stop.sh & oozie-run.sh scripts (rkanter via tucu)
-OOZIE-1219 The timezone cookie should never expire (rkanter)
-OOZIE-1220 Make the login example cookie expire (rkanter)
 OOZIE-1227 In a coordinator, specifying the <app-path> without a namenode causes it to fail (rkanter)
 OOZIE-1226 Workflow lib path not found in classpath for a subworkflow (rkanter)
 OOZIE-1184 Demo example job.properties has an unused parameter (udai via rkanter)
@@ -264,6 +300,7 @@ OOZIE-827 StatusTransitService fails to 
 OOZIE-815 Remove select * from queries related to coord action (Virag via Mohammad)
 OOZIE-826 TestCoordKillXCommand's testCoordKillXCommandUniqueness testcase is failing after interrupt changes(virag via Mohammad)
 OOZIE-819 Interrupt map doesn't have unique set of commands for a given jobid(virag via Mohammad)
+OOZIE-810 Modify Oozie POM to pickup doxia 9.2y from a repo where it is avail(tucu via Mohammad)
 OOZIE-241 EL function(s) to expose action output data as a (XML/JSON/PROP) blob (tucu)
 OOZIE-825 Update oozie-sharelib-hive to use Hive 0.9.0 (virag via tucu)
 OOZIE-820 OOZIE-820 Shell action to support env-var value with = sign (mona via tucu)
@@ -402,7 +439,6 @@ OOZIE-600 Bump-up the version to 3.2.0-S
 
 -- Oozie 3.1.3 release
 
-OOZIE-810 Modify Oozie POM to pickup doxia 9.2y from a repo where it is avail(tucu via Mohammad)
 OOZIE-683 Add DISCLAIMER file in the root.(Mohammad)
 OOZIE-681 Update the contents of readme.txt.(Mohammad)
 OOZIE-680 oozie's assembly creates an extra level of empty subdirectory for docs. (rvs via tucu)

Added: oozie/trunk/sharelib/hcatalog/pom.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/sharelib/hcatalog/pom.xml?rev=1449911&view=auto
==============================================================================
--- oozie/trunk/sharelib/hcatalog/pom.xml (added)
+++ oozie/trunk/sharelib/hcatalog/pom.xml Mon Feb 25 21:42:07 2013
@@ -0,0 +1,235 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.oozie</groupId>
+        <artifactId>oozie-main</artifactId>
+        <version>4.0.0-SNAPSHOT</version>
+        <relativePath>../..</relativePath>
+    </parent>
+    <groupId>org.apache.oozie</groupId>
+    <artifactId>oozie-sharelib-hcatalog</artifactId>
+    <version>4.0.0-SNAPSHOT</version>
+    <description>Apache Oozie Share Lib HCatalog</description>
+    <name>Apache Oozie Share Lib HCatalog</name>
+    <packaging>jar</packaging>
+
+    <properties>
+        <sharelib.action.postfix>hcatalog</sharelib.action.postfix>
+        <sharelib.transitive.filtering>true</sharelib.transitive.filtering>
+    </properties>
+
+    <dependencies>
+        <dependency>
+             <groupId>org.apache.oozie</groupId>
+             <artifactId>oozie-hcatalog</artifactId>
+             <scope>compile</scope>
+             <exclusions>
+                 <exclusion>
+                    <groupId>org.apache.hcatalog</groupId>
+                    <artifactId>hcatalog-server-extensions</artifactId>
+                </exclusion>
+                 <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-builtins</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.codehaus.jackson</groupId>
+                    <artifactId>jackson-core-asl</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.codehaus.jackson</groupId>
+                    <artifactId>jackson-mapper-asl</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-collections</groupId>
+                    <artifactId>commons-collections</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.httpcomponents</groupId>
+                    <artifactId>httpcore</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.httpcomponents</groupId>
+                    <artifactId>httpclient</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.derby</groupId>
+                    <artifactId>derby</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.googlecode.javaewah</groupId>
+                    <artifactId>JavaEWAH</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.hamcrest</groupId>
+                    <artifactId>hamcrest-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.json</groupId>
+                    <artifactId>json</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.jboss.netty</groupId>
+                    <artifactId>netty</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.antlr</groupId>
+                    <artifactId>antlr</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.mockito</groupId>
+                    <artifactId>mockito-all</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.datanucleus</groupId>
+                    <artifactId>datanucleus-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.datanucleus</groupId>
+                    <artifactId>datanucleus-rdbms</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.datanucleus</groupId>
+                    <artifactId>datanucleus-enhancer</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.datanucleus</groupId>
+                    <artifactId>datanucleus-connectionpool</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-cli</groupId>
+                    <artifactId>commons-cli</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-lang</groupId>
+                    <artifactId>commons-lang</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-codec</groupId>
+                    <artifactId>commons-codec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-dbcp</groupId>
+                    <artifactId>commons-dbcp</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-pool</groupId>
+                    <artifactId>commons-pool</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-logging</groupId>
+                    <artifactId>commons-logging</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-logging</groupId>
+                    <artifactId>commons-logging-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>asm</groupId>
+                    <artifactId>asm</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.antlr</groupId>
+                    <artifactId>antlr-runtime</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.antln</groupId>
+                    <artifactId>stringtemplate</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>antlr</groupId>
+                    <artifactId>antlr</artifactId>
+                </exclusion>
+             </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+                <filtering>true</filtering>
+            </resource>
+        </resources>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>gen-classpath</id>
+                        <phase>generate-test-resources</phase>
+                        <goals>
+                            <goal>build-classpath</goal>
+                        </goals>
+                        <configuration>
+                            <includeScope>compile</includeScope>
+                            <outputFile>${project.build.directory}/classpath</outputFile>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <finalName>partial-sharelib</finalName>
+                    <appendAssemblyId>false</appendAssemblyId>
+                    <descriptors>
+                        <descriptor>../../src/main/assemblies/partial-sharelib.xml</descriptor>
+                    </descriptors>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
+

Modified: oozie/trunk/sharelib/pom.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/sharelib/pom.xml?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/sharelib/pom.xml (original)
+++ oozie/trunk/sharelib/pom.xml Mon Feb 25 21:42:07 2013
@@ -33,6 +33,7 @@
 
     <modules>
         <module>streaming</module>
+        <module>hcatalog</module>
         <module>pig</module>
         <module>hive</module>
         <module>sqoop</module>

Added: oozie/trunk/src/main/assemblies/hcataloglib.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/src/main/assemblies/hcataloglib.xml?rev=1449911&view=auto
==============================================================================
--- oozie/trunk/src/main/assemblies/hcataloglib.xml (added)
+++ oozie/trunk/src/main/assemblies/hcataloglib.xml Mon Feb 25 21:42:07 2013
@@ -0,0 +1,35 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<assembly>
+    <id>hcataloglib</id>
+    <formats>
+        <format>dir</format>
+    </formats>
+    <includeBaseDirectory>false</includeBaseDirectory>
+
+    <dependencySets>
+        <dependencySet>
+            <useProjectArtifact>false</useProjectArtifact>
+            <useTransitiveDependencies>false</useTransitiveDependencies>
+            <useTransitiveFiltering>true</useTransitiveFiltering>
+            <unpack>false</unpack>
+            <outputDirectory>hcataloglib-${project.version}</outputDirectory>
+        </dependencySet>
+    </dependencySets>
+
+</assembly>

Added: oozie/trunk/src/main/assemblies/hcataloglibs.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/src/main/assemblies/hcataloglibs.xml?rev=1449911&view=auto
==============================================================================
--- oozie/trunk/src/main/assemblies/hcataloglibs.xml (added)
+++ oozie/trunk/src/main/assemblies/hcataloglibs.xml Mon Feb 25 21:42:07 2013
@@ -0,0 +1,33 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<assembly>
+    <id>hcataloglibs</id>
+    <formats>
+        <format>dir</format>
+        <format>tar.gz</format>
+    </formats>
+    <includeBaseDirectory>true</includeBaseDirectory>
+    <baseDirectory>oozie-${project.version}</baseDirectory>
+    <fileSets>
+        <!-- HCatalog libs -->
+        <fileSet>
+            <directory>${basedir}/../hcataloglibs/hcatalog-0.6/target/hcataloglibs</directory>
+            <outputDirectory>/hcataloglibs</outputDirectory>
+        </fileSet>
+    </fileSets>
+</assembly>

Modified: oozie/trunk/src/main/assemblies/sharelib.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/src/main/assemblies/sharelib.xml?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/src/main/assemblies/sharelib.xml (original)
+++ oozie/trunk/src/main/assemblies/sharelib.xml Mon Feb 25 21:42:07 2013
@@ -55,6 +55,10 @@
            <directory>${basedir}/distcp/target/partial-sharelib</directory>
            <outputDirectory>/</outputDirectory>
        </fileSet>
+       <fileSet>
+           <directory>${basedir}/hcatalog/target/partial-sharelib</directory>
+           <outputDirectory>/</outputDirectory>
+       </fileSet>
     </fileSets>
 
 </assembly>

Modified: oozie/trunk/tests/pig/src/test/java/org/apache/oozie/action/hadoop/TestPigActionExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/tests/pig/src/test/java/org/apache/oozie/action/hadoop/TestPigActionExecutor.java?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/tests/pig/src/test/java/org/apache/oozie/action/hadoop/TestPigActionExecutor.java (original)
+++ oozie/trunk/tests/pig/src/test/java/org/apache/oozie/action/hadoop/TestPigActionExecutor.java Mon Feb 25 21:42:07 2013
@@ -29,6 +29,7 @@ import org.apache.oozie.WorkflowActionBe
 import org.apache.oozie.WorkflowJobBean;
 import org.apache.oozie.action.hadoop.ActionExecutorTestCase.Context;
 import org.apache.oozie.client.WorkflowAction;
+import org.apache.oozie.service.URIHandlerService;
 import org.apache.oozie.service.WorkflowAppService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.HadoopAccessorService;
@@ -70,6 +71,12 @@ public class TestPigActionExecutor exten
             "store B into '$OUT' USING PigStorage();\n";
 
     @Override
+    protected void setUp() throws Exception {
+        System.setProperty("oozie.test.metastore.server", "false");
+        super.setUp();
+    }
+
+    @Override
     protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.classes", PigActionExecutor.class.getName());
@@ -93,8 +100,8 @@ public class TestPigActionExecutor exten
         classes.add(LauncherSecurityManager.class);
         classes.add(LauncherException.class);
         classes.add(LauncherMainException.class);
-        classes.add(FileSystemActions.class);
         classes.add(PrepareActionsDriver.class);
+        classes.addAll(Services.get().get(URIHandlerService.class).getClassesForLauncher());
         classes.add(ActionStats.class);
         classes.add(ActionType.class);
         classes.add(LauncherMain.class);

Modified: oozie/trunk/tests/pig/src/test/java/org/apache/oozie/action/hadoop/TestPigMain.java
URL: http://svn.apache.org/viewvc/oozie/trunk/tests/pig/src/test/java/org/apache/oozie/action/hadoop/TestPigMain.java?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/tests/pig/src/test/java/org/apache/oozie/action/hadoop/TestPigMain.java (original)
+++ oozie/trunk/tests/pig/src/test/java/org/apache/oozie/action/hadoop/TestPigMain.java Mon Feb 25 21:42:07 2013
@@ -39,6 +39,7 @@ public class TestPigMain extends PigTest
 
     @Override
     protected void setUp() throws Exception {
+        System.setProperty("oozie.test.metastore.server", "false");
         super.setUp();
         SECURITY_MANAGER = System.getSecurityManager();
     }

Modified: oozie/trunk/webapp/src/main/webapp/oozie-console.js
URL: http://svn.apache.org/viewvc/oozie/trunk/webapp/src/main/webapp/oozie-console.js?rev=1449911&r1=1449910&r2=1449911&view=diff
==============================================================================
--- oozie/trunk/webapp/src/main/webapp/oozie-console.js (original)
+++ oozie/trunk/webapp/src/main/webapp/oozie-console.js Mon Feb 25 21:42:07 2013
@@ -2112,9 +2112,7 @@ function initConsole() {
             listeners:
             { select: { fn:function(combo, value)
                 {
-                    Ext.state.Manager.setProvider(new Ext.state.CookieProvider({
-                        expires: new Date(new Date().getTime()+315569259747) // about 10 years from now!
-                    }));
+                    Ext.state.Manager.setProvider(new Ext.state.CookieProvider());
                     Ext.state.Manager.set("TimezoneId",this.value);
                 }
             }}