Index: src/docs/src/documentation/content/xdocs/inputoutput.xml =================================================================== --- src/docs/src/documentation/content/xdocs/inputoutput.xml (revision 1300810) +++ src/docs/src/documentation/content/xdocs/inputoutput.xml (working copy) @@ -31,7 +31,7 @@

Authentication

- +

If a failure results in a message like "2010-11-03 16:17:28,225 WARN hive.metastore ... - Unable to connect metastore with URI thrift://..." in /tmp/<username>/hive.log, then make sure you have run "kinit <username>@FOO.COM" to get a kerberos ticket and to be able to authenticate to the HCatalog server.

If a failure results in a message like "2010-11-03 16:17:28,225 WARN hive.metastore ... - Unable to connect metastore with URI thrift://..." in /tmp/<username>/hive.log, then make sure you have run "kinit <username>@FOO.COM" to get a Kerberos ticket and to be able to authenticate to the HCatalog server.

@@ -40,48 +40,52 @@
HCatInputFormat

The HCatInputFormat is used with MapReduce jobs to read data from HCatalog managed tables.

-

HCatInputFormat exposes a new Hadoop 20 MapReduce API for reading data as if it had been published to a table. If a MapReduce job uses this InputFormat to write output, the default InputFormat configured for the table is used as the underlying InputFormat and the new partition is published to the table after the job completes. Also, the maximum number of partitions that a job can work on is limited to 100K.

+

HCatInputFormat exposes a Hadoop 0.20 MapReduce API for reading data as if it had been published to a table.

API

The API exposed by HCatInputFormat is shown below.

-

To use HCatInputFormat to read data, first instantiate a HCatTableInfo with the necessary information from the table being read - and then call setInput on the HCatInputFormat.

+

To use HCatInputFormat to read data, first instantiate as InputJobInfo with the necessary information from the table being read + and then call setInput with the InputJobInfo.

You can use the setOutputSchema method to include a projection schema, to specify specific output fields. If a schema is not specified, this default to the table level schema.

You can use the getTableSchema methods to determine the table schema for a specified input table.

- - /** - * Set the input to use for the Job. This queries the metadata server with - * the specified partition predicates, gets the matching partitions, puts - * the information in the conf object. The inputInfo object is updated with - * information needed in the client context - * @param job the job object - * @param inputInfo the table input info - * @throws IOException the exception in communicating with the metadata server - */ - public static void setInput(Job job, HCatTableInfo inputInfo) throws IOException; + /** + * Set the input to use for the Job. This queries the metadata server with + * the specified partition predicates, gets the matching partitions, puts + * the information in the conf object. The inputInfo object is updated with + * information needed in the client context + * @param job the job object + * @param inputJobInfo the input info for table to read + * @throws IOException the exception in communicating with the metadata server + */ + public static void setInput(Job job, + InputJobInfo inputJobInfo) throws IOException; - /** - * Set the schema for the HCatRecord data returned by HCatInputFormat. - * @param job the job object - * @param hcatSchema the schema to use as the consolidated schema - */ - public static void setOutputSchema(Job job,HCatSchema hcatSchema) throws Exception; + /** + * Set the schema for the HCatRecord data returned by HCatInputFormat. + * @param job the job object + * @param hcatSchema the schema to use as the consolidated schema + */ + public static void setOutputSchema(Job job,HCatSchema hcatSchema) + throws IOException; - /** - * Gets the HCatalog schema for the table specified in the HCatInputFormat.setInput call - * on the specified job context. This information is available only after HCatInputFormat.setInput - * has been called for a JobContext. - * @param context the context - * @return the table schema - * @throws Exception if HCatInputFormat.setInput has not been called for the current context - */ - public static HCatSchema getTableSchema(JobContext context) throws Exception + /** + * Gets the HCatTable schema for the table specified in the HCatInputFormat.setInput call + * on the specified job context. This information is available only after HCatInputFormat.setInput + * has been called for a JobContext. + * @param context the context + * @return the table schema + * @throws IOException if HCatInputFormat.setInput has not been called + * for the current context + */ + public static HCatSchema getTableSchema(JobContext context) + throws IOException; +
@@ -101,52 +105,199 @@

The first call on the HCatOutputFormat must be setOutput; any other call will throw an exception saying the output format is not initialized. The schema for the data being written out is specified by the setSchema method. You must call this method, providing the schema of data you are writing. If your data has same schema as table schema, you can use HCatOutputFormat.getTableSchema() to get the table schema and then pass that along to setSchema().

-/** + /** * Set the info about the output to write for the Job. This queries the metadata server * to find the StorageDriver to use for the table. Throws error if partition is already published. * @param job the job object - * @param outputInfo the table output info + * @param outputJobInfo the table output info * @throws IOException the exception in communicating with the metadata server */ - public static void setOutput(Job job, HCatTableInfo outputInfo) throws IOException; + @SuppressWarnings("unchecked") + public static void setOutput(Job job, OutputJobInfo outputJobInfo) throws IOException; /** * Set the schema for the data being written out to the partition. The * table schema is used by default for the partition if this is not called. * @param job the job object * @param schema the schema for the data - * @throws IOException the exception */ - public static void setSchema(Job job, HCatSchema schema) throws IOException; + public static void setSchema(final Job job, final HCatSchema schema) throws IOException; - /** - * Gets the table schema for the table specified in the HCatOutputFormat.setOutput call - * on the specified job context. - * @param context the context - * @return the table schema - * @throws IOException if HCatOutputFormat.setOutput has not been called for the passed context - */ - public static HCatSchema getTableSchema(JobContext context) throws IOException + /** + * Gets the table schema for the table specified in the HCatOutputFormat.setOutput call + * on the specified job context. + * @param context the context + * @return the table schema + * @throws IOException if HCatOutputFromat.setOutput has not been called for the passed context + */ + public static HCatSchema getTableSchema(JobContext context) throws IOException; +
+ +
- Partition Schema Semantics - -

The partition schema specified can be different from the current table level schema. The rules about what kinds of schema are allowed are:

- - - -
+Examples + + +

Running MapReduce with HCatalog

+ + +export HADOOP_HOME=<path_to_hadoop_install> +export HCAT_JARS=`<path_to_hcat_install>/bin/hcat -classpath` + +

+Your MapReduce program will need to know where the thrift server to connect to is. The easiest way to do this is +pass it as an argument to your Java program. You will need to pass the Hive and HCatalog jars MapReduce as well, +via the -libjars argument.

+ +

For tarball installations:

+$HADOOP_HOME/bin/hadoop --config $HADOOP_HOME/conf -libjars $HCAT_JARS thrift://<thrift_host>:<thrift_port> + + +

For rpm installations:

+ + +/usr/bin/hadoop --config /etc/hadoop -libjars $HCAT_JARS thrift://<thrift_host>:<thrift_port> + + +

Examples

+ +

+The following very simple MapReduce program reads data from one table which it assumes to have an integer in the +second column, and counts how many different values it sees. That is, is does the equivalent of select col1, +count(*) from $table group by col1;. +

+ + +public class GroupByAge extends Configured implements Tool { + + public static class Map extends + Mapper<WritableComparable, HCatRecord, IntWritable, IntWritable> { + + int age; + + @Override + protected void map( + WritableComparable key, + HCatRecord value, + org.apache.hadoop.mapreduce.Mapper<WritableComparable, HCatRecord, IntWritable, IntWritable>.Context context) + throws IOException, InterruptedException { + age = (Integer) value.get(1); + context.write(new IntWritable(age), new IntWritable(1)); + } + } + + public static class Reduce extends Reducer<IntWritable, IntWritable, + WritableComparable, HCatRecord> { + + + @Override + protected void reduce(IntWritable key, java.lang.Iterable<IntWritable> + values, org.apache.hadoop.mapreduce.Reducer<IntWritable,IntWritable,WritableComparable,HCatRecord>.Context context) + throws IOException ,InterruptedException { + int sum = 0; + Iterator<IntWritable> iter = values.iterator(); + while (iter.hasNext()) { + sum++; + iter.next(); + } + HCatRecord record = new DefaultHCatRecord(2); + record.set(0, key.get()); + record.set(1, sum); + + context.write(null, record); + } + } + + public int run(String[] args) throws Exception { + Configuration conf = getConf(); + args = new GenericOptionsParser(conf, args).getRemainingArgs(); + + String inputTableName = args[0]; + String outputTableName = args[1]; + String dbName = null; + + Job job = new Job(conf, "GroupByAge"); + HCatInputFormat.setInput(job, InputJobInfo.create(dbName, + inputTableName, null)); + // initialize HCatOutputFormat + + job.setInputFormatClass(HCatInputFormat.class); + job.setJarByClass(GroupByAge.class); + job.setMapperClass(Map.class); + job.setReducerClass(Reduce.class); + job.setMapOutputKeyClass(IntWritable.class); + job.setMapOutputValueClass(IntWritable.class); + job.setOutputKeyClass(WritableComparable.class); + job.setOutputValueClass(DefaultHCatRecord.class); + HCatOutputFormat.setOutput(job, OutputJobInfo.create(dbName, + outputTableName, null)); + HCatSchema s = HCatOutputFormat.getTableSchema(job); + System.err.println("INFO: output schema explicitly set for writing:" + + s); + HCatOutputFormat.setSchema(job, s); + job.setOutputFormatClass(HCatOutputFormat.class); + return (job.waitForCompletion(true) ? 0 : 1); + } + + public static void main(String[] args) throws Exception { + int exitCode = ToolRunner.run(new GroupByAge(), args); + System.exit(exitCode); + } +} + + +

Notice a number of important points about this program: +



+1) The implementation of Map takes HCatRecord as an input and the implementation of Reduce produces it as an output. +

+2) This example program assumes the schema of the input, but it could also retrieve the schema via +HCatOutputFormat.getOutputSchema() and retrieve fields based on the results of that call. +

+3) The input descriptor for the table to be read is created by calling InputJobInfo.create. It requires the database name, +table name, and partition filter. In this example the partition filter is null, so all partitions of the table +will be read. +

+4) The output descriptor for the table to be written is created by calling OutputJobInfo.create. It requires the +database name, the table name, and a Map of partition keys and values that describe the partition being written. +In this example it is assumed the table is unpartitioned, so this Map is null. +

+ +

To scan just selected partitions of a table, a filter describing the desired partitions can be passed to +InputJobInfo.create. This filter can contain the operators '=', '<', '>', '<=', '>=', '<>', 'and', 'or', and 'like'. For example, you could select one partition of the web_logs table used above in the Pig examples by changing

+ +HCatInputFormat.setInput(job, InputJobInfo.create(dbName, inputTableName, null)); + +

+to +

+ +HCatInputFormat.setInput(job, + InputJobInfo.create(dbName, inputTableName, "datestamp=\"20110924\"")); + +

+This filter must reference only partition columns. Values from other columns will cause the job to fail.

+

+To write to a single partition you can change the above example to have a Map of key value pairs that describe all +of the partition keys and values for that partition. In our example web_logs table, there is only one partition +column (datestamp), so our Map will have only one entry. Change

+ +HCatOutputFormat.setOutput(job, OutputJobInfo.create(dbName, outputTableName, null)); + +

to

+ +Map partitions = new HashMap<String, String>(1); +partitions.put("datestamp", "20110924"); +HCatOutputFormat.setOutput(job, OutputJobInfo.create(dbName, outputTableName, partitions)); + + +

To write multiple partitions simultaneously you can leave the Map null, but all of the partitioning columns must be present in the data you are writing. +

+ + Index: src/docs/src/documentation/content/xdocs/supportedformats.xml =================================================================== --- src/docs/src/documentation/content/xdocs/supportedformats.xml (revision 1300810) +++ src/docs/src/documentation/content/xdocs/supportedformats.xml (working copy) @@ -22,8 +22,10 @@ Storage Formats -

HCatalog can read PigStorage and RCFile formatted files. The input drivers for the formats are PigStorageInputDriver and RCFileInputDriver respectively. HCatalog currently produces only RCFile formatted output. The output driver for the same is RCFileOutputDriver.

+

As of version 0.4, HCatalog uses Hive's SerDe class to serialize and deserialize data. SerDes are provided for RCFile, CSV text, JSON text, and SequenceFile formats.

-

Hive and HCatalog applications can interoperate (each can read the output of the other) as long as they use a common format. Currently, the only common format is RCFile.

+

Users can write SerDes for custom formats using the instructions at https://cwiki.apache.org/confluence/display/Hive/SerDe.

+ + Index: src/docs/src/documentation/content/xdocs/dynpartition.xml =================================================================== --- src/docs/src/documentation/content/xdocs/dynpartition.xml (revision 1300810) +++ src/docs/src/documentation/content/xdocs/dynpartition.xml (working copy) @@ -27,7 +27,7 @@
Overview -

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.

+

When writing data in HCatalog it is possible to write all records to a single partition. In this case the partition column(s) need not be in the output data.

The following Pig script illustrates this:

@@ -40,17 +40,16 @@

-

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.

+

In cases where you want to write data to multiple partitions simultaneously, this can be done by placing partition columns in the data and not specifying partition values when storing the data.

-

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:

- A = load 'raw' using HCatLoader(); ... -store Z into 'processed' using HCatStorer("ds=20110110"); +store Z into 'processed' using HCatStorer(); -

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

+

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 must 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 to 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.

@@ -80,7 +79,7 @@

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.

-

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

+

For example, let's say a=1 for all values across our dataset and b takes the values 1 and 2. Then the following statement...

store A into 'mytable' using HCatStorer(); @@ -115,22 +114,9 @@

And to write to multiple partitions, separate jobs will have to be kicked off with each of the above.

-

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.

+

With dynamic partitioning, 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.

- - -
- Compaction -

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.

-
- - -
- References -

See HCatalog 0.2 Architecture

- -
Index: src/docs/src/documentation/content/xdocs/rpminstall.xml =================================================================== --- src/docs/src/documentation/content/xdocs/rpminstall.xml (revision 1300810) +++ src/docs/src/documentation/content/xdocs/rpminstall.xml (working copy) @@ -24,21 +24,21 @@
- Server Installation + Server Installation from RPM

Prerequisites

Throughout these instructions when you see a word in italics it - indicates a place where you should replace the word with a + indicates a place where you should replace the word with an appropriate value such as a hostname or password.

Thrift Server Install

@@ -66,7 +66,7 @@ machine as the Thrift server. For large clusters we recommend that they not be the same machine. For the purposes of these instructions we will refer to this machine as - hcatdb.acme.com

+ hcatdb.acme.com.

Install MySQL server on hcatdb.acme.com. You can obtain packages for MySQL from MySQL's @@ -85,7 +85,7 @@

mysql> quit;

mysql -u hive -D hivemetastoredb -hhcatdb.acme.com -p < /usr/share/hcatalog/scripts/hive-schema-0.7.0.mysql.sql

-

Thrift server config

+

Thrift Server Configuration

Now you need to edit your /etc/hcatalog/hive-site.xml file. Open this file in your favorite text editor. The following table shows the values you need to configure.

@@ -115,24 +115,24 @@ hive.metastore.uris - You need to set the hostname to your Thrift - server. Replace SVRHOST with the name of the + Set the hostname of your Thrift + server by replacing SVRHOST with the name of the machine you are installing the Thrift server on. hive.metastore.sasl.enabled - Set to false by default. Set to true if its a secure environment. + Set to false by default. Set to true if it is a secure environment. hive.metastore.kerberos.keytab.file - The path to the Kerberos keytab file containg the metastore - thrift server's service principal. Need to set only in secure enviroment. + The path to the Kerberos keytab file containing the metastore + Thrift server's service principal. Need to set only in secure enviroment. hive.metastore.kerberos.principal - The service principal for the metastore thrift server. You can + The service principal for the metastore Thrift server. You can reference your host as _HOST and it will be replaced with - actual hostname. Need to set only in secure environment. + the actual hostname. Need to set only in secure environment. @@ -142,13 +142,13 @@
Starting the Server -

sudo service start hcatalog-server

+

sudo service hcatalog-server start

Logging -

Server activity logs and gc logs are located in +

Server activity logs are located in /var/log/hcat_server. Logging configuration is located at /etc/hcatalog/log4j.properties. Server logging uses DailyRollingFileAppender by default. It will generate a new @@ -158,7 +158,7 @@

Stopping the Server -

sudo service stop hcatalog-server

+

sudo service hcatalog-server stop

@@ -190,18 +190,18 @@ hive.metastore.uris - You need to set the hostname wish your Thrift - server to use by replacing SVRHOST with the name of the + Set the hostname of your Thrift + server by replacing SVRHOST with the name of the machine you are installing the Thrift server on. hive.metastore.sasl.enabled - Set to false by default. Set to true if its a secure environment. + Set to false by default. Set to true if it is a secure environment. hive.metastore.kerberos.principal - The service principal for the metastore thrift server. You can + The service principal for the metastore Thrift server. You can reference your host as _HOST and it will be replaced with actual hostname. Need to set only in secure environment. Index: src/docs/src/documentation/content/xdocs/install.xml =================================================================== --- src/docs/src/documentation/content/xdocs/install.xml (revision 1300810) +++ src/docs/src/documentation/content/xdocs/install.xml (working copy) @@ -24,23 +24,62 @@
- Server Installation + Server Installation from Source

Prerequisites

    +
  • Machine to build the installation tar on
  • Machine on which the server can be installed - this should have - access to the hadoop cluster in question, and be accessible from + access to the Hadoop cluster in question, and be accessible from the machines you launch jobs from
  • -
  • MySQL db
  • +
  • an RDBMS - we recommend MySQL and provide instructions for it
  • Hadoop cluster
  • -
  • Unix user that the server will run as, and an associated kerberos - service principal and keytabs.
  • +
  • Unix user that the server will run as, and, if you are running your + cluster in secure mode, an associated Kerberos service principal and keytabs.

Throughout these instructions when you see a word in italics it indicates a place where you should replace the word with a locally appropriate value such as a hostname or password.

+

Building a tarball

+ +

If you downloaded HCatalog from Apache or another site as a source release, + you will need to first build a tarball to install. You can tell if you have + a source release by looking at the name of the object you downloaded. If + it is named hcatalog-src-0.4.0-incubating.tar.gz (notice the + src in the name) then you have a source release.

+ +

If you do not already have Apache Ant installed on your machine, you + will need to obtain it. You can get it from the + Apache Ant website. Once you download it, you will need to unpack it + somewhere on your machine. The directory wheryou unpack it will be referred + to as ant_home in this document.

+ +

If you do not already have Apache Forrest installed on your machine, you + will need to obtain it. You can get it from the + Apache Forrest website. Once you download it, you will need to unpack + it somewhere on your machine. The directory where you unpack it will be referred + to as forrest_home in this document.

+ +

To produce a tarball from this do the following:

+ +

Create a directory to expand the source release in. Copy the source + release to that directory and unpack it.

+

mkdir /tmp/hcat_source_release

+

cp hcatalog-src-0.4.0-incubating.tar.gz /tmp/hcat_source_release

+

cd /tmp/hcat_source_release

+

tar xzf hcatalog-src-0.4.0-incubating.tar.gz

+ +

Change directories into the unpacked source release and build the + installation tarball.

+

cd hcatalog-src-0.4.0-incubating

+

ant_home/bin/ant -Dhcatalog.version=0.4.0 + -Dforrest.home=forrest_home tar

+ +

The tarball for installation should now be at + build/hcatalog-0.4.0.tar.gz

+

Database Setup

Select a machine to install the database on. This need not be the same @@ -65,12 +104,12 @@

mysql> flush privileges;

mysql> quit;

-

In a temporary directory, untar the HCatalog artifact

+

In a temporary directory, untar the HCatalog installation tarball.

-

tar xzf hcatalog-version.tar.gz

+

tar xzf hcatalog-0.4.0.tar.gz

Use the database installation script found in the package to create the - database

+ database.

mysql -u hive -D hivemetastoredb -hhcatdb.acme.com -p < share/hcatalog/hive/external/metastore/scripts/upgrade/mysql/hive-schema-0.7.0.mysql.sql

Thrift Server Setup

@@ -95,7 +134,7 @@ directory must be owned by the hcat user. We recommend /usr/local/hcat. If necessary, create the directory.

-

Download the HCatalog release into a temporary directory, and untar +

Copy the HCatalog installation tarball into a temporary directory, and untar it. Then change directories into the new distribution and run the HCatalog server installation script. You will need to know the directory you chose as root and the @@ -105,8 +144,8 @@ the port number you wish HCatalog to operate on which you will use to set portnum.

-

tar zxf hcatalog-version.tar.gz - cd hcatalog-version

+

tar zxf hcatalog-0.4.0.tar.gz

+

cd hcatalog-0.4.0

share/hcatalog/scripts/hcat_server_install.sh -r root -d dbroot -h hadoop_home -p portnum

Now you need to edit your root/etc/hcatalog/hive-site.xml file. @@ -126,20 +165,20 @@ javax.jdo.option.ConnectionPassword dbpassword value you used in setting up the MySQL server - above + above. hive.metastore.warehouse.dir The directory can be a URI or an absolute file path. If it is an absolute file path, it will be resolved to a URI by the metastore:

-- If default hdfs was specified in core-site.xml, path resolves to HDFS location.

-- Otherwise, path is resolved as local file: URI.

-

This setting becomes effective when creating new tables (takes precedence over default DBS.DB_LOCATION_URI at time of table creation).

+

This setting becomes effective when creating new tables (it takes precedence over default DBS.DB_LOCATION_URI at the time of table creation).

hive.metastore.uris - You need to set the hostname to your Thrift - server. Replace SVRHOST with the name of the + Set the hostname of your Thrift + server by replacing SVRHOST with the name of the machine you are installing the Thrift server on. You can also change the port the Thrift server runs on by changing the default value of 3306. @@ -147,19 +186,19 @@ hive.metastore.sasl.enabled Set to true by default. Set to false if you do not wish to - secure the thrift interface. This can be convenient for testing. + secure the Thrift interface. This can be convenient for testing. We do not recommend turning this off in production. hive.metastore.kerberos.keytab.file - The path to the Kerberos keytab file containg the metastore - thrift server's service principal. + The path to the Kerberos keytab file containing the metastore + Thrift server's service principal. hive.metastore.kerberos.principal - The service principal for the metastore thrift server. You can + The service principal for the metastore Thrift server. You can reference your host as _HOST and it will be replaced with your - actual hostname + actual hostname. @@ -188,7 +227,7 @@
Logging -

Server activity logs and gc logs are located in +

Server activity logs are located in root/var/log/hcat_server. Logging configuration is located at root/conf/log4j.properties. Server logging uses DailyRollingFileAppender by default. It will generate a new @@ -211,7 +250,7 @@

Select a root directory for your installation of HCatalog client. We recommend /usr/local/hcat. If necessary, create the directory.

-

Download the HCatalog release into a temporary directory, and untar +

Copy the HCatalog installation tarball into a temporary directory, and untar it.

tar zxf hcatalog-version.tar.gz

@@ -233,13 +272,13 @@ The directory can be a URI or an absolute file path. If it is an absolute file path, it will be resolved to a URI by the metastore:

-- If default hdfs was specified in core-site.xml, path resolves to HDFS location.

-- Otherwise, path is resolved as local file: URI.

-

This setting becomes effective when creating new tables (takes precedence over default DBS.DB_LOCATION_URI at time of table creation).

+

This setting becomes effective when creating new tables (it takes precedence over default DBS.DB_LOCATION_URI at the time of table creation).

hive.metastore.uris - You need to set the hostname wish your Thrift - server to use by replacing SVRHOST with the name of the + Set the hostname of your Thrift + server by replacing SVRHOST with the name of the machine you are installing the Thrift server on. You can also change the port the Thrift server runs on by changing the default value of 3306. Index: src/docs/src/documentation/content/xdocs/notification.xml =================================================================== --- src/docs/src/documentation/content/xdocs/notification.xml (revision 1300810) +++ src/docs/src/documentation/content/xdocs/notification.xml (working copy) @@ -23,13 +23,13 @@ -

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:

+

HCatalog 0.2 provides 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:

  • Notification when a new partition is added
  • Notification when a set of partitions is added
-

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.

+

No additional work is required to send a notification when a new partition is added: the existing addPartition call will send the notification message.

Notification for a New Partition @@ -46,7 +46,7 @@

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.

  • -

    The topic name corresponding to a particular table is stored in table properties and can be retrieved using following piece of code:

    +

    The topic name corresponding to a particular table is stored in table properties and can be retrieved using the following piece of code:

    HiveMetaStoreClient msc = new HiveMetaStoreClient(hiveConf); String topicName = msc.getTable("mydb", "myTbl").getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME); @@ -76,14 +76,18 @@ } -

    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

    +

    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 is tested with ActiveMQ as a JMS provider, although any JMS provider can be used. ActiveMQ can be obtained from: http://activemq.apache.org/activemq-550-release.html .

Notification for a Set of Partitions +

Sometimes a user wants to wait until a collection of partitions is finished. For example, you may want to start processing after all partitions for a day are done. However, HCatalog has no notion of collections or hierarchies of partitions. To support this, HCatalog allows data writers to signal when they are finished writing a collection of partitions. Data readers may wait for this signal before beginning to read.

+

The example code below illustrates how to send a notification when a set of partitions has been added.

+

To signal, a data writer does this:

+ HiveMetaStoreClient msc = new HiveMetaStoreClient(conf); @@ -138,8 +142,8 @@ MapMessage mapMsg = (MapMessage)msg; Enumeration<String> 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: + // Enumerate over all keys. This will print key-value pairs specifying the + // particular partition 44which was marked done. In this case, it will print: // date : 20110711 // country: * Index: src/docs/src/documentation/content/xdocs/cli.xml =================================================================== --- src/docs/src/documentation/content/xdocs/cli.xml (revision 1300810) +++ src/docs/src/documentation/content/xdocs/cli.xml (working copy) @@ -32,7 +32,7 @@

Authentication

- +

If a failure results in a message like "2010-11-03 16:17:28,225 WARN hive.metastore ... - Unable to connect metastore with URI thrift://..." in /tmp/<username>/hive.log, then make sure you have run "kinit <username>@FOO.COM" to get a kerberos ticket and to be able to authenticate to the HCatalog server.

If a failure results in a message like "2010-11-03 16:17:28,225 WARN hive.metastore ... - Unable to connect metastore with URI thrift://..." in /tmp/<username>/hive.log, then make sure you have run "kinit <username>@FOO.COM" to get a Kerberos ticket and to be able to authenticate to the HCatalog server.

If other errors occur while using the HCatalog CLI, more detailed messages (if any) are written to /tmp/<username>/hive.log.

@@ -45,9 +45,9 @@
  • -g: Usage is -g mygroup .... This indicates to HCatalog that table that needs to be created must have group as "mygroup"
  • -p: Usage is -p rwxr-xr-x .... This indicates to HCatalog that table that needs to be created must have permissions as "rwxr-xr-x"
  • -
  • -f: Usage is -f myscript.hcatalog .... This indicates to hcatalog that myscript.hcatalog is a file which contains DDL commands it needs to execute.
  • -
  • -e: Usage is -e 'create table mytable(a int);' .... This indicates to HCatalog to treat the following string as DDL command and execute it.
  • -
  • -D: Usage is -Dname=value .... This sets the hadoop value for given property
  • +
  • -f: Usage is -f myscript.hcatalog .... This indicates to HCatalog that myscript.hcatalog is a file which contains DDL commands it needs to execute.
  • +
  • -e: Usage is -e 'create table mytable(a int);' .... This indicates to HCatalog to treat the following string as a DDL command and execute it.
  • +
  • -D: Usage is -Dkey=value .... This sets the Hadoop value for the given property. The value is passed to HCatalog as a Java System Property.

Note the following:

@@ -67,8 +67,6 @@

Assumptions

When using the HCatalog CLI, you cannot specify a permission string without read permissions for owner, such as -wxrwxr-x. If such a permission setting is desired, you can use the octal version instead, which in this case would be 375. Also, any other kind of permission string where the owner has read permissions (for example r-x------ or r--r--r--) will work fine.

- -
@@ -76,117 +74,56 @@
HCatalog DDL -

HCatalog supports a subset of the Hive Data Definition Language. For those commands that are supported, any variances are noted below.

+

HCatalog supports all Hive Data Definition Language except those operations that require running a MapReduce job. For commands that are supported, any variances are noted below.

+

HCatalog does not support the following Hive DDL commands:

+
    +
  • IMPORT FROM ...
  • +
  • EXPORT TABLE
  • +
  • CREATE TABLE ... AS SELECT
  • +
  • ALTER TABLE ... REBUILD
  • +
  • ALTER TABLE ... CONCATENATE
  • +
  • ANALYZE TABLE ... COMPUTE STATISTICS
  • +
Create/Drop/Alter Table -

CREATE TABLE

-

The STORED AS clause in Hive is:

- -[STORED AS file_format] -file_format: - : SEQUENCEFILE - | TEXTFILE - | RCFILE - | INPUTFORMAT input_format_classname OUTPUTFORMAT output_format_classname - -

The STORED AS clause in HCatalog is:

- -[STORED AS file_format] -file_format: - : RCFILE - | INPUTFORMAT input_format_classname OUTPUTFORMAT output_format_classname - INPUTDRIVER input_driver_classname OUTPUTDRIVER output_driver_classname - +

CREATE TABLE

-

Note the following:

-
    -
  • CREATE TABLE command must contain a "STORED AS" clause; if it doesn't it will result in an exception containing message "STORED AS specification is either incomplete or incorrect."



    - - - - -

    In this release, HCatalog supports only reading PigStorage formated text files and only writing RCFile formatted files. Therefore, for this release, the command must contain a "STORED AS" clause and either use RCFILE as the file format or specify org.apache.hadoop.hive.ql.io.RCFileInputFormat and org.apache.hadoop.hive.ql.io.RCFileOutputFormat as INPUTFORMAT and OUTPUTFORMAT respectively.

    -

    -
  • -
  • For partitioned tables, partition columns can only be of type String. -
  • -
  • CLUSTERED BY clause is not supported. If provided error message will contain "Operation not supported. HCatalog doesn't allow Clustered By in create table." -
  • -
+

If you create a table with a CLUSTERED BY clause you will not be able to write to it with Pig or MapReduce. This is because they do not understand how to partition the table, so attempting to write to it would cause data corruption.

+

CREATE TABLE AS SELECT

-

Not supported. Throws an exception with message "Operation Not Supported".

-

CREATE TABLE LIKE

-

Not supported. Throws an exception with message "Operation Not Supported".

+

Not supported. Throws an exception with the message "Operation Not Supported".

+

DROP TABLE

Supported. Behavior the same as Hive.

ALTER TABLE

- -ALTER TABLE table_name ADD partition_spec [ LOCATION 'location1' ] partition_spec [ LOCATION 'location2' ] ... - partition_spec: - : PARTITION (partition_col = partition_col_value, partition_col = partiton_col_value, ...) - -

Note the following:

-
    -
  • Allowed only if TABLE table_name was created using HCatalog. Else, throws an exception containing error message "Operation not supported. Partitions can be added only in a table created through HCatalog. It seems table tablename was not created through HCatalog" -
  • -
-

- -

ALTER TABLE FILE FORMAT

- -ALTER TABLE table_name SET FILEFORMAT file_format - -

Note the following:

-
    -
  • Here file_format must be same as the one described above in CREATE TABLE. Else, throw an exception "Operation not supported. Not a valid file format."
  • -
  • CLUSTERED BY clause is not supported. If provided will result in an exception "Operation not supported."
  • -
- -

ALTER TABLE Change Column Name/Type/Position/Comment

- -ALTER TABLE table_name CHANGE [COLUMN] col_old_name col_new_name column_type [COMMENT col_comment] [FIRST|AFTER column_name] - -

Not supported. Throws an exception with message "Operation Not Supported".

+

Supported except for the REBUILD and CONCATENATE options. Behavior the same as Hive.

- -

ALTER TABLE Add/Replace Columns

- -ALTER TABLE table_name ADD|REPLACE COLUMNS (col_name data_type [COMMENT col_comment], ...) - -

Note the following:

-
    -
  • ADD Columns is allowed. Behavior same as of Hive.
  • -
  • Replace column is not supported. Throws an exception with message "Operation Not Supported".
  • -
- - -

ALTER TABLE TOUCH

- -ALTER TABLE table_name TOUCH; -ALTER TABLE table_name TOUCH PARTITION partition_spec; - -

Not supported. Throws an exception with message "Operation Not Supported".

+

+
Create/Drop/Alter View +

Note: Pig and MapReduce coannot read from or write to views.

+

CREATE VIEW

-

Not supported. Throws an exception with message "Operation Not Supported".

+

Supported. Behavior same as Hive.

DROP VIEW

-

Not supported. Throws an exception with message "Operation Not Supported".

+

Supported. Behavior same as Hive.

ALTER VIEW

-

Not supported. Throws an exception with message "Operation Not Supported".

+

Supported. Behavior same as Hive.

+
@@ -204,12 +141,29 @@

DESCRIBE

Supported. Behavior same as Hive.

+
+ + +
+ Create/Drop Index + +

CREATE and DROP INDEX operations are supported.

+

Note: Pig and MapReduce cannot write to a table that has auto rebuild on, because Pig and MapReduce do not know how to rebuild the index.

+
+ + +
+ Create/Drop Function + +

CREATE and DROP FUNCTION operations are supported, but created functions must still be registered in Pig and placed in CLASSPATH for MapReduce.

+ +
Other Commands -

Any command not listed above is NOT supported and throws an exception with message "Operation Not Supported".

+

Any command not listed above is NOT supported and throws an exception with the message "Operation Not Supported".

Index: src/docs/src/documentation/content/xdocs/index.xml =================================================================== --- src/docs/src/documentation/content/xdocs/index.xml (revision 1300810) +++ src/docs/src/documentation/content/xdocs/index.xml (working copy) @@ -25,8 +25,9 @@
HCatalog -

HCatalog is a table management and storage management layer for Hadoop that enables users with different data processing tools – Pig, MapReduce, Hive, Streaming – to more easily read and write data on the grid. HCatalog’s table abstraction presents users with a relational view of data in the Hadoop distributed file system (HDFS) and ensures that users need not worry about where or in what format their data is stored – RCFile format, text files, sequence files.

-

(Note: In this release, Streaming is not supported. Also, HCatalog supports only writing RCFile formatted files and only reading PigStorage formated text files.)

+

HCatalog is a table management and storage management layer for Hadoop that enables users with different data processing tools – Pig, MapReduce, Hive, Streaming – to more easily read and write data on the grid. HCatalog’s table abstraction presents users with a relational view of data in the Hadoop distributed file system (HDFS) and ensures that users need not worry about where or in what format their data is stored – RCFile format, text files, or sequence files.

+

(Note: In this release, Streaming is not supported.)

+

HCatalog supports reading and writing files in any format for which a SerDe can be written. By default, HCatalog supports RCFile, CSV, JSON, and sequence file formats. To use a custom format, you must provide the InputFormat, OutputFormat, and SerDe.

@@ -37,7 +38,6 @@
HCatalog Architecture

HCatalog is built on top of the Hive metastore and incorporates components from the Hive DDL. HCatalog provides read and write interfaces for Pig and MapReduce and a command line interface for data definitions.

-

(Note: HCatalog notification is not available in this release.)

@@ -45,37 +45,36 @@
Interfaces -

The HCatalog interface for Pig – HCatLoader and HCatStorer – is an implementation of the Pig load and store interfaces. HCatLoader accepts a table to read data from; you can indicate which partitions to scan by immediately following the load statement with a partition filter statement. HCatStorer accepts a table to write to and a specification of partition keys to create a new partition. Currently HCatStorer only supports writing to one partition. HCatLoader and HCatStorer are implemented on top of HCatInputFormat and HCatOutputFormat respectively (see HCatalog Load and Store).

+

The HCatalog interface for Pig – HCatLoader and HCatStorer – is an implementation of the Pig load and store interfaces. HCatLoader accepts a table to read data from; you can indicate which partitions to scan by immediately following the load statement with a partition filter statement. HCatStorer accepts a table to write to and optionally a specification of partition keys to create a new partition. You can write to a single partition by specifying the partition key(s) and value(s) in the STORE clause; and you can write to multiple partitions if the partition key(s) are columns in the data being stored. HCatLoader and HCatStorer are implemented on top of HCatInputFormat and HCatOutputFormat, respectively (see HCatalog Load and Store).

-

The HCatalog interface for MapReduce – HCatInputFormat and HCatOutputFormat – is an implementation of Hadoop InputFormat and OutputFormat. HCatInputFormat accepts a table to read data from and a selection predicate to indicate which partitions to scan. HCatOutputFormat accepts a table to write to and a specification of partition keys to create a new partition. Currently HCatOutputFormat only supports writing to one partition (see HCatalog Input and Output).

+

The HCatalog interface for MapReduce – HCatInputFormat and HCatOutputFormat – is an implementation of Hadoop InputFormat and OutputFormat. HCatInputFormat accepts a table to read data from and optionally a selection predicate to indicate which partitions to scan. HCatOutputFormat accepts a table to write to and optionally a specification of partition keys to create a new partition. You can write to a single partition by specifying the partition key(s) and value(s) in the STORE clause; and you can write to multiple partitions if the partition key(s) are columns in the data being stored. (See HCatalog Input and Output.)

-

Note: Currently there is no Hive-specific interface. Since HCatalog uses Hive's metastore, Hive can read data in HCatalog directly as long as a SerDe for that data already exists. In the future we plan to write a HCatalogSerDe so that users won't need storage-specific SerDes and so that Hive users can write data to HCatalog. Currently, this is supported - if a Hive user writes data in the RCFile format, it is possible to read the data through HCatalog. Also, see Supported data formats.

+

Note: There is no Hive-specific interface. Since HCatalog uses Hive's metastore, Hive can read data in HCatalog directly.

-

Data is defined using HCatalog's command line interface (CLI). The HCatalog CLI supports most of the DDL portion of Hive's query language, allowing users to create, alter, drop tables, etc. The CLI also supports the data exploration part of the Hive command line, such as SHOW TABLES, DESCRIBE TABLE, etc. (see the HCatalog Command Line Interface).

+

Data is defined using HCatalog's command line interface (CLI). The HCatalog CLI supports all Hive DDL that does not require MapReduce to execute, allowing users to create, alter, drop tables, etc. (Unsupported Hive DDL includes import/export, CREATE TABLE AS SELECT, ALTER TABLE options REBUILD and CONCATENATE, and ANALYZE TABLE ... COMPUTE STATISTICS.) The CLI also supports the data exploration part of the Hive command line, such as SHOW TABLES, DESCRIBE TABLE, etc. (see the HCatalog Command Line Interface).

Data Model -

HCatalog presents a relational view of data in HDFS. Data is stored in tables and these tables can be placed in databases. Tables can also be hash partitioned on one or more keys; that is, for a given value of a key (or set of keys) there will be one partition that contains all rows with that value (or set of values). For example, if a table is partitioned on date and there are three days of data in the table, there will be three partitions in the table. New partitions can be added to a table, and partitions can be dropped from a table. Partitioned tables have no partitions at create time. Unpartitioned tables effectively have one default partition that must be created at table creation time. There is no guaranteed read consistency when a partition is dropped.

+

HCatalog presents a relational view of data. Data is stored in tables and these tables can be placed in databases. Tables can also be hash partitioned on one or more keys; that is, for a given value of a key (or set of keys) there will be one partition that contains all rows with that value (or set of values). For example, if a table is partitioned on date and there are three days of data in the table, there will be three partitions in the table. New partitions can be added to a table, and partitions can be dropped from a table. Partitioned tables have no partitions at create time. Unpartitioned tables effectively have one default partition that must be created at table creation time. There is no guaranteed read consistency when a partition is dropped.

-

Partitions contain records. Once a partition is created records cannot be added to it, removed from it, or updated in it. (In the future some ability to integrate changes to a partition will be added.) Partitions are multi-dimensional and not hierarchical. Records are divided into columns. Columns have a name and a datatype. HCatalog supports the same datatypes as Hive (see HCatalog Load and Store).

+

Partitions contain records. Once a partition is created records cannot be added to it, removed from it, or updated in it. Partitions are multi-dimensional and not hierarchical. Records are divided into columns. Columns have a name and a datatype. HCatalog supports the same datatypes as Hive (see HCatalog Load and Store).

Data Flow Example -

This simple data flow example shows how HCatalog is used to move data from the grid into a database. - From the database, the data can then be analyzed using Hive.

+

This simple data flow example shows how HCatalog can help grid users share and access data.

First Joe in data acquisition uses distcp to get data onto the grid.

hadoop distcp file:///file.dat hdfs://data/rawevents/20100819/data -hcat "alter table rawevents add partition 20100819 hdfs://data/rawevents/20100819/data" +hcat "alter table rawevents add partition (ds='20100819') location 'hdfs://data/rawevents/20100819/data'"

Second Sally in data processing uses Pig to cleanse and prepare the data.

-

Without HCatalog, Sally must be manually informed by Joe that data is available, or use Oozie and poll on HDFS.

+

Without HCatalog, Sally must be manually informed by Joe when data is available, or poll on HDFS.

A = load '/data/rawevents/20100819/data' as (alpha:int, beta:chararray, …); B = filter A by bot_finder(zeta) = 0; @@ -83,7 +82,7 @@ store Z into 'data/processedevents/20100819/data'; -

With HCatalog, Oozie will be notified by HCatalog data is available and can then start the Pig job

+

With HCatalog, HCatalog will send a JMS message that data is available. The Pig job can then be started.

A = load 'rawevents' using HCatLoader; B = filter A by date = '20100819' and by bot_finder(zeta) = 0; @@ -99,14 +98,14 @@ select advertiser_id, count(clicks) from processedevents where date = '20100819' -group by adverstiser_id; +group by advertiser_id;

With HCatalog, Robert does not need to modify the table structure.

select advertiser_id, count(clicks) from processedevents where date = ‘20100819’ -group by adverstiser_id; +group by advertiser_id;
Index: src/docs/src/documentation/content/xdocs/importexport.xml =================================================================== --- src/docs/src/documentation/content/xdocs/importexport.xml (revision 1300810) +++ src/docs/src/documentation/content/xdocs/importexport.xml (working copy) @@ -1,555 +0,0 @@ - - - - - -
- Import and Export Commands -
- - - -
- Overview -

The HCatalog IMPORT and EXPORT commands enable you to:

-
    -
  • Extract the data and the metadata associated with a table in HCatalog as a stand-alone package so that these can be transferred across HCatalog instances.
  • -
  • Create the data and metadata associated with a table in a setup where there is no HCatalog metastore.
  • -
  • Import the data and the metadata into an existing HCatalog instance.
  • -
  • Use the exported package as input to both pig and mapreduce jobs.
  • -
-

-

The output location of the exported dataset is a directory that has the following structure:

-
    -
  • A _metadata file that contains the metadata of the table, and if the table is partitioned, for all the exported partitions.
  • -
  • A subdirectory hierarchy for each exported partition (or just one "data" subdirectory, in case of a non-partitioned table) that contains the data files of the table/partitions.
  • -
-

-

Note that this directory structure can be created using the EXPORT as well as HCatEximOuptutFormat for MapReduce or HCatPigStorer for Pig. And the data can be consumed using the IMPORT command as well as HCatEximInputFormat for MapReduce or HCatPigLoader for Pig.

-
- - -
- Export Command -

Exports a table to a specified location.

- -
- Syntax - - - - -
-

EXPORT TABLE tablename [PARTITION (partcol1=val1, partcol2=val2, ...)] TO 'filepath'

-
-
- -
- Terms - - - - - - - - - - - - - -
-

TABLE tablename

-
-

The table to be exported. The table can be a simple table or a partitioned table.

-

If the table is partitioned, you can specify a specific partition of the table by specifying values for all of the partitioning columns or specifying a subset of the partitions of the table by specifying a subset of the partition column/value specifications. In this case, the conditions are implicitly ANDed to filter the partitions to be exported.

-
-

PARTITION (partcol=val ...)

-
-

The partition column/value specifications.

-
-

TO 'filepath'

-
-

The filepath (in single quotes) designating the location for the exported table. The file path can be:

-
    -
  • a relative path ('project/data1')
  • -
  • an absolute path ('/user/hcat/project/data1')
  • -
  • a full URI with scheme and, optionally, an authority ('hdfs://namenode:9000/user/hcat/project/data1')
  • -
-
-
- -
- Usage -

The EXPORT command exports a table's data and metadata to the specified location. Because the command actually copies the files defined for the table/partions, you should be aware of the following:

-
    -
  • No record level filtering, ordering, etc. is done as part of the export.
  • -
  • Since HCatalog only does file-level copies, the data is not transformed in anyway while performing the export/import.
  • -
  • You, the user, are responsible for ensuring that the correct binaries are available in the target environment (compatible serde classes, hcat storage drivers, etc.).
  • -
-

Also, note the following:

-
    -
  • The data and the metadata for the table to be exported should exist.
  • -
  • The target location must not exist or must be an empty directory.
  • -
  • You must have access as per the hcat access control mechanisms.
  • -
  • You should have write access to the target location.
  • -
  • Currently only hdfs is supported in production mode for the target filesystem. pfile can also be used for testing purposes.
  • -
-
- -
- Examples -

The examples assume the following tables:

-
    -
  • dept - non partitioned
  • -
  • empl - partitioned on emp_country, emp_state, has four partitions ("us"/"ka", "us"/"tn", "in"/"ka", "in"/"tn")
  • -
-

-

Example 1

- -EXPORT TABLE dept TO 'exports/dept'; - -

This example exports the entire table to the target location. The table and the exported copy are now independent; any further changes to the table (data or metadata) do not impact the exported copy. The exported copy can be manipulated/deleted w/o any effect on the table.

-
    -
  • output directoryg: exports/dept
  • -
  • _metadata - the metadata file
  • -
  • data - a directory which now contains all the data files
  • -
- -

-

Example 2

- -EXPORT TABLE empl TO 'exports/empl'; - -

This example exports the entire table including all the partitions' data/metadata to the target location.

-
    -
  • output directory: exports/empl
  • -
  • _metadata - the metadata file with info on the table as well as the four partitions below
  • -
  • emp_country=in/emp_state=ka - a directory which now contains all the data files for in/ka partition
  • -
  • emp_country=in/emp_state=tn - a directory which now contains all the data files for in/tn partition
  • -
  • emp_country=us/emp_state=ka - a directory which now contains all the data files for us/ka partition
  • -
  • emp_country=us/emp_state=tn - a directory which now contains all the data files for us/tn partition
  • -
- -

-

Example 3

- -EXPORT TABLE empl PARTITION (emp_country='in') TO 'exports/empl-in'; - -

This example exports a subset of the partitions - those which have country = in - to the target location.

-
    -
  • output directory: exports/empl
  • -
  • _metadata - the metadata file with info on the table as well as the two partitions below
  • -
  • emp_country=in/emp_state=ka - a directory which now contains all the data files for in/ka partition
  • -
  • emp_country=in/emp_state=tn - a directory which now contains all the data files for in/tn partition
  • -
- -

-

Example 4

- -EXPORT TABLE empl PARTITION (emp_country='in', emp_state='tn') TO 'exports/empl-in'; - -

This example exports a single partition - that which has country = in, state = tn - to the target location.

-
    -
  • output directory: exports/empl
  • -
  • _metadata - the metadata file with info on the table as well as the partitions below
  • -
  • emp_country=in/emp_state=tn - a directory which now contains all the data files for in/tn partition
  • -
-
- -
- - -
- Import Command -

Imports a table from a specified location.

- -
- Syntax - - - - -
-

IMPORT [[EXTERNAL] TABLE tablename [PARTITION (partcol1=val1, partcol2=val2, ...)]] FROM 'filepath' [LOCATION 'tablepath']

-
-
- -
- Terms - - - - - - - - - - - - - - - - - - - - - -
-

EXTERNAL

-
-

Indicates that the imported table is an external table.

-
-

TABLE tablename

-
-

The target to be imported, either a table or a partition.

-

If the table is partitioned, you can specify a specific partition of the table by specifying values for all of the partitioning columns, or specify all the (exported) partitions by not specifying any of the partition parameters in the command.

-
-

PARTITION (partcol=val ...)

-
-

The partition column/value specifications.

-
-

FROM 'filepath'

-
-

The filepath (in single quotes) designating the source location the table will be copied from. The file path can be:

-
    -
  • a relative path ('project/data1')
  • -
  • an absolute path ('/user/hcat/project/data1')
  • -
  • a full URI with scheme and, optionally, an authority ('hdfs://namenode:9000/user/hcat/project/data1')
  • -
-
-

LOCATION 'tablepath'

-
-

(optional) The tablepath (in single quotes) designating the target location the table will be copied to.

-

If not specified, then:

-
    -
  • For managed tables, the default location of the table within the warehouse/database directory structure is used.
  • -
  • For external tables, the data is imported in-place; that is, no copying takes place.
  • -
-
-
- -
- Usage -

The IMPORT command imports a table's data and metadata from the specified location. The table can be a managed table (data and metadata are both removed on drop table/partition) or an external table (only metadata is removed on drop table/partition). For more information, see Hive's Create/Drop Table.

- -

Because the command actually copies the files defined for the table/partions, you should be aware of the following:

-
    -
  • No record level filtering is performed, ordering, etc. is done as part of the import.
  • -
  • Since HCatalog only does file-level copies, the data is not transformed in anyway while performing the export/import.
  • -
  • You, the user, are responsible for ensuring that the correct binaries are available in the target environment (compatible serde classes, hcat storage drivers, etc.).
  • -
-

Also, note the following:

-
    -
  • The filepath should contain the files as created by the export command, or by HCatEximOutputFormat, or by pig HCatEximStorer.
  • -
  • Currently only hdfs is supported in production mode for the filesystem. pfile can be used for testing purposes.
  • -
  • The target table may or may not exist prior to the import. If it does exist, it should be compatible with the imported table/command. -
      -
    • The column schema and the partitioning schema should match. If partitioned, there should not be any existing partitions with the same specs as the imported partitions.
    • -
    • The target table/partition should be empty.
    • -
    • External/Location checks: -
        -
      • The original table type is ignored on import. You specify the required table type as part of the command.
      • -
      • For non-partitioned tables, the new table location as specified by the command should match the existing table location.
      • -
      • For partitioned tables, the table type (external/managed) should match.
      • -
      • For non-partitioned tables imported as external table, you will be asked to the drop the existing table first.
      • -
      -
    • -
    • The HCatalog storage driver specification should match.
    • -
    • The serde, sort and bucket specs should match.
    • -
    -
  • -
  • You must have access rights as per the hcat access control mechanisms.
  • -
  • You should have read access to the source location.
  • -
-
- -
- Examples -

The examples assume the following tables:

-
    -
  • dept - non partitioned
  • -
  • empl - partitioned on emp_country, emp_state, has four partitions ("us"/"ka", "us"/"tn", "in"/"ka", "in"/"tn")
  • -
-

-

Example 1

- -IMPORT FROM 'exports/dept'; - -

This example imports the table as a managed target table, default location. The metadata is stored in the metastore and the table's data files in the warehouse location of the current database.

-

-

Example 2

- -IMPORT TABLE renamed_name FROM 'exports/dept'; - -

This example imports the table as a managed target table, default location. The imported table is given a new name.

- -

-

Example 3

- -IMPORT EXTERNAL TABLE name FROM 'exports/dept'; - -

This example imports the table as an external target table, imported in-place. The metadata is copied to the metastore.

- -

-

Example 4

- -IMPORT EXTERNAL TABLE name FROM 'exports/dept' LOCATION 'tablestore/dept'; - -

This example imports the table as an external target table, imported to another location. The metadata is copied to the metastore.

- -

-

Example 5

- -IMPORT TABLE name FROM 'exports/dept' LOCATION 'tablestore/dept'; - -

This example imports the table as a managed target table, non-default location. The metadata is copied to the metastore.

- -

-

Example 6

- -IMPORT TABLE empl FROM 'exports/empl'; - -

This example imports all the exported partitions since the source was a partitioned table.

- -

-

Example 7

- -IMPORT TABLE empl PARTITION (emp_country='in', emp_state='tn') FROM 'exports/empl'; - -

This example imports only the specified partition.

-
- -
- - -
- Usage with MapReduce -

HCatEximOutputFormat and HCatEximInputFormat can be used in Hadoop environments where there is no HCatalog instance available. HCatEximOutputFormat can be used to create an 'exported table' dataset, which later can be imported into a HCatalog instance. It can also be later read via HCatEximInputFormat or HCatEximLoader.

- -
-HCatEximOutputFormat - - public static void setOutput(Job job, String dbname, String tablename, String location, - HCatSchema partitionSchema, List<String> partitionValues, HCatSchema columnSchema) throws HCatException; - - public static void setOutput(Job job, String dbname, String tablename, String location, - HCatSchema partitionSchema, - List<String> partitionValues, - HCatSchema columnSchema, - String isdname, String osdname, - String ifname, String ofname, - String serializationLib) throws HCatException; - -

The user can specify the parameters of the table to be created by means of the setOutput method. The metadata and the data files are created in the specified location.

-

The target location must be empty and the user must have write access.

-
- -
-HCatEximInputFormat - - public static List<HCatSchema> setInput(Job job, - String location, - Map<String, String> partitionFilter) throws IOException; - - public static void setOutputSchema(Job job, HCatSchema hcatSchema) throws IOException; - -

The user specifies the data collection location and optionally a filter for the partitions to be loaded via the setInput method. Optionally, the user can also specify the projection columns via the setOutputSchema method.

-

The source location should have the correct layout as for a exported table, and the user should have read access.

-
- -
- - -
- Usage with Pig -

HCatEximStorer and HCatEximLoader can be used in hadoop/pig environments where there is no HCatalog instance available. HCatEximStorer can be used to create an 'exported table' dataset, which later can be imported into a HCatalog instance. It can also be later read via HCatEximInputFormat or HCatEximLoader.

- -
-HCatEximStorer - - public HCatEximStorer(String outputLocation) - throws FrontendException, ParseException; - public HCatEximStorer(String outputLocation, String partitionSpec) - throws FrontendException, ParseException; - public HCatEximStorer(String outputLocation, String partitionSpec, String schema) - throws FrontendException, ParseException; - - -

The HCatEximStorer is initialized with the output location for the exported table. Optionally the user can specify the partition specification for the data, plus rename the schema elements as part of the storer.

-

The rest of the storer semantics use the same design as HCatStorer.

- -

Example

- -A = LOAD 'empdata' USING PigStorage(',') - AS (emp_id:int,emp_name:chararray,emp_dob:chararray,emp_sex:chararray,emp_country:chararray,emp_state:chararray); -INTN = FILTER A BY emp_country == 'IN' AND emp_state == 'TN'; -INKA = FILTER A BY emp_country == 'IN' AND emp_state == 'KA'; -USTN = FILTER A BY emp_country == 'US' AND emp_state == 'TN'; -USKA = FILTER A BY emp_country == 'US' AND emp_state == 'KA'; -STORE INTN INTO 'default.employee' USING org.apache.HCatalog.pig.HCatEximStorer('exim/pigout/employee', 'emp_country=in,emp_state=tn'); -STORE INKA INTO 'default.employee' USING org.apache.HCatalog.pig.HCatEximStorer('exim/pigout/employee', 'emp_country=in,emp_state=ka'); -STORE USTN INTO 'default.employee' USING org.apache.HCatalog.pig.HCatEximStorer('exim/pigout/employee', 'emp_country=us,emp_state=tn'); -STORE USKA INTO 'default.employee' USING org.apache.HCatalog.pig.HCatEximStorer('exim/pigout/employee', 'emp_country=us,emp_state=ka'); - -
- - -
-HCatEximLoader - -public HCatEximLoader(); - -

The HCatEximLoader is passed the location of the exported table as usual by the LOAD statement. The loader loads the metadata and data as required from the location. Note that partition filtering is not done efficiently when eximloader is used; the filtering is done at the record level rather than at the file level.

-

The rest of the loader semantics use the same design as HCatLoader.

-

Example

- -A = LOAD 'exim/pigout/employee' USING org.apache.HCatalog.pig.HCatEximLoader(); -dump A; - -
- -
- - -
-Use Cases -

Use Case 1

-

Transfer data between different HCatalog/hadoop instances, with no renaming of tables.

-
    -
  • Instance A - HCatalog: export table A into 'locationA';
  • -
  • Hadoop: distcp hdfs://locationA hdfs://locationB
  • -
  • Instance B - HCatalog: import from 'locationB';
  • -
- -

-

Use Case 2

-

Transfer data to a hadoop instance which does not have HCatalog and process it there.

-
    -
  • Instance A - HCatalog: export table A into 'locationA';
  • -
  • Hadoop: distcp hdfs://locationA hdfs://locationB
  • -
  • Instance B - Map/Reduce job example -
  • -
- - //job setup - ... - HCatEximInputFormat.setInput(job, "hdfs://locationB", partitionSpec); - job.setInputFormatClass(HCatEximInputFormat.class); - ... - - //map setup - protected void setup(Context context) throws IOException, InterruptedException { - super.setup(context); - ... - recordSchema = HCatBaseInputFormat.getTableSchema(context); - ... - } - - //map task - public void map(LongWritable key, HCatRecord value, Context context) throws IOException, - InterruptedException { - ... - String colValue = value.getString("emp_name", recordSchema); - ... - } - - -
    -
  • Instance B - Pig example -
  • -
- - ... - A = LOAD '/user/krishnak/pig-exports/employee-nonpartn' USING org.apache.HCatalog.pig.HCatEximLoader(); - ... - - - -

-

Use Case 3

-

Create an exported dataset in a hadoop instance which does not have HCatalog and then import into HCatalog in a different instance.

-
    -
  • Instance A - Map/Reduce job example
  • -
- - //job setup - ... - List<HCatFieldSchema> columns = new ArrayList<HCatFieldSchema>(); - columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_id", - Constants.INT_TYPE_NAME, ""))); - ... - List<HCatFieldSchema> partKeys = new ArrayList<HCatFieldSchema>(); - partKeys.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_country", - Constants.STRING_TYPE_NAME, ""))); - partKeys.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_state", - Constants.STRING_TYPE_NAME, ""))); - HCatSchema partitionSchema = new HCatSchema(partKeys); - List<String> partitionVals = new ArrayList<String>(); - partitionVals.add(...); - partitionVals.add(...); - ... - HCatEximOutputFormat.setOutput(job, "default", "employee", "hdfs:/user/krishnak/exim/employee", - partitionSchema, partitionVals, new HCatSchema(columns)); - job.setOutputFormatClass(HCatEximOutputFormat.class); - ... - - //map setup - protected void setup(Context context) throws IOException, InterruptedException { - super.setup(context); - ... - recordSchema = HCatEximOutputFormat.getTableSchema(context); - ... - } - - //map task - public void map(LongWritable key, HCatRecord value, Context context) throws IOException, - InterruptedException { - ... - HCatRecord record = new DefaultHCatRecord(recordSchema.size()); - record.setInteger("emp_id", recordSchema, Integer.valueOf(cols[0])); - record.setString("emp_name", recordSchema, cols[1]); - ... - context.write(key, record); - ... - } - - - -
    -
  • Instance A - Pig example
  • -
- - ... -STORE INTN INTO 'default.employee' - USING org.apache.HCatalog.pig.HCatEximStorer('/user/krishnak/pig-exports/employee', 'emp_country=IN,emp_state=TN'); - ... - - -
    -
  • Hadoop: distcp hdfs://locationA hdfs://locationB
  • -
  • Instance B - HCatalog: import from 'locationB';
  • -
-
- - -
Index: src/docs/src/documentation/content/xdocs/loadstore.xml =================================================================== --- src/docs/src/documentation/content/xdocs/loadstore.xml (revision 1300810) +++ src/docs/src/documentation/content/xdocs/loadstore.xml (working copy) @@ -27,27 +27,28 @@
Set Up -

The HCatLoader and HCatStorer interfaces are used with Pig scripts to read and write data in HCatalog managed tables. If you run your Pig script using the "pig" command (the bin/pig Perl script) no set up is required.

+

The HCatLoader and HCatStorer interfaces are used with Pig scripts to read and write data in HCatalog managed tables. If you run your Pig script using the "pig" command (the bin/pig script) no set up is required.

$ pig mypig.script

If you run your Pig script using the "java" command (java -cp pig.jar...), then the hcat jar needs to be included in the classpath of the java command line (using the -cp option). Additionally, the following properties are required in the command line:

    -
  • -Dhive.metastore.uris=thrift://<hcatalog server hostname>:9080
  • -
  • -Dhive.metastore.kerberos.principal=<hcatalog server kerberos principal>
  • +
  • -Dhcat.metastore.uri=thrift://<hcatalog server hostname>:<portnum>
  • +
  • -Dhcat.metastore.principal=<hcatalog server kerberos principal> +

    (In the unsecure case, this value will be blank.)
$ java -cp pig.jar hcatalog.jar - -Dhive.metastore.uris=thrift://<hcatalog server hostname>:9080 - -Dhive.metastore.kerberos.principal=<hcatalog server kerberos principal> myscript.pig + -Dhcat.metastore.uri=thrift://<hcatalog server hostname>:9080 + -Dhcat.metastore.principal=<hcatalog server kerberos principal> myscript.pig

Authentication

- +

If a failure results in a message like "2010-11-03 16:17:28,225 WARN hive.metastore ... - Unable to connect metastore with URI thrift://..." in /tmp/<username>/hive.log, then make sure you have run "kinit <username>@FOO.COM" to get a kerberos ticket and to be able to authenticate to the HCatalog server.

If you are using a secure cluster and a failure results in a message like "2010-11-03 16:17:28,225 WARN hive.metastore ... - Unable to connect metastore with URI thrift://..." in /tmp/<username>/hive.log, then make sure you have run "kinit <username>@FOO.COM" to get a Kerberos ticket and to be able to authenticate to the HCatalog server.

@@ -62,14 +63,17 @@ Usage

HCatLoader is accessed via a Pig load statement.

-A = LOAD 'dbname.tablename' USING org.apache.hcatalog.pig.HCatLoader(); +A = LOAD 'tablename' USING org.apache.hcatalog.pig.HCatLoader();

Assumptions

-

You must specify the database name and table name using this format: 'dbname.tablename'. Both the database and table must be created prior to running your Pig script. The Hive metastore lets you create tables without specifying a database; if you created tables this way, then the database name is 'default' and the string becomes 'default.tablename'.

+

You must specify the table name in single quotes: LOAD 'tablename'. If you are using a non-default database you must specify your input as 'dbname.tablename'. If you are using Pig 0.9.2 or earlier, you must create your database and table prior to running the Pig script. Beginning with Pig 0.10 you can issue these create commands in Pig using the SQL command.

+

The Hive metastore lets you create tables without specifying a database; if you created tables this way, then the database name is 'default'.

If the table is partitioned, you can indicate which partitions to scan by immediately following the load statement with a partition filter statement - (see Examples).

-
+ (see Examples).

+
+ +
HCatalog Data Types

Restrictions apply to the types of columns HCatLoader can read.

@@ -90,12 +94,12 @@

primitives (int, long, float, double, string)

-

int, long, float, double

string to chararray

+

int, long, float, double, string to chararray

-

map (key type should be string, valuetype can be a primitive listed above)

+

map (key type should be string, valuetype must be string)

map

@@ -103,72 +107,139 @@ -

List<primitive> or List<map> where map is of the type noted above

+

List<any type>

-

bag, with the primitive or map type as the field in each tuple of the bag

+

bag

-

struct<primitive fields>

+

struct<any type fields>

tuple

- - -

List<struct<primitive fields>>

- - -

bag, where each tuple in the bag maps to struct <primitive fields>

- -
+
+Running Pig with HCatalog + +

Pig does not automatically pick up HCatalog jars. You will need tell Pig where your HCatalog jars are. +These include the Hive jars used by the HCatalog client. To do this, you must define the environment +variable PIG_CLASSPATH with the appropriate jars. HCat can tell you the jars it needs. In order to do this it +needs to know where Hadoop is installed. Also, you need to tell Pig the URI for your metastore, in the PIG_OPTS +variable. In the case where you have installed Hadoop and HCatalog via tar, you can do:

+ + +export HADOOP_HOME=<path_to_hadoop_install> +export HCAT_JARS=`<path_to_hcat_install>/bin/hcat -classpath` +export PIG_CLASSPATH=${PIG_CLASSPATH}:${HCAT_JARS} +export PIG_OPTS=-Dhcat.metastore.uri=thrift:/<hostname>:<port> + +<path_to_pig_install>/bin/pig -Dpig.additional.jars=$HCAT_JARS script.pig + + +

When installed as an rpm, HCatalog already knows where Hadoop is, so it becomes:

+ + +export HCAT_JARS=`/usr/bin/hcat -classpath` +export PIG_CLASSPATH=${PIG_CLASSPATH}:${HCAT_JARS} +export PIG_OPTS=-Dhcat.metastore.uri=thrift://<hostname>:<port> + +/usr/bin/pig -Dpig.additional.jars=$HCAT_JARS script.pig + + +
+ + +
Examples + +

This load statement will load all partitions of the specified table.

/* myscript.pig */ -A = LOAD 'dbname.tablename' USING org.apache.hcatalog.pig.HCatLoader(); +A = LOAD 'tablename' USING org.apache.hcatalog.pig.HCatLoader(); ... ... -

If only some partitions of the specified table are needed, include a partition filter statement immediately following the load statement. -The filter statement can include conditions on partition as well as non-partition columns.

+

If only some partitions of the specified table are needed, include a partition filter statement immediately following the load statement in the data flow. (In the script, however, a filter statement might not immediately follow its load statement.) The filter statement can include conditions on partition as well as non-partition columns.

/* myscript.pig */ -A = LOAD 'dbname.tablename' USING org.apache.hcatalog.pig.HCatLoader(); - -B = filter A by date == '20100819' and by age < 30; -- datestamp is a partition column; age is not - -C = filter A by date == '20100819' and by country == 'US'; -- datestamp and country are partition columns +A = LOAD 'tablename' USING org.apache.hcatalog.pig.HCatLoader(); + +-- date is a partition column; age is not + +B = filter A by date == '20100819' and age < 30; + +-- both date and country are partition columns + +C = filter A by date == '20100819' and country == 'US'; ... ... -

Certain combinations of conditions on partition and non-partition columns are not allowed in filter statements. -For example, the following script results in this error message:



-ERROR 1112: Unsupported query: You have an partition column (datestamp ) in a construction like: (pcond and ...) or ( pcond and ...) where pcond is a condition on a partition column.



-A workaround is to restructure the filter condition by splitting it into multiple filter conditions, with the first condition immediately following the load statement. -

+

Example of scanning a whole table

-/* This script produces an ERROR */ +a = load 'student_data' using org.apache.hcatalog.pig.HCatLoader(); +b = foreach a generate name, age; + -A = LOAD 'default.search_austria' USING org.apache.hcatalog.pig.HCatLoader(); -B = FILTER A BY - ( (datestamp < '20091103' AND browser < 50) - OR (action == 'click' and browser > 100) - ); -... -... + +

Notice that the schema is automatically provided to Pig, there's no need to declare name and age as fields, as if +you were loading from a file.

+ +

Example of scanning a single partition. Assume the table web_logs is partitioned by the column datestamp:

+ + +a = load 'web_logs' using org.apache.hcatalog.pig.HCatLoader(); +b = filter a by datestamp == '20110924'; +

Pig will push the datestamp filter shown here to HCatalog, so that HCat knows to just scan the partition where +datestamp = '20110924'. You can combine this filter with others via 'and':

+ + +a = load 'web_logs' using org.apache.hcatalog.pig.HCatLoader(); +b = filter a by datestamp == '20110924' and user is not null; + + +

Pig will split the above filter, pushing the datestamp portion to HCatalog and retaining the user is not null part +to apply itself. You can also give a more complex filter to retrieve a set of partitions:

+ + +a = load 'web_logs' using org.apache.hcatalog.pig.HCatLoader(); +b = filter a by datestamp >= '20110924' and datestamp <= '20110931'; + + +

or

+ + +a = load 'web_logs' using org.apache.hcatalog.pig.HCatLoader(); +b = filter a by datestamp >= '20110924' or datestamp <= '20110925'; + + +

You can write to non-partitioned table simply by using HCatStorer. The contents of the table will be overwritten:

+ +store z into 'student_data' using org.apache.hcatalog.pig.HCatStorer(); + +

To add one new partition to a partitioned table, specify the partition value in store function. Pay careful +attention to the quoting, as the whole string must be single quoted and separated with an equals sign:

+ +store z into 'web_data' using org.apache.hcatalog.pig.HCatStorer('datestamp=20110924'); + +

To write into multiple partitions at one, make sure that the partition column is present in your data, then call +HCatStorer with no argument:

+ +store z into 'web_data' using org.apache.hcatalog.pig.HCatStorer(); -- datestamp must a field in the relation z +
+
@@ -189,26 +260,28 @@ ... my_processed_data = ... -STORE my_processed_data INTO 'dbname.tablename' - USING org.apache.hcatalog.pig.HCatStorer('month=12,date=25,hour=0300','a:int,b:chararray,c:map[]'); +STORE my_processed_data INTO 'tablename' USING + org.apache.hcatalog.pig.HCatStorer('month=12,date=25,hour=0300','a:int,b:chararray,c:map[]');

Assumptions

-

You must specify the database name and table name using this format: 'dbname.tablename'. Both the database and table must be created prior to running your Pig script. The Hive metastore lets you create tables without specifying a database; if you created tables this way, then the database name is 'default' and string becomes 'default.tablename'.

- +

You must specify the table name in single quotes: LOAD 'tablename'. Both the database and table must be created prior to running your Pig script. If you are using a non-default database you must specify your input as 'dbname.tablename'. If you are using Pig 0.9.2 or earlier, you must create your database and table prior to running the Pig script. Beginning with Pig 0.10 you can issue these create commands in Pig using the SQL command.

+

The Hive metastore lets you create tables without specifying a database; if you created tables this way, then the database name is 'default'.

For the USING clause, you can have two string arguments:

+

If partition columns are present in data they should not be specified as a STORE argument. Instead HCatalog will use these values to place records in the appropriate partition(s). It is valid to specify some partition keys in the STORE statement and have other partition keys in the data.

- + +
HCatalog Data Types

Restrictions apply to the types of columns HCatStorer can write.

@@ -229,15 +302,12 @@

primitives (int, long, float, double, string)

-

int, long, float, double, string



- Note: HCatStorer does NOT support writing table columns of type smallint or tinyint. - To be able to write form Pig using the HCatalog storer, table columns must by of type int or bigint. -

+

int, long, float, double, string to chararray

-

map (key type should be string, valuetype can be a primitive listed above)

+

map (key type should be string, valuetype must be string)

map

@@ -245,28 +315,20 @@ -

List<primitive> or List<map> where map is of the type noted above

+

List<any type>

-

bag, with the primitive or map type as the field in each tuple of the bag

+

bag

-

struct<primitive fields>

+

struct<any type fields>

tuple

- - -

List<struct<primitive fields>>

- - -

bag, where each tuple in the bag maps to struct <primitive fields>

- -