You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hcatalog-commits@incubator.apache.org by ha...@apache.org on 2011/07/21 18:11:14 UTC

svn commit: r1149292 - in /incubator/hcatalog/trunk: ./ src/docs/src/documentation/content/xdocs/

Author: hashutosh
Date: Thu Jul 21 18:11:11 2011
New Revision: 1149292

URL: http://svn.apache.org/viewvc?rev=1149292&view=rev
Log:
HCATALOG-66: 0.2 Docs

Added:
    incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/dynpartition.xml
    incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/notification.xml
Modified:
    incubator/hcatalog/trunk/CHANGES.txt
    incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/cli.xml
    incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/inputoutput.xml
    incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/loadstore.xml
    incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/site.xml
    incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/supportedformats.xml
    incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/tabs.xml

Modified: incubator/hcatalog/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/CHANGES.txt?rev=1149292&r1=1149291&r2=1149292&view=diff
==============================================================================
--- incubator/hcatalog/trunk/CHANGES.txt (original)
+++ incubator/hcatalog/trunk/CHANGES.txt Thu Jul 21 18:11:11 2011
@@ -14,6 +14,8 @@ Trunk (unreleased changes)
     (Krishna Kumar via macyang)
     
   IMPROVEMENTS
+    HCAT-66. HCatalog 0.2.0 Documentation (chandec via hashutosh)
+
     HCAT-54. Javadoc is not being built as part of HCatalog docs (hashutosh) 
 
     HCAT-35. HCatalog fails to compile with Pig 0.9 (hashutosh)

Modified: incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/cli.xml
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/cli.xml?rev=1149292&r1=1149291&r2=1149292&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/cli.xml (original)
+++ incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/cli.xml Thu Jul 21 18:11:11 2011
@@ -19,7 +19,7 @@
 
 <document>
   <header>
-    <title>HCatalog Command Line Interface</title>
+    <title>Command Line Interface</title>
   </header>
   <body>
 

Added: incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/dynpartition.xml
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/dynpartition.xml?rev=1149292&view=auto
==============================================================================
--- incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/dynpartition.xml (added)
+++ incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/dynpartition.xml Thu Jul 21 18:11:11 2011
@@ -0,0 +1,136 @@
+<?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.
+-->
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document>
+  <header>
+    <title>Dynamic Partitioning </title>
+  </header>
+  <body>
+  
+<!-- ==================================================================== -->  
+<section>
+    <title>Overview</title>
+    
+ <p>In earlier versions of HCatalog, to read data users could specify that they were interested in reading from the table and specify various partition key/value combinations to prune, as if specifying a SQL-like where clause. However, to write data the abstraction was not as seamless. We still required users to write out data to the table, partition-by-partition, but these partitions required fine-grained knowledge of which key/value pairs they needed. We required this knowledge in advance, and we required the user to have already grouped the requisite data accordingly before attempting to store. </p>
+    
+    <p>The following Pig script illustrates this: </p>
+<source>
+A = load 'raw' using HCatLoader(); 
+... 
+split Z into for_us if region='us', for_eu if region='eu', for_asia if region='asia'; 
+store for_us into 'processed' using HCatStorer("ds=20110110, region=us"); 
+store for_eu into 'processed' using HCatStorer("ds=20110110, region=eu"); 
+store for_asia into 'processed' using HCatStorer("ds=20110110, region=asia"); 
+</source>    
+<p></p>   
+    
+    <p>This approach had a major issue. MapReduce programs and Pig scripts needed to be aware of all the possible values of a key, and these values needed to be maintained and/or modified when new values were introduced. With more partitions, scripts began to look cumbersome. And if each partition being written launched a separate HCatalog store, we were increasing the load on the HCatalog server and launching more jobs for the store by a factor of the number of partitions.</p>
+    
+    <p>A better approach is to have HCatalog determine all the partitions required from the data being written. This would allow us to simplify the above script into the following: </p>  
+    
+<source>
+A = load 'raw' using HCatLoader(); 
+... 
+store Z into 'processed' using HCatStorer("ds=20110110"); 
+</source> 
+
+<p>The way dynamic partitioning works is that HCatalog locates partition columns in the data passed to it and uses the data in these columns to split the rows across multiple partitions. (The data passed to HCatalog <strong>must</strong> have a schema that matches the schema of the destination table and hence should always contain partition columns.)  It is important to note that partition columns can’t contain null values or the whole process will fail. It is also important note that all partitions created during a single run are part of a transaction and if any part of the process fails none of the partitions will be added to the table.</p>
+</section>
+  
+<!-- ==================================================================== -->  
+<section>
+      <title>Usage with Pig</title>
+      
+<p>Usage from Pig is very simple!  Instead of specifying all keys as one normally does for a store, users can specify the keys that are actually needed. HCatOutputFormat will trigger on dynamic partitioning usage if necessary (if a key value is not specified) and will inspect the data to write it out appropriately.  </p>
+
+<p>So this statement...</p>
+<source>
+store A into 'mytable' using HCatStorer("a=1, b=1");
+</source>
+
+<p>...is equivalent to any of the following statements, if the data has only values where a=1 and b=1: </p>
+
+<source>
+store A into 'mytable' using HCatStorer();
+</source>
+
+<source>
+store A into 'mytable' using HCatStorer("a=1");
+</source>
+
+<source>
+store A into 'mytable' using HCatStorer("b=1");
+</source>
+
+<p>On the other hand, if there is data that spans more than one partition, then HCatOutputFormat will automatically figure out how to spray the data appropriately. </p>
+
+<p>For example, let's say a=1 for all values across our dataset and b takes the value 1 and 2. Then the following statement... </p>
+<source>
+store A into 'mytable' using HCatStorer();
+</source>
+
+<p>...is equivalent to either of these statements: </p>
+<source>
+store A into 'mytable' using HCatStorer("a=1");
+</source>
+
+<source>
+split A into A1 if b='1', A2 if b='2';
+store A1 into 'mytable' using HCatStorer("a=1, b=1");
+store A2 into 'mytable' using HCatStorer("a=1, b=2");
+</source>
+</section>
+
+<!-- ==================================================================== -->  
+<section>
+<title>Usage from MapReduce</title>
+<p>As with Pig, the only change in dynamic partitioning that a MapReduce programmer sees is that they don't have to specify all the partition key/value combinations.</p>   
+
+<p>A current code example for writing out a specific partition for (a=1,b=1) would go something like this: </p>  
+   
+<source>
+Map&lt;String, String&gt; partitionValues = new HashMap&lt;String, String&gt;();
+partitionValues.put("a", "1");
+partitionValues.put("b", "1");
+HCatTableInfo info = HCatTableInfo.getOutputTableInfo(
+    serverUri, serverKerberosPrincipal, dbName, tblName, partitionValues);
+HCatOutputFormat.setOutput(job, info);
+</source> 
+
+<p>And to write to multiple partitions, separate jobs will have to be kicked off with each of the above.</p>   
+
+<p>With dynamic partition, we simply specify only as many keys as we know about, or as required. It will figure out the rest of the keys by itself and spray out necessary partitions, being able to create multiple partitions with a single job.</p>   
+
+</section>
+
+<!-- ==================================================================== -->  
+<section>
+      <title>Compaction</title> 
+<p>Dynamic partitioning potentially results in a large number of files and more namenode load. To address this issue, we utilize HAR to archive partitions after writing out as part of the HCatOutputCommitter action. Compaction is disabled by default. To enable compaction, use the Hive parameter hive.archive.enabled, specified in the client side hive-site.xml. The current behavior of compaction is to fail the entire job if compaction fails. </p>
+</section>
+
+<!-- ==================================================================== -->  
+<section>
+      <title>References</title>
+   <p>See <a href="https://cwiki.apache.org/HCATALOG/hcatalog02design.html">HCatalog 0.2 Architecture</a>  </p>      
+      
+</section>
+  
+  </body>
+</document>

Modified: incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/inputoutput.xml
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/inputoutput.xml?rev=1149292&r1=1149291&r2=1149292&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/inputoutput.xml (original)
+++ incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/inputoutput.xml Thu Jul 21 18:11:11 2011
@@ -19,7 +19,7 @@
 
 <document>
   <header>
-    <title>HCatalog Input and Output Interfaces</title>
+    <title>Input and Output Interfaces</title>
   </header>
   <body>
 

Modified: incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/loadstore.xml
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/loadstore.xml?rev=1149292&r1=1149291&r2=1149292&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/loadstore.xml (original)
+++ incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/loadstore.xml Thu Jul 21 18:11:11 2011
@@ -19,7 +19,7 @@
 
 <document>
   <header>
-    <title>HCatalog Load and Store Interfaces</title>
+    <title>Load and Store Interfaces</title>
   </header>
   <body>
  

Added: incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/notification.xml
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/notification.xml?rev=1149292&view=auto
==============================================================================
--- incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/notification.xml (added)
+++ incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/notification.xml Thu Jul 21 18:11:11 2011
@@ -0,0 +1,157 @@
+<?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.
+-->
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document>
+  <header>
+    <title>Notification</title>
+  </header>
+  <body>
+  
+ <p> In HCatalog 2.0 we introduce notifications for certain events happening in the system. This way applications such as Oozie can wait for those events and schedule the work that depends on them. The current version of HCatalog supports two kinds of events: </p>
+<ul>
+<li>Notification when a new partition is added</li>
+<li>Notification when a set of partitions is added</li>
+</ul>
+
+<p>No additional work is required to send a notification when a new partition is added: the existing addPartition call will send the notification message. This means that your existing code, when running with 0.2, will automatically send the notifications. </p>
+
+<section>
+<title>Notification for a New Partition</title>
+
+<p>To receive notification that a new partition has been added, you need to follow these three steps.</p>
+ 
+ <p>1. To start receiving messages, create a connection to a message bus as shown here:</p>
+ <source>
+ConnectionFactory connFac = new ActiveMQConnectionFactory(amqurl);
+Connection conn = connFac.createConnection();
+conn.start();
+ </source>
+ 
+  <p>2. Subscribe to a topic you are interested in. When subscribing on a message bus, you need to subscribe to a particular topic to receive the messages that are being delivered on that topic. </p>
+  <ul>
+  <li>  
+  <p>The topic name corresponding to a particular table is stored in table properties and can be retrieved using following piece of code: </p>
+ <source>
+HiveMetaStoreClient msc = new HiveMetaStoreClient(hiveConf);
+String topicName = msc.getTable("mydb", "myTbl").getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME);
+ </source>
+ </li>
+  
+  <li>  
+  <p>Use the topic name to subscribe to a topic as follows: </p>
+ <source>
+Session session = conn.createSession(true, Session.SESSION_TRANSACTED);
+Destination hcatTopic = session.createTopic(topicName);
+MessageConsumer consumer = session.createConsumer(hcatTopic);
+consumer.setMessageListener(this);
+ </source>
+ </li>
+  </ul>
+
+  <p>3. To start receiving messages you need to implement the JMS interface <code>MessageListener</code>, which, in turn, will make you implement the method <code>onMessage(Message msg)</code>. This method will be called whenever a new message arrives on the message bus. The message contains a partition object representing the corresponding partition, which can be retrieved as shown here: </p>
+ <source>
+@Override
+   public void onMessage(Message msg) {
+      // We are interested in only add_partition events on this table.
+      // So, check message type first.
+      if(msg.getStringProperty(HCatConstants.HCAT_EVENT).equals(HCatConstants.HCAT_ADD_PARTITION_EVENT)){
+          Object obj = (((ObjectMessage)msg).getObject());
+      }
+   }
+ </source>
+ 
+  <p>You need to have a JMS jar in your classpath to make this work. Additionally, you need to have a JMS provider’s jar in your classpath. HCatalog uses ActiveMQ as a JMS provider. In principle, any JMS provider can be used in client side; however, ActiveMQ is recommended. ActiveMQ can be obtained from: http://activemq.apache.org/activemq-550-release.html </p>
+</section>
+
+<section>
+<title>Notification for a Set of Partitions</title>
+
+<p>The example code below illustrates how to send a notification when a set of partitions has been added.</p>
+
+<source>
+HiveMetaStoreClient msc = new HiveMetaStoreClient(conf);
+
+// Create a map, specifying partition key names and values
+Map&lt;String,String&gt; partMap = new HashMap&lt;String, String&gt;();
+partMap.put("date","20110711");
+partMap.put("country","*");
+
+// Mark the partition as "done"
+msc.markPartitionForEvent("mydb", "mytbl", partMap, PartitionEventType.LOAD_DONE);
+</source>
+
+<p>To receive this notification, the consumer needs to do the following:</p>
+<ol>
+<li>Repeat steps one and two from above to establish the connection to the notification system and to subscribe to the topic.</li>
+<li>Receive the notification as shown in this example:
+<source>
+HiveMetaStoreClient msc = new HiveMetaStoreClient(conf);
+
+// Create a map, specifying partition key names and values
+Map&lt;String,String&gt; partMap = new HashMap&lt;String, String&gt;();
+partMap.put("date","20110711");
+partMap.put("country","*");
+
+// Mark the partition as "done"
+msc.markPartitionForEvent("mydb", "mytbl", partMap, PartitionEventType.LOAD_DONE);
+</source>
+</li>
+</ol>
+
+
+
+<p>If the consumer has registered with the message bus and is currently live, it will get the callback from the message bus once the producer marks the partition as "done".  Alternatively, the consumer can ask explicitly for a particular partition from the metastore. The following code illustrates the usage from a consumer's perspective:</p>
+
+<source>
+// Enquire to metastore whether a particular partition has been marked or not.
+boolean marked = msc.isPartitionMarkedForEvent("mydb", "mytbl", partMap, PartitionEventType.LOAD_DONE);
+
+// Or register to a message bus and get asynchronous callback.
+ConnectionFactory connFac = new ActiveMQConnectionFactory(amqurl);
+Connection conn = connFac.createConnection();
+conn.start();
+Session session = conn.createSession(true, Session.SESSION_TRANSACTED);
+Destination hcatTopic = session.createTopic(topic);
+MessageConsumer consumer = session.createConsumer(hcatTopic);
+consumer.setMessageListener(this);
+
+
+public void onMessage(Message msg) {
+
+                                
+  MapMessage mapMsg = (MapMessage)msg;
+  Enumeration&lt;String&gt; keys = mapMsg.getMapNames();
+  
+  // Enumerate over all keys. This will print key value pairs specifying the particular partition 
+  // which was marked done. In this case, it will print:
+  // date : 20110711
+  // country: *
+
+  while(keys.hasMoreElements()){
+    String key = keys.nextElement();
+    System.out.println(key + " : " + mapMsg.getString(key));
+  }
+  System.out.println("Message: "+msg);
+</source>
+
+<p>Notification is enabled by default. To disable notification, you need to leave <code>hive.metastore.event.listeners</code> blank or remove it from <code>hive-site.xml.</code></p>
+</section>
+    
+  </body>
+</document>

Modified: incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/site.xml
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/site.xml?rev=1149292&r1=1149291&r2=1149292&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/site.xml (original)
+++ incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/site.xml Thu Jul 21 18:11:11 2011
@@ -40,11 +40,14 @@ See http://forrest.apache.org/docs/linki
 
   <docs label="HCatalog"> 
     <index label="Overview" href="index.html" />
-    <index label="Pig Load &amp; Store " href="loadstore.html" />
-    <index label="MapReduce Input &amp; Output " href="inputoutput.html" />
-    <index label="Cmd Line Interface " href="cli.html" />
-    <index label="Supported data formats" href="supportedformats.html" />
     <index label="Installation" href="install.html" />
+    <index label="Load &amp; Store Interfaces" href="loadstore.html" />
+    <index label="Input &amp; Output Interfaces " href="inputoutput.html" />
+    <index label="Command Line Interface " href="cli.html" />
+    <index label="Storage Formats" href="supportedformats.html" />
+    <index label="Dynamic Partitioning" href="dynpartition.html" />
+    <index label="Notification" href="notification.html" />    
+
     <api   label="API Docs" href="api/index.html"/>
   </docs>  
 

Modified: incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/supportedformats.xml
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/supportedformats.xml?rev=1149292&r1=1149291&r2=1149292&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/supportedformats.xml (original)
+++ incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/supportedformats.xml Thu Jul 21 18:11:11 2011
@@ -19,7 +19,7 @@
 
 <document>
   <header>
-    <title>Supported storage formats</title>
+    <title>Storage Formats</title>
   </header>
   <body>
   <p>HCatalog can read PigStorage and RCFile formatted files. The input drivers for the formats are PigStorageInputDriver, ULTInputDriver and RCFileInputDriver respectively. HCatalog currently produces only RCFile formatted output. The output driver for the same is RCFileOutputDriver. </p>

Modified: incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/tabs.xml
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/tabs.xml?rev=1149292&r1=1149291&r2=1149292&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/tabs.xml (original)
+++ incubator/hcatalog/trunk/src/docs/src/documentation/content/xdocs/tabs.xml Thu Jul 21 18:11:11 2011
@@ -30,6 +30,6 @@
    Tabs can be embedded to a depth of two. The second level of tabs will only 
     be displayed when their parent tab is selected.    
   -->
-  <tab label="HCatalog 0.1.0 Documentation" dir="" type="visible" /> 
+  <tab label="HCatalog 0.2.0 Documentation" dir="" type="visible" /> 
 
 </tabs>