Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: add link to DDL doc for JsonSerde

Developer Guide

Table of Contents

Code Organization and a

...

Brief Architecture

Introduction

Hive has 3 main components:

...

  • 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.
  • Ant Utilities (trunk/ant) - This component contains the implementation of some ant tasks that are used by the build infrastructure.
  • Scripts (trunk/bin) - This component contains all the scripts provided in the distribution including the scripts to run the Hive cliHive 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 Hive tests.
  • trunk/ivy - This directory contains the ivy Ivy files used by the build infrastructure to manage dependencies on different hadoop Hadoop versions.
  • trunk/lib - This directory contains the run time libraries needed by Hive.
  • trunk/testlibs - This directory contains the junit.jar used by the junit JUnit target in the build infrastructure.
  • trunk/testutils (Deprecated)

Hive SerDe

What is a

...

SerDe?

  • !SerDe is a short name for "Serializer and Deserializer."
  • Hive uses SerDe (and ! FileFormat) to read and write table rows.
  • HDFS files !InputFileFormat)--> InputFileFormat --> <key, value> --(> Deserializer --> Row object
  • 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 row object is stored into the "value".

...

Note that org.apache.hadoop.hive.serde is the deprecated old serde SerDe library. Please look at org.apache.hadoop.hive.serde2 for the latest version.

Hive currently use uses these FileFormat classes to read and write HDFS files:

  • !TextInputFormat/HiveIgnoreKeyTextOutputFormat: These 2 classes read/write data in plain text file format.
  • !SequenceFileInputFormat/SequenceFileOutputFormat: These 2 classes read/write data in hadoop !Hadoop SequenceFile format.

Hive currently use uses 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).)!ThriftSerDe
  • LazySimpleSerDe: This

    !

    SerDe

    is

    can be 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).

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.
  • 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.

Some important points about !SerDe:

  • !SerDe, not the DDL, defines the table schema. Some !SerDe implementations use the DDL for configuration, but the !SerDe can also override that.
  • 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.

    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)

...

  • the same data format as MetadataTypedColumnsetSerDe and TCTLSeparatedProtocol, however, it creates Objects in a lazy way which provides better performance. Starting in Hive 0.14.0 it also supports read/write data with a specified encode charset, for example:

    Code Block
    ALTER TABLE person SET SERDEPROPERTIES ('serialization.encoding'='GBK');

    LazySimpleSerDe can treat 'T', 't', 'F', 'f', '1', and '0' as extended, legal boolean literals if the configuration property hive.lazysimple.extended_boolean_literal is set to true (Hive 0.14.0 and later). The default is false, which means only 'TRUE' and 'FALSE' are treated as legal boolean literals.

  • 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, TCTLSeparatedProtocol (which writes data in delimited records).

Also:

  • For JSON files, JsonSerDe was added in Hive 0.12.0. An Amazon SerDe is available at s3://elasticmapreduce/samples/hive-ads/libs/jsonserde.jar for releases prior to 0.12.0.
  • An Avro SerDe was added in Hive 0.9.1.  Starting in Hive 0.14.0 its specification is implicit with the STORED AS AVRO clause.
  • A SerDe for the ORC file format was added in Hive 0.11.0.
  • A SerDe for Parquet was added via plug-in in Hive 0.10 and natively in Hive 0.13.0.
  • A SerDe for CSV was added in Hive 0.14.

See SerDe for detailed information about input and output processing. Also see Storage Formats in the HCatalog manual, including CTAS Issue with JSON SerDe. For information about how to create a table with a custom or native SerDe, see Row Format, Storage Format, and SerDe.

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.
  • 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.
  • For examples, see SerDe - how to add a new SerDe below.

Some important points about SerDe:

  • SerDe, not the DDL, defines the table schema. Some SerDe implementations use the DDL for configuration, but the SerDe can also override that.
  • 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.

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.

NOTE: Apache Hive recommends that custom ObjectInspectors created for use with custom SerDes have a no-argument constructor in addition to their normal constructors for serialization purposes. See HIVE-5380 for more details.

Registration of Native SerDes

As of Hive 0.14 a registration mechanism has been introduced for native Hive SerDes.  This allows dynamic binding between a "STORED AS" keyword in place of a triplet of {SerDe, InputFormat, and OutputFormat} specification, in CreateTable statements.

The following mappings have been added through this registration mechanism:

SyntaxEquivalent

STORED AS AVRO /

STORED AS AVROFILE

ROW FORMAT SERDE
  'org.apache.hadoop.hive.serde2.avro.AvroSerDe'
  STORED AS INPUTFORMAT
  'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat'
  OUTPUTFORMAT
  'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat'

STORED AS ORC /

STORED AS ORCFILE

ROW FORMAT SERDE
  'org.apache.hadoop.hive.ql.io.orc.OrcSerde'
  STORED AS INPUTFORMAT
  'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat'
  OUTPUTFORMAT
  'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'

STORED AS PARQUET /

STORED AS PARQUETFILE

ROW FORMAT SERDE
  'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
  STORED AS INPUTFORMAT
  'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
  OUTPUTFORMAT
  'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
STORED AS RCFILE
STORED AS INPUTFORMAT
  'org.apache.hadoop.hive.ql.io.RCFileInputFormat'
  OUTPUTFORMAT
  'org.apache.hadoop.hive.ql.io.RCFileOutputFormat'
STORED AS SEQUENCEFILE
STORED AS INPUTFORMAT
  'org.apache.hadoop.mapred.SequenceFileInputFormat'
  OUTPUTFORMAT
  'org.apache.hadoop.mapred.SequenceFileOutputFormat'
STORED AS TEXTFILE
STORED AS INPUTFORMAT
  'org.apache.hadoop.mapred.TextInputFormat'
  OUTPUTFORMAT
  'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'

To add a new native SerDe with STORED AS keyword, follow these steps:

  1. Create a storage format descriptor class extending from AbstractStorageFormatDescriptor.java that returns a "stored as" keyword and the names of InputFormat, OutputFormat, and SerDe classes.

  2. Add the name of the storage format descriptor class to the StorageFormatDescriptor registration file.

MetaStore

MetaStore contains metadata regarding tables, partitions and databases. This is used by Query Processor during plan generation.

  • Metastore Server - This is the thrift Thrift server (interface defined in metastore/if/hive_metastore.if) that services metadata requests from clients. It delegates most of the requests underlying meta data store and the Hadoop file system which contains data.
  • 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 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.

...

Plan

Operators

UDFs and UDAFs

A helpful overview of the Hive query processor can be found in this Hive Anatomy slide deck.

...

A helpful overview of the Hive query processor can be found in this Hive Anatomy slide deck.

Compiling and Running Hive

Note
titleAnt to Maven

As of version 0.13 Hive uses Maven instead of Ant for its build. The following instructions are not up to date.

See the Hive Developer FAQ for updated instructions.

Hive can be made to compile against different versions of Hadoop.

...

From the root of the source tree:

Code Block

ant package

will make Hive compile against hadoop 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 Hadoop branch 0.19. (Hive is not sensitive to minor revisions of Hadoop versions).

...

  • One can specify a custom distribution directory by using:
Code Block

ant -Dtarget.dir=<my-install-dir> package
  • One can specify a version of hadoop Hadoop other than 0.19.0 by using (using 0.17.1 as an example):
Code Block

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 Hadoop tree is available. This can be done by specifying the root of the hadoop Hadoop source tree to be used, for example:
Code Block

ant -Dhadoop.root=~/src/hadoop-19/build/hadoop-0.19.2-dev -Dhadoop.version=0.19.2-dev

note that:

  • hiveHive's build script assumes that hadoop.root is pointing to a distribution tree for hadoop Hadoop created by running ant package in hadoopHadoop.
  • hadoop.version must match the version used in building hadoopHadoop.

In this particular example - ~/src/hadoop-19 is a checkout of the hadoop 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.

Run hive Hive from the command line with '$HIVE_HOME/bin/hive', where $HIVE_HOME is typically build/dist under your hive Hive repository top-level directory.

Code Block

$ build/dist/bin/hive

If hive Hive fails at runtime, try 'ant very-clean package' to delete the ivy Ivy cache before rebuilding.

Running Hive Without a Hadoop Cluster

From Thejas:

Code Block

export HIVE_OPTS='--hiveconf mapred.job.tracker=local --hiveconf fs.default.name=file:///tmp \
    --hiveconf hive.metastore.warehouse.dir=file:///tmp/warehouse \
    --hiveconf javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=/tmp/metastore_db;create=true'

...

Layout of the unit tests

Hive uses junit 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 junitJUnit. 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.
    • qureies/positive (Will be deprecated) - This directory contains the query files (.q files) for the positive test cases for the compiler. These only test the compiler and do not run the execution code.
    • queries/negative (Will be deprecated) - This directory contains the query files (.q files) for the negative test cases for the compiler. These only test the compiler and do not run the execution code.
  • Test Results:
    • results/clientnegative - The expected results from the queries in queries/clientnegative.
    • results/clientpositive - The expected results from the queries in queries/clientpositive.
    • results/compiler/errors - The expected results from the queries in queries/negative.
    • results/compiler/parse - The expected Abstract Syntax Tree output for the queries in queries/positive.
    • results/compiler/plan - The expected query plans for the queries in queries/positive.
  • Velocity Templates to Generate the testsTests:
    • templates/!TestCliDriver.vm - Generates the tests from queries/clientpositive.
    • templates/!TestNegativeCliDriver.vm - Generates the tests from queries/clientnegative.
    • templates/!TestParse.vm - Generates the tests from queries/positive.
    • templates/!TestParseNegative.vm - Generates the tests from queries/negative.Tables in the unit tests

...

Running unit tests

Note
titleAnt to Maven

As of version 0.13 Hive uses Maven instead of Ant for its build. The following instructions are not up to date.

See the Hive Developer FAQ for updated instructions.

Run all tests:

Code Block

ant package test

Run all positive test queries:

Code Block

ant test -Dtestcase=TestCliDriver

Run a specific positive test query:

Code Block

ant test -Dtestcase=TestCliDriver -Dqfile=groupby1.q

...

Run the set of unit tests matching a regex, e.g. partition_wise_fileformat tests 10-16:

Code Block

ant test -Dtestcase=TestCliDriver -Dqfile_regex=partition_wise_fileformat1[0-6]

Note that this option matches against the basename of the test without the .q suffix.

Apparently the hive Hive tests do not run successfully after a clean unless you run ant package first. Not sure why build.xml doesn't encode this dependency.

Adding new unit tests

't encode this dependency.

Adding new unit tests

Note
titleAnt to Maven

As of version 0.13 Hive uses Maven instead of Ant for its build. The following instructions are not up to date.

See the Hive Developer FAQ for updated instructions. See also Tips for Adding New Tests in Hive and How to Contribute: Add a Unit Test.

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).

Code Block

ant test -Dtestcase=TestCliDriver -Dqfile=myname.q -Doverwrite=true

Then we can create a patch by:

Code Block

svn add ql/src/test/queries/clientpositive/myname.q ql/src/test/results/clientpositive/myname.q.out
svn diff > patch.txt

Similarly, to add negative client tests, write a new query input file in ql/src/test/queries/clientnegative and run the same command, this time specifying the testcase name as ! TestNegativeCliDriver instead of ! TestCliDriver. Note that for negative client tests, the output file if created using the overwrite flag can be be found in the directory ql/src/test/results/clientnegative.See also Tips for adding new Tests.

Debugging Hive Code

Debugging Hive code

Anchor
DebuggingHiveCode
DebuggingHiveCode

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). Debugging is different for client-side and server-side code, as described below.

Debugging Client-Side Code

The client-side code are running runs on your local machine so you can easily debug it using Eclipse the same way as you debug a any regular local Java code. Here are the steps to debug code within a unit test.

  • make Make sure that you have run ant model-jar in hive/metastore and ant gen-test in hive since the last time you ran ant clean.
  • To run all of the unit tests for the Cli, open CLI:
    • Open up TestCliDriver.java
    • click Click Run->Debug Configurations, select TestCliDriver, and click Debug.
  • To run a single test within TestCliDriver.java:
    • Begin running the whole TestCli suite as before.
    • Once it finishes the setup and starts executing the JUnit tests, stop the test execution.
    • Find the desired test in the JUnit pane,
    • Right click on that test and select Debug.

Debugging Server-Side Code

The server-side code is distributed and running runs 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

    .

    :

    Code Block
    
        > 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.

    Code Block
    
        > 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.

    Code Block
    
        > 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:

    Code Block
    
        > export HADOOP_OPTS=$HIVE_DEBUG
        > ant test -Dtestcase=TestCliDriver -Dqfile=<mytest>.q
    

    The unit test will run until it shows:

    Code Block
    
         [junit] Listening for transport dt_socket at address: 8000
    


  • Now, you can use jdb to attach to port 8000 to debug

    Code Block
    
        > 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 task 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.

Debugging without Ant (Client and Server Side)

There is another way of debugging hive Hive code without going through antAnt.
You need to install hadoop Hadoop and set the environment variable HADOOP_HOME to that.

Code Block

    > export HADOOP_HOME=<your hadoop home>
 

Then, start hiveHive:

Code Block

    >  ./build/dist/bin/hive --debug
 

It will then act similar to the debugging steps outlines in Debugging Hive code. It is faster since there is no need to compile hive Hive code,
and go through antAnt. It can be used to debug both client side and server side hiveHive.

If you want to debug a particular query, start hive
and Hive and perform the stops steps needed before that query. Then start, hive start Hive again in debug to debug that query.

Code Block

    >  ./build/dist/bin/hive
    >  perform steps before the query
 


Code Block

    >  ./build/dist/bin/hive --debug
    >  run the query
 

Note that the local file system will be used, so the space no on your machine will not be released automatically (unlike debugging via antAnt, where the tables created in test are automatically dropped at the end of the test). Make sure to either drop the tables explicitly, or drop the data from /User/hive/warehouse.

...

Please refer to Hive User Group Meeting August 2009 Page 74-87.