From hcatalog-commits-return-518-apmail-incubator-hcatalog-commits-archive=incubator.apache.org@incubator.apache.org Mon Oct 3 18:37:52 2011 Return-Path: X-Original-To: apmail-incubator-hcatalog-commits-archive@minotaur.apache.org Delivered-To: apmail-incubator-hcatalog-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 1C2F0912D for ; Mon, 3 Oct 2011 18:37:52 +0000 (UTC) Received: (qmail 19599 invoked by uid 500); 3 Oct 2011 18:37:51 -0000 Delivered-To: apmail-incubator-hcatalog-commits-archive@incubator.apache.org Received: (qmail 19573 invoked by uid 500); 3 Oct 2011 18:37:51 -0000 Mailing-List: contact hcatalog-commits-help@incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hcatalog-dev@incubator.apache.org Delivered-To: mailing list hcatalog-commits@incubator.apache.org Received: (qmail 19547 invoked by uid 99); 3 Oct 2011 18:37:51 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 03 Oct 2011 18:37:51 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 03 Oct 2011 18:37:49 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 8473123888EA; Mon, 3 Oct 2011 18:37:29 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1178508 - in /incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs: importexport.xml site.xml supportedformats.xml Date: Mon, 03 Oct 2011 18:37:29 -0000 To: hcatalog-commits@incubator.apache.org From: hashutosh@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20111003183729.8473123888EA@eris.apache.org> Author: hashutosh Date: Mon Oct 3 18:37:28 2011 New Revision: 1178508 URL: http://svn.apache.org/viewvc?rev=1178508&view=rev Log: Backport export-import to 0.2 tree Added: incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/importexport.xml Modified: incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/site.xml incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/supportedformats.xml Added: incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/importexport.xml URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/importexport.xml?rev=1178508&view=auto ============================================================================== --- incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/importexport.xml (added) +++ incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/importexport.xml Mon Oct 3 18:37:28 2011 @@ -0,0 +1,555 @@ + + + + + +
+ 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';
  • +
+
+ + +
Modified: incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/site.xml URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/site.xml?rev=1178508&r1=1178507&r2=1178508&view=diff ============================================================================== --- incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/site.xml (original) +++ incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/site.xml Mon Oct 3 18:37:28 2011 @@ -44,6 +44,7 @@ See http://forrest.apache.org/docs/linki + Modified: incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/supportedformats.xml URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/supportedformats.xml?rev=1178508&r1=1178507&r2=1178508&view=diff ============================================================================== --- incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/supportedformats.xml (original) +++ incubator/hcatalog/branches/branch-0.2/src/docs/src/documentation/content/xdocs/supportedformats.xml Mon Oct 3 18:37:28 2011 @@ -22,7 +22,7 @@ Storage Formats -

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

+

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

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.