Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

Info
titleVersion information

Avro Data Stored in HBase Columns

As of Hive 0.9.0 the HBase integration requires at least HBase 0.92, earlier versions of Hive were working with HBase 0.89/0.90


Info
titleVersion information

Hive 1.x will remain compatible with HBase 0.98.x and lower versions. Hive 2.x will be compatible with HBase 1.x and higher. (See HIVE-10990 for details.) Consumers wanting to work with HBase 1.x using Hive 1.x will need to compile Hive 1.x stream code themselves.

Introduction

This page documents the Hive/HBase integration support originally introduced in HIVE-705. This feature allows Hive QL statements to access HBase tables for both read (SELECT) and write (INSERT). It is even possible to combine access to HBase tables with native Hive tables via joins and unions.

...

No Format
CREATE TABLE hbase_table_1(key int, value string) 
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf1:val")
TBLPROPERTIES ("hbase.table.name" = "xyz", "hbase.mapred.output.outputtable" = "xyz");

The hbase.columns.mapping property is required and will be explained in the next section. The hbase.table.name property is optional; it controls the name of the table as known by HBase, and allows the Hive table to have a different name. In this example, the table is known as hbase_table_1 within Hive, and as xyz within HBase. If not specified, then the Hive and HBase and HBase table names will be identical. The hbase.mapred.output.outputtable property is optional; it's needed if you plan to insert data to the table (the property is used by hbase.mapreduce.TableOutputFormat)

After executing the command above, you should be able to see the new (empty) table in the HBase shell:

...

No Format
CREATE EXTERNAL TABLE hbase_table_2(key int, value string) 
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf1:val")
TBLPROPERTIES("hbase.table.name" = "some_existing_table", "hbase.mapred.output.outputtable" = "some_existing_table");

Again, hbase.columns.mapping is required (and will be validated against the existing HBase table's column families), whereas hbase.table.name is optional.

Column Mapping

. The hbase.mapred.output.outputtable is optional.

Column Mapping

There are two SERDEPROPERTIES that control the mapping of HBase columns to Hive:

...

  • for each Hive column, the table creator must specify a corresponding entry in the comma-delimited hbase.columns.mapping string (so for a Hive table with n columns, the string should have n entries); whitespace should not be used in between entries since these will be interperted as part of the column name, which is almost certainly not what you want
  • a mapping entry must be either :key, :timestamp or of the form column-family-name:[column-name][#(binary|string) (the type specification that delimited by # was added in Hive 0.9.0, earlier versions interpreted everything as strings)
    • If no type specification is given the value from hbase.table.default.storage.type will be used
    • Any prefixes of the valid values are valid too (i.e. #b instead of #binary)
    • If you specify a column as binary the bytes in the corresponding HBase cells are expected to be of the form that HBase's Bytes class yields.
  • there must be exactly one :key mapping (we don't support compound keys yetthis can be mapped either to a string or struct column–see Simple Composite Keys and Complex Composite Keys)
  • (note that before HIVE-1228 in Hive 0.6, :key was not supported, and the first Hive column implicitly mapped to the key; as of Hive 0.6, it is now strongly recommended that you always specify the key explictly; we will drop support for implicit key mapping in the future)
  • if no column-name is given, then the Hive column will map to all columns in the corresponding HBase column family, and the Hive MAP datatype must be used to allow access to these (possibly sparse) columns
  • Since HBase 1.1 (HBASE-2828) there is currently no a way to access the HBase timestamp attribute , and queries always access data with the latest timestamp.
  • Since HBase does not associate datatype information with columns, the serde converts everything to string representation before storing it in HBase; there is currently no way to plug in a custom serde per column
  • using the special :timestamp mapping. It needs to be either bigint or timestamp.
  • it is not necessary to reference every it is not necessary to reference every HBase column family, but those that are not mapped will be inaccessible via the Hive table; it's possible to map multiple Hive tables to the same HBase table

...

No Format
CREATE TABLE hbase_table_1(key int, value map<int,int>) 
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
WITH SERDEPROPERTIES (
"hbase.columns.mapping" = ":key,cf:"
);
FAILED: Error in metadata: java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException org.apache.hadoop.hive.hbase.HBaseSerDe: hbase column family 'cf:' should be mapped to map<string,?> but is mapped to map<int,int>)

...

Hive

...

MAP to HBase Column

...

Prefix

Also note that starting with Hive 0.12, wildcards can also be used to retrieve columns. For instance, if you want to retrieve all columns in HBase that start with the prefix "col_prefix", a query like the following should workTable definitions such as the following are illegal because a
Hive column mapped to an entire column family must have MAP type:

No Format
CREATE TABLE hbase_table_1(keyvalue intmap<string,int>, valuerow_key stringint) 
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
WITH SERDEPROPERTIES (
"hbase.columns.mapping" = "cf:keycol_prefix.*,cf:key"
);
FAILED: Error in metadata: java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException org.apache.hadoop.hive.hbase.HBaseSerDe: hbase column family 'cf:' should be mapped to map<string,?> but is mapped to string)

Example with binary columns

Relying on the default value of hbase.table.default.storage.type:

No Format
CREATE TABLE hbase_table_1 (key int, value string, foobar double)
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
WITH SERDEPROPERTIES (
"hbase.columns.mapping" = ":key#b,cf:val,cf:foo#b"
);

Specifying hbase.table.default.storage.type:

...

The same restrictions apply though. That is, the key of the map should be a string as it maps to the HBase column name and the value can be the type of value that you are retrieving. One other restriction is that all the values under the given prefix should be of the same type. That is, all of them should be of type "int" or type "string" and so on.

Hiding Column Prefixes

Starting with Hive 1.3.0, it is possible to hide column prefixes in select query results.  There is the SerDe boolean property hbase.columns.mapping.prefix.hide (false by default), which defines if the prefix should be hidden in keys of Hive map:

No Format
CREATE TABLE hbase_table_1(tags map<string,int>, row_key string) 
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
WITH SERDEPROPERTIES (
"hbase.columns.mapping" = "cf:tag_.*,:key",
"hbase.columns.mapping.prefix.hide" = "true"
);

Then a value of the column "tags" (select tags from hbase_table_1) will be:

"x" : 1

instead of:

"tag_x" : 1

Illegal: Hive Primitive to HBase Column Family

Table definitions such as the following are illegal because a
Hive column mapped to an entire column family must have MAP type:

No Format
CREATE TABLE hbase_table_1(key int, value string) 
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
WITH SERDEPROPERTIES (
"hbase.columns.mapping" = ":key,cf:"
);
FAILED: Error in metadata: java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException org.apache.hadoop.hive.hbase.HBaseSerDe: hbase column family 'cf:' should be mapped to map<string,?> but is mapped to string)

Example with Binary Columns

Relying on the default value of hbase.table.default.storage.type:

No Format
CREATE TABLE hbase_table_1 (key int, value string, foobar double)
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
WITH SERDEPROPERTIES (
"hbase.columns.mapping" = ":key#b,cf:val,cf:foo#b"
);

Specifying hbase.table.default.storage.type:

No Format
CREATE TABLE hbase_table_1 (key int, value string, foobar double)
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
WITH SERDEPROPERTIES (
"hbase.columns.mapping" = ":key,cf:val#s,cf:foo",
"hbase.table.default.storage.type" = "binary"
);

Simple Composite Row Keys

Info
titleVersion information

As of Hive 0.13.0

Hive can read and write delimited composite keys to HBase by mapping the HBase row key to a Hive struct, and using ROW FORMAT DELIMITED...COLLECTION ITEMS TERMINATED BY. Example:

Code Block
-- Create a table with a composite row key consisting of two string fields, delimited by '~'
CREATE EXTERNAL TABLE delimited_example(key struct<f1:string, f2:string>, value string) 
ROW FORMAT DELIMITED 
COLLECTION ITEMS TERMINATED BY '~' 
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' 
WITH SERDEPROPERTIES ( 
  'hbase.columns.mapping'=':key,f:c1');

Complex Composite Row Keys and HBaseKeyFactory

Info

As of Hive 0.14.0 with HIVE-6411 (0.13.0 also supports complex composite keys, but using a different interface–see HIVE-2599 for that interface)

For more complex use cases, Hive allows users to specify an HBaseKeyFactory which defines the mapping of a key to fields in a Hive struct. This can be configured using the property "hbase.composite.key.factory" in the SERDEPROPERTIES option:

Code Block
-- Parse a row key with 3 fixed width fields each of width 10
-- Example taken from: https://svn.apache.org/repos/asf/hive/trunk/hbase-handler/src/test/queries/positive/hbase_custom_key2.q
CREATE TABLE hbase_ck_4(key struct<col1:string,col2:string,col3:string>, value string)
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
WITH SERDEPROPERTIES (
    "hbase.table.name" = "hbase_custom2",
    "hbase.mapred.output.outputtable" = "hbase_custom2",
    "hbase.columns.mapping" = ":key,cf:string",
    "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory2");

"hbase.composite.key.factory" should be the fully qualified class name of a class implementing HBaseKeyFactory. See SampleHBaseKeyFactory2 for a fixed length example in the same package. This class must be on your classpath in order for the above example to work. TODO: place these in an accessible place; they're currently only in test code.

Avro Data Stored in HBase Columns

Info

As of Hive 0.14.0 with HIVE-6147

Hive 0.14.0 onward supports storing and querying Avro objects in HBase columns by making them visible as structs to Hive. This allows Hive to perform ad hoc analysis of HBase data which can be deeply structured. Prior to 0.14.0, the HBase Hive integration only supported querying primitive data types in columns.

An example HiveQL statement where test_col_fam is the column family and test_col is the column name:

Code Block
CREATE EXTERNAL TABLE test_hbase_avro
ROW FORMAT SERDE 'org.apache.hadoop.hive.hbase.HBaseSerDe' 
STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' 
WITH SERDEPROPERTIES (
	"hbase.columns.mapping" = ":key,test_col_fam:test_col", 
	"test_col_fam.test_col.serialization.type" = "avro",
	"test_col_fam.test_col.avro.schema.url" = "hdfs://testcluster/tmp/schema.avsc")
TBLPROPERTIES (
    "hbase.table.name" = "hbase_avro_table",
    "hbase.mapred.output.outputtable" = "hbase_avro_table",
    "hbase.struct.autogenerate"="true");

The important properties to note are the following three:

Code Block
"test_col_fam.test_col.serialization.type" = "avro"

This property tells Hive that the given column under the given column family is an Avro column, so Hive needs to deserialize it accordingly.

Code Block
"test_col_fam.test_col.avro.schema.url" = "hdfs://testcluster/tmp/schema.avsc"

Using this property you specify where the reader schema is for the column that will be used to deserialize. This can be on HDFS like mentioned here, or provided inline using something like "test_col_fam.test_col.avro.schema.literal" property. If you have a custom store where you store this schema, you can write a custom implementation of AvroSchemaRetriever and plug that in using the "avro.schema.retriever property" using a property like "test_col_fam.test_col.avro.schema.retriever". You would need to ensure that the jar with this custom class is on the Hive classpath. For a usage discussion and links to other resources, see HIVE-6147.

Code Block
"hbase.struct.autogenerate" = "true"

Specifying this property lets Hive auto-deduce the columns and types using the schema that was provided. This allows you to avoid manually creating the columns and types for Avro schemas, which can be complicated and deeply nested.

...

Put Timestamps

Info
titleVersion information

As of Hive 0.9.0

...

Jira Issues
urlhttps://issues.apache.org/jira/sr/jira.issueviews:searchrequest-xml/temp/SearchRequest.xml?jqlQuery=project+%3D+HIVE+AND+component+in+%28%22HBase+Handler%22%29+and+Resolution+%3D+unresolved&tempMax=1000