You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by Apache Wiki <wi...@apache.org> on 2009/11/06 20:00:50 UTC

[Hadoop Wiki] Update of "Hive/DeveloperGuide" by Ning Zhang

Dear Wiki user,

You have subscribed to a wiki page or wiki category on "Hadoop Wiki" for change notification.

The "Hive/DeveloperGuide" page has been changed by Ning Zhang.
http://wiki.apache.org/hadoop/Hive/DeveloperGuide?action=diff&rev1=22&rev2=23

--------------------------------------------------

  == Code Organization and a brief architecture ==
  === Introduction ===
  Hive comprises of 3 main components:
+ 
   * Serializers/Deserializers (trunk/serde) - This component has the framework libraries that allow users to develop serializers and deserializers for their own data formats. This component also contains some builtin serialization/deserialization families.
   * MetaStore (trunk/metastore) - This component implements the metadata server which is used to hold all the information about tables and partitions that are in the warehouse.
   * Query Processor (trunk/ql) - This component implements the processing framework for converting SQL to a graph of map/reduce jobs and also the execution time framework to run those jobs in the order of dependencies.
  
  Apart from these major components, Hive also contains a number of other components. These are as follows:
+ 
   * Command Line Interface (trunk/cli) - This component has all the java code used by the Hive command line interface.
   * Hive Server (trunk/service) - This component implements all the APIs that can be used by other clients (such as JDBC drivers) to talk to Hive.
   * Common (trunk/common) - This component contains common infrastructure needed by the rest of the code. Currently, this contains all the java sources for managing and passing Hive configurations(HiveConf) to all the other code components.
@@ -19, +21 @@

   * Scripts (trunk/bin) - This component contains all the scripts provided in the distribution including the scripts to run the Hive cli(bin/hive).
  
  The following top level directories contain helper libraries, packaged configuration files etc..:
+ 
   * trunk/conf - This directory contains the packaged hive-default.xml and hive-site.xml.
   * trunk/data - This directory contains some data sets and configurations used in the hive tests.
   * trunk/ivy - This directory contains the ivy files used by the build infrastructure to manage dependencies on different hadoop versions.
@@ -28, +31 @@

  
  === SerDe ===
  What is !SerDe
+ 
-   * !SerDe is a short name for Serializer and Deserializer.
+  * !SerDe is a short name for Serializer and Deserializer.
-   * Hive uses SerDe (and !FileFormat) to read from/write to tables.
+  * Hive uses SerDe (and !FileFormat) to read from/write to tables.
-   * HDFS files --(!InputFileFormat)--> <key, value> --(Deserializer)--> Row object
+  * HDFS files --(!InputFileFormat)--> <key, value> --(Deserializer)--> Row object
-   * Row object --(Serializer)--> <key, value> --(!OutputFileFormat)--> HDFS files
+  * Row object --(Serializer)--> <key, value> --(!OutputFileFormat)--> HDFS files
  
  Note that the "key" part is ignored when reading, and is always a constant when writing. Basically the row object is only stored into the "value".
  
@@ -40, +44 @@

  Note that org.apache.hadoop.hive.serde is the deprecated old serde library. Please look at org.apache.hadoop.hive.serde2 for the latest version.
  
  Hive currently use these FileFormat classes to read and write HDFS files:
+ 
-   * !TextInputFormat/NoKeyTextOutputFormat: These 2 classes read/write data in plain text file format.
+  * !TextInputFormat/NoKeyTextOutputFormat: These 2 classes read/write data in plain text file format.
-   * !SequenceFileInputFormat/SequenceFileOutputFormat: These 2 classes read/write data in hadoop !SequenceFile format.
+  * !SequenceFileInputFormat/SequenceFileOutputFormat: These 2 classes read/write data in hadoop !SequenceFile format.
  
  Hive currently use these !SerDe classes to serialize and deserialize data:
+ 
-   * !MetadataTypedColumnsetSerDe: This !SerDe is used to read/write delimited records like CSV, tab-separated control-A separated records (sorry, quote is not supported yet.)
+  * !MetadataTypedColumnsetSerDe: This !SerDe is used to read/write delimited records like CSV, tab-separated control-A separated records (sorry, quote is not supported yet.)
-   * !ThriftSerDe: This !SerDe is used to read/write thrift serialized objects.  The class file for the Thrift object must be loaded first.
+  * !ThriftSerDe: This !SerDe is used to read/write thrift serialized objects.  The class file for the Thrift object must be loaded first.
-   * !DynamicSerDe: This !SerDe also read/write thrift serialized objects, but it understands thrift DDL so the schema of the object can be provided at runtime.  Also it supports a lot of different protocols, including !TBinaryProtocol, !TJSONProtocol, TCTL!SeparatedProtocol (which writes data in delimited records).
+  * !DynamicSerDe: This !SerDe also read/write thrift serialized objects, but it understands thrift DDL so the schema of the object can be provided at runtime.  Also it supports a lot of different protocols, including !TBinaryProtocol, !TJSONProtocol, TCTL!SeparatedProtocol (which writes data in delimited records).
  
  How to write your own !SerDe:
+ 
-   * In most cases, users want to write a Deserializer instead of a !SerDe, because users just want to read their own data format instead of writing to it.
+  * In most cases, users want to write a Deserializer instead of a !SerDe, because users just want to read their own data format instead of writing to it.
-   * For example, the !RegexDeserializer will deserialize the data using the configuration parameter 'regex', and possibly a list of column names (see serde2.MetadataTypedColumnsetSerDe). Please see serde2/Deserializer.java for details.
+  * For example, the !RegexDeserializer will deserialize the data using the configuration parameter 'regex', and possibly a list of column names (see serde2.MetadataTypedColumnsetSerDe). Please see serde2/Deserializer.java for details.
-   * If your !SerDe supports DDL (basically, !SerDe with parameterized columns and column types), you probably want to implement a Protocol based on !DynamicSerDe, instead of writing a !SerDe from scratch. The reason is that the framework passes DDL to !SerDe through "thrift DDL" format, and it's non-trivial to write a "thrift DDL" parser.
+  * If your !SerDe supports DDL (basically, !SerDe with parameterized columns and column types), you probably want to implement a Protocol based on !DynamicSerDe, instead of writing a !SerDe from scratch. The reason is that the framework passes DDL to !SerDe through "thrift DDL" format, and it's non-trivial to write a "thrift DDL" parser.
  
  Some important points of !SerDe:
+ 
-   * !SerDe, not the DDL, defines the table schema. Some !SerDe implementations use the DDL for configuration, but !SerDe can also override that.
+  * !SerDe, not the DDL, defines the table schema. Some !SerDe implementations use the DDL for configuration, but !SerDe can also override that.
-   * Column types can be arbitrarily nested arrays, maps and structures.
+  * Column types can be arbitrarily nested arrays, maps and structures.
-   * The callback design of !ObjectInspector allows lazy deserialization with CASE/IF or when using complex or nested types.
+  * The callback design of !ObjectInspector allows lazy deserialization with CASE/IF or when using complex or nested types.
  
  ==== ObjectInspector ====
  Hive uses !ObjectInspector to analyze the internal structure of the row object and also the structure of the individual columns.
  
  !ObjectInspector provides a uniform way to access complex objects that can be stored in multiple formats in the memory, including:
+ 
   * Instance of a Java class (Thrift or native Java)
   * A standard Java object (we use java.util.List to represent Struct and Array, and use java.util.Map to represent Map)
   * A lazily-initialized object (For example, a Struct of string fields stored in a single Java string object with starting offset for each field)
  
- A complex object can be represented by a pair of !ObjectInspector and Java Object.
- The !ObjectInspector not only tells us the structure of the Object, but also gives us ways to access the internal fields inside the Object.
+ A complex object can be represented by a pair of !ObjectInspector and Java Object. The !ObjectInspector not only tells us the structure of the Object, but also gives us ways to access the internal fields inside the Object.
- 
  
  === MetaStore ===
  MetaStore contains metadata regarding tables, partitions and databases. This is used by Query Processor during plan generation.
@@ -77, +84 @@

   * Object Store - ObjectStore class handles access to the actual metadata is stored in the SQL store. The current implementation uses JPOX ORM solution which is based of JDA specification. It can be used with any database that is supported by JPOX. New meta stores (file based or xml based) can added by implementing the interface MetaStore. FileStore is a partial implementation of an older version of metastore which may be deprecated soon.
   * Metastore Client - There are python, java, php thrift clients in metastore/src. Java generated client is extended with HiveMetaStoreClient which is used by Query Processor (ql/metadta). This is the main interface to all other Hive components.
  
- 
  === Query Processor ===
  The following are the main components of the Hive Query Processor:
+ 
   * Parse and SemanticAnalysis (ql/parse) - This component contains the code for parsing SQL, converting it into Abstract Syntax Trees, converting the Abstract Syntax Trees into Operator Plans and finally converting the operator plans into a directed graph of tasks which are executed by Driver.java.
   * Optimizer (ql/optimizer) - This component contains some simple rule based optimizations like pruning non referenced columns from table scans (column pruning) that the Hive Query Processor does while converting SQL to a series of map/reduce tasks.
   * Plan Components (ql/plan) - This component contains the classes (which are called descriptors), that are used by the compiler (Parser, SemanticAnalysis and Optimizer) to pass the information to operator trees that is used by the execution code.
@@ -101, +108 @@

  ==== Plan ====
  ==== Operators ====
  ==== UDFs and UDAFs ====
- 
  == Compiling Hive ==
  Hive can be made to compile against different versions of Hadoop.
  
  === Default Mode ===
  From the root of the source tree:
+ 
  {{{
  ant package
  }}}
- 
  will make Hive compile against hadoop version 0.19.0. Note that:
+ 
   * Hive uses Ivy to download the hadoop-0.19.0 distribution. However once downloaded, it's cached and not downloaded multiple times
   * This will create a distribution directory in build/dist (relative to the source root) from where one can launch Hive. This distribution should only be used to execute queries against hadoop branch 0.19. (Hive is not sensitive to minor revisions of Hadoop versions).
  
  === Advanced Mode ===
   * One can specify a custom distribution directory by using:
+ 
  {{{
  ant -Dtarget.dir=<my-install-dir> package
  }}}
   * One can specify a version of hadoop other than 0.19.0 by using (using 0.17.1 as an example):
+ 
  {{{
  ant -Dhadoop.version=0.17.1 package
  }}}
   * One can also compile against a custom version of the Hadoop tree (only release 0.4 and above). This is also useful if running Ivy is problematic (in disconnected mode for example) - but a hadoop tree is available. This can be done by specifying the root of the hadoop source tree to be used, for example:
+ 
  {{{
  ant -Dhadoop.root=~/src/hadoop-19/build/hadoop-0.19.2-dev -Dhadoop.version=0.19.2-dev
  }}}
  note that:
+ 
-   * hive's build script assumes that {{{hadoop.root}}} is pointing to a distribution tree for hadoop created by running ant package in hadoop
+  * hive's build script assumes that {{{hadoop.root}}} is pointing to a distribution tree for hadoop created by running ant package in hadoop
-   * {{{hadoop.version}}} must match the version used in building hadoop
+  * {{{hadoop.version}}} must match the version used in building hadoop
  
  In this particular example - {{{~/src/hadoop-19}}} is a checkout of the hadoop 19 branch that uses {{{0.19.2-dev}}} as default version and creates a distribution directory in {{{build/hadoop-0.19.2-dev}}} by default.
  
+ == Unit tests and debugging ==
- 
- == Unit tests ==
  === Layout of the unit tests ===
  Hive uses junit for unit tests. Each of the 3 main components of Hive have their unit test implementations in the corresponding src/test directory e.g. trunk/metastore/src/test has all the unit tests for metastore, trunk/serde/src/test has all the unit tests for serde and trunk/ql/src/test has all the unit tests for the query processor. The metastore and serde unit tests provide the !TestCase implementations for junit. The query processor tests on the other hand are generated using Velocity. The main directories under trunk/ql/src/test that contain these tests and the corresponding results are as follows:
+ 
   * Test Queries:
    * queries/clientnegative - This directory contains the query files (.q files) for the negative test cases. These are run through the CLI classes and therefore test the entire query processor stack.
    * queries/clientpositive - This directory contains the query files (.q files) for the positive test cases. Thesre are run through the CLI classes and therefore test the entire query processor stack.
@@ -158, +169 @@

  === Tables in the unit tests ===
  === Running unit tests ===
  Run all tests:
+ 
  {{{
  ant test
  }}}
- 
  Run all positive test queries:
+ 
  {{{
  ant test -Dtestcase=TestCliDriver
  }}}
- 
  Run a specific positive test query:
+ 
  {{{
  ant test -Dtestcase=TestCliDriver -Dqfile=groupby1.q
  }}}
- 
  The about test produces the following files:
+ 
   * {{{build/ql/test/TEST-org.apache.hadoop.hive.cli.TestCliDriver.txt}}} - Log output for the test.  This can be helpful when examining test failures.
   * {{{build/ql/test/logs/groupby1.q.out}}} - Actual query result for the test.  This result is compared to the expected result as part of the test.
  
@@ -180, +192 @@

  First, write a new myname.q in ql/src/test/queries/clientpositive
  
  Then, run the test with the query and overwrite the result (useful when you add a new test)
+ 
  {{{
  ant test -Dtestcase=TestCliDriver -Dqfile=myname.q -Doverwrite=true
  }}}
- 
  Then we can create a patch by:
+ 
  {{{
  svn add ql/src/test/queries/clientpositive/myname.q ql/src/test/results/clientpositive/myname.q.out
  svn diff > patch.txt
  }}}
+ Debugging Hive
+ 
+ === Debugging Hive code ===
+ Hive code includes both client-side code (e.g., compiler, semantic analyzer, and optimizer of HiveQL) and server-side code (e.g., operator/task/SerDe implementations). The client-side code are running on your local machine so you can easily debug it using Eclipse the same way as you debug a regular local Java code.  The server-side code is distributed and running on the Hadoop cluster, so debugging server-side Hive code is a little bit complicated. In addition to printing to log files using log4j, you can also attach the debugger to a different JVM under unit test (single machine mode). Below are the steps on how to debug on server-side code.
+ 
+  * Compile Hive code with javac.debug=on. Under Hive checkout directory. {{{
+     > ant -Djavac.debug=on package
+   }}} If you have already built Hive without javac.debug=on, you can clean the build and then run the above command. {{{
+     > ant clean  # not necessary if the first time to compile
+     > ant -Djavac.debug=on package 
+   }}}
+  * Run ant test with additional options to tell the Java VM that is running Hive server-side code to wait for the debugger to attach. First define some convenient macros for debugging. You can put it in your .bashrc or .cshrc. {{{
+     > export HIVE_DEBUG_PORT=8000
+     > export $HIVE_DEBUG="-Xdebug -Xrunjdwp:transport=dt_socket,address=${HIVE_DEBUG_PORT},server=y,suspend=y"
+   }}} In particular HIVE_DEBUG_PORT is the port number that the JVM is listening on and the debugger will attach to. Then run the unit test as follows: {{{
+     > HADOOP_OPTS=$HIVE_DEBUG ant test -Dtestcase=TestCliDriver -Dqfile=<mytest>.q
+   }}} The unit test will run until it shows: {{{
+      [junit] Listening for transport dt_socket at address: 8000
+   }}}
+  * Now, you can use jdb to attach to port 8000 to debug{{{
+     > jdb -attach 8000
+ }}}  or if you are running Eclipse and the Hive projects are already imported, you can debug with Eclipse. Under Eclipse Run -> Debug Configurations, find "Remote Java Application" at the bottom of the left panel. There should be a MapRedTask configuration already. If there is no such configuration, you can create one with the following property:
+      * Name: any time such as MapRedTask
+      * Project:  the Hive project that you imported.
+      * Connection Type: Standard (Socket Attach)
+      * Connection Properties:
+        * Host: localhost
+        * Port: 8000
+      Then hit the "Debug" button and Eclipse will attach to the JVM listening on port 8000 and continue running till the end. If you define breakpoints in the source code before hitting the "Debug" button, it will stop there. The rest is the same as debugging client-side Hive.
  
  
  == Pluggable interfaces ==
- 
  === File Formats ===
  Please refer to [[http://www.slideshare.net/ragho/hive-user-meeting-august-2009-facebook|Hive User Group Meeting August 2009]] Page 59-63.