Sqoop User Guide (v1.4.6)

Sqoop User Guide (v1.4.6)


  Licensed to the Apache Software Foundation (ASF) under one
  or more contributor license agreements.  See the NOTICE file
  distributed with this work for additional information
  regarding copyright ownership.  The ASF licenses this file
  to you under the Apache License, Version 2.0 (the
  "License"); you may not use this file except in compliance
  with the License.  You may obtain a copy of the License at

      http://www.apache.org/licenses/LICENSE-2.0

  Unless required by applicable law or agreed to in writing, software
  distributed under the License is distributed on an "AS IS" BASIS,
  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  See the License for the specific language governing permissions and
  limitations under the License.

1. Introduction

Sqoop is a tool designed to transfer data between Hadoop andrelational databases or mainframes. You can use Sqoop to import data from arelational database management system (RDBMS) such as MySQL or Oracle or amainframe into the Hadoop Distributed File System (HDFS),transform the data in Hadoop MapReduce, and then export the data backinto an RDBMS.

Sqoop automates most of this process, relying on the database todescribe the schema for the data to be imported. Sqoop uses MapReduceto import and export the data, which provides parallel operation aswell as fault tolerance.

This document describes how to get started using Sqoop to move databetween databases and Hadoop or mainframe to Hadoop and provides referenceinformation for the operation of the Sqoop command-line tool suite. Thisdocument is intended for:

  • System and application programmers
  • System administrators
  • Database administrators
  • Data analysts
  • Data engineers

2. Supported Releases

This documentation applies to Sqoop v1.4.6.

3. Sqoop Releases

Sqoop is an open source software product of the Apache Software Foundation.

Software development for Sqoop occurs at http://sqoop.apache.orgAt that site you can obtain:

  • New releases of Sqoop as well as its most recent source code
  • An issue tracker
  • A wiki that contains Sqoop documentation

4. Prerequisites

The following prerequisite knowledge is required for this product:

  • Basic computer technology and terminology
  • Familiarity with command-line interfaces such as bash
  • Relational database management systems
  • Basic familiarity with the purpose and operation of Hadoop

Before you can use Sqoop, a release of Hadoop must be installed andconfigured. Sqoop is currently supporting 4 major Hadoop releases - 0.20,0.23, 1.0 and 2.0.

This document assumes you are using a Linux or Linux-like environment.If you are using Windows, you may be able to use cygwin to accomplishmost of the following tasks. If you are using Mac OS X, you should seefew (if any) compatibility errors. Sqoop is predominantly operated andtested on Linux.

5. Basic Usage

With Sqoop, you can import data from a relational database system or amainframe into HDFS. The input to the import process is either database tableor mainframe datasets. For databases, Sqoop will read the table row-by-rowinto HDFS. For mainframe datasets, Sqoop will read records from each mainframedataset into HDFS. The output of this import process is a set of filescontaining a copy of the imported table or datasets.The import process is performed in parallel. For this reason, theoutput will be in multiple files. These files may be delimited textfiles (for example, with commas or tabs separating each field), orbinary Avro or SequenceFiles containing serialized record data.

A by-product of the import process is a generated Java class whichcan encapsulate one row of the imported table. This class is usedduring the import process by Sqoop itself. The Java source code forthis class is also provided to you, for use in subsequent MapReduceprocessing of the data. This class can serialize and deserialize datato and from the SequenceFile format. It can also parse thedelimited-text form of a record. These abilities allow you to quicklydevelop MapReduce applications that use the HDFS-stored records inyour processing pipeline. You are also free to parse the delimitedsrecord data yourself, using any other tools you prefer.

After manipulating the imported records (for example, with MapReduceor Hive) you may have a result data set which you can thenexportback to the relational database. Sqoop’s export process will reada set of delimited text files from HDFS in parallel, parse them intorecords, and insert them as new rows in a target database table, forconsumption by external applications or users.

Sqoop includes some other commands which allow you to inspect thedatabase you are working with. For example, you can list the availabledatabase schemas (with thesqoop-list-databases tool) and tableswithin a schema (with thesqoop-list-tables tool). Sqoop alsoincludes a primitive SQL execution shell (thesqoop-eval tool).

Most aspects of the import, code generation, and export processes canbe customized. For databases, you can control the specific row range orcolumns imported. You can specify particular delimiters and escape charactersfor the file-based representation of the data, as well as the file formatused. You can also control the class or package names used ingenerated code. Subsequent sections of this document explain how tospecify these and other arguments to Sqoop.

6. Sqoop Tools

Sqoop is a collection of related tools. To use Sqoop, you specify thetool you want to use and the arguments that control the tool.

If Sqoop is compiled from its own source, you can run Sqoop without a formalinstallation process by running thebin/sqoop program. Usersof a packaged deployment of Sqoop (such as an RPM shipped with Apache Bigtop)will see this program installed as/usr/bin/sqoop. The remainder of thisdocumentation will refer to this program assqoop. For example:

$ sqoop tool-name [tool-arguments]
[Note]Note

The following examples that begin with a $ character indicatethat the commands must be entered at a terminal prompt (such asbash). The$ character represents the prompt itself; you shouldnot start these commands by typing a$. You can also enter commandsinline in the text of a paragraph; for example,sqoop help. Theseexamples do not show a $ prefix, but you should enter them the sameway. Don’t confuse the $ shell prompt in the examples with the $that precedes an environment variable name. For example, the stringliteral$HADOOP_HOME includes a "$".

Sqoop ships with a help tool. To display a list of all availabletools, type the following command:

$ sqoop help
usage: sqoop COMMAND [ARGS]

Available commands:
  codegen            Generate code to interact with database records
  create-hive-table  Import a table definition into Hive
  eval               Evaluate a SQL statement and display the results
  export             Export an HDFS directory to a database table
  help               List available commands
  import             Import a table from a database to HDFS
  import-all-tables  Import tables from a database to HDFS
  import-mainframe   Import mainframe datasets to HDFS
  list-databases     List available databases on a server
  list-tables        List available tables in a database
  version            Display version information

See 'sqoop help COMMAND' for information on a specific command.

You can display help for a specific tool by entering: sqoop help(tool-name); for example,sqoop help import.

You can also add the --help argument to any command:sqoop import--help.

6.1. Using Command Aliases

In addition to typing the sqoop (toolname) syntax, you can use aliasscripts that specify thesqoop-(toolname) syntax. For example, thescripts sqoop-import, sqoop-export, etc. each select a specifictool.

6.2. Controlling the Hadoop Installation

You invoke Sqoop through the program launch capability provided byHadoop. The sqoop command-line program is a wrapper which runs thebin/hadoop script shipped with Hadoop. If you have multipleinstallations of Hadoop present on your machine, you can select theHadoop installation by setting the$HADOOP_COMMON_HOME and$HADOOP_MAPRED_HOME environment variables.

For example:

$ HADOOP_COMMON_HOME=/path/to/some/hadoop \
  HADOOP_MAPRED_HOME=/path/to/some/hadoop-mapreduce \
  sqoop import --arguments...

or:

$ export HADOOP_COMMON_HOME=/some/path/to/hadoop
$ export HADOOP_MAPRED_HOME=/some/path/to/hadoop-mapreduce
$ sqoop import --arguments...

If either of these variables are not set, Sqoop will fall back to$HADOOP_HOME. If it is not set either, Sqoop will use the defaultinstallation locations for Apache Bigtop,/usr/lib/hadoop and/usr/lib/hadoop-mapreduce, respectively.

The active Hadoop configuration is loaded from $HADOOP_HOME/conf/,unless the$HADOOP_CONF_DIR environment variable is set.

6.3. Using Generic and Specific Arguments

To control the operation of each Sqoop tool, you use generic andspecific arguments.

For example:

$ sqoop help import
usage: sqoop import [GENERIC-ARGS] [TOOL-ARGS]

Common arguments:
   --connect <jdbc-uri>     Specify JDBC connect string
   --connect-manager <class-name>     Specify connection manager class to use
   --driver <class-name>    Manually specify JDBC driver class to use
   --hadoop-mapred-home <dir>      Override $HADOOP_MAPRED_HOME
   --help                   Print usage instructions
   --password-file          Set path for file containing authentication password
   -P                       Read password from console
   --password <password>    Set authentication password
   --username <username>    Set authentication username
   --verbose                Print more information while working
   --hadoop-home <dir>     Deprecated. Override $HADOOP_HOME

[...]

Generic Hadoop command-line arguments:
(must preceed any tool-specific arguments)
Generic options supported are
-conf <configuration file>     specify an application configuration file
-D <property=value>            use value for given property
-fs <local|namenode:port>      specify a namenode
-jt <local|jobtracker:port>    specify a job tracker
-files <comma separated list of files>    specify comma separated files to be copied to the map reduce cluster
-libjars <comma separated list of jars>    specify comma separated jar files to include in the classpath.
-archives <comma separated list of archives>    specify comma separated archives to be unarchived on the compute machines.

The general command line syntax is
bin/hadoop command [genericOptions] [commandOptions]

You must supply the generic arguments -conf, -D, and so on after thetool name but before any tool-specific arguments (such as--connect). Note that generic Hadoop arguments are preceeded by asingle dash character (-), whereas tool-specific arguments startwith two dashes (--), unless they are single character arguments such as-P.

The -conf, -D, -fs and -jt arguments control the configurationand Hadoop server settings. For example, the-D mapred.job.name=<job_name> canbe used to set the name of the MR job that Sqoop launches, if not specified,the name defaults to the jar name for the job - which is derived from the usedtable name.

The -files, -libjars, and-archives arguments are not typically used withSqoop, but they are included as part of Hadoop’s internal argument-parsingsystem.

6.4. Using Options Files to Pass Arguments

When using Sqoop, the command line options that do not change frominvocation to invocation can be put in an options file for convenience.An options file is a text file where each line identifies an option inthe order that it appears otherwise on the command line. Option filesallow specifying a single option on multiple lines by using theback-slash character at the end of intermediate lines. Also supportedare comments within option files that begin with the hash character.Comments must be specified on a new line and may not be mixed withoption text. All comments and empty lines are ignored when optionfiles are expanded. Unless options appear as quoted strings, anyleading or trailing spaces are ignored. Quoted strings if used mustnot extend beyond the line on which they are specified.

Option files can be specified anywhere in the command line as long asthe options within them follow the otherwise prescribed rules ofoptions ordering. For instance, regardless of where the options areloaded from, they must follow the ordering such that generic optionsappear first, tool specific options next, finally followed by optionsthat are intended to be passed to child programs.

To specify an options file, simply create an options file in aconvenient location and pass it to the command line via--options-file argument.

Whenever an options file is specified, it is expanded on thecommand line before the tool is invoked. You can specify more thanone option files within the same invocation if needed.

For example, the following Sqoop invocation for import canbe specified alternatively as shown below:

$ sqoop import --connect jdbc:mysql://localhost/db --username foo --table TEST

$ sqoop --options-file /users/homer/work/import.txt --table TEST

where the options file /users/homer/work/import.txt contains the following:

import
--connect
jdbc:mysql://localhost/db
--username
foo

The options file can have empty lines and comments for readability purposes.So the above example would work exactly the same if the options file/users/homer/work/import.txt contained the following:

#
# Options file for Sqoop import
#

# Specifies the tool being invoked
import

# Connect parameter and value
--connect
jdbc:mysql://localhost/db

# Username parameter and value
--username
foo

#
# Remaining options should be specified in the command line.
#

6.5. Using Tools

The following sections will describe each tool’s operation. Thetools are listed in the most likely order you will find them useful.

7. sqoop-import

7.1. Purpose

The import tool imports an individual table from an RDBMS to HDFS.Each row from a table is represented as a separate record in HDFS.Records can be stored as text files (one record per line), or inbinary representation as Avro or SequenceFiles.

7.2. Syntax

$ sqoop import (generic-args) (import-args)
$ sqoop-import (generic-args) (import-args)

While the Hadoop generic arguments must precede any import arguments,you can type the import arguments in any order with respect to oneanother.

[Note]Note

In this document, arguments are grouped into collectionsorganized by function. Some collections are present in several tools(for example, the "common" arguments). An extended description of theirfunctionality is given only on the first presentation in thisdocument.

Table 1. Common arguments

ArgumentDescription
--connect <jdbc-uri>Specify JDBC connect string
--connection-manager <class-name>Specify connection manager class to use
--driver <class-name>Manually specify JDBC driver class to use
--hadoop-mapred-home <dir>Override $HADOOP_MAPRED_HOME
--helpPrint usage instructions
--password-fileSet path for a file containing the authentication password
-PRead password from console
--password <password>Set authentication password
--username <username>Set authentication username
--verbosePrint more information while working
--connection-param-file <filename>Optional properties file that provides connection parameters
--relaxed-isolationSet connection transaction isolation to read uncommitted for the mappers.

7.2.1. Connecting to a Database Server

Sqoop is designed to import tables from a database into HDFS. To doso, you must specify aconnect string that describes how to connect to thedatabase. Theconnect string is similar to a URL, and is communicated toSqoop with the--connect argument. This describes the server anddatabase to connect to; it may also specify the port. For example:

$ sqoop import --connect jdbc:mysql://database.example.com/employees

This string will connect to a MySQL database named employees on thehostdatabase.example.com. It’s important that you do not use the URLlocalhost if you intend to use Sqoop with a distributed Hadoopcluster. The connect string you supply will be used on TaskTracker nodesthroughout your MapReduce cluster; if you specify theliteral name localhost, each node will connect to a differentdatabase (or more likely, no database at all). Instead, you should usethe full hostname or IP address of the database host that can be seenby all your remote nodes.

You might need to authenticate against the database before you canaccess it. You can use the--username to supply a username to the database.Sqoop provides couple of different ways to supply a password,secure and non-secure, to the database which is detailed below.

Secure way of supplying password to the database. You should save the password in a file on the users home directory with 400permissions and specify the path to that file using the--password-fileargument, and is the preferred method of entering credentials. Sqoop willthen read the password from the file and pass it to the MapReduce clusterusing secure means with out exposing the password in the job configuration.The file containing the password can either be on the Local FS or HDFS.For example:

$ sqoop import --connect jdbc:mysql://database.example.com/employees \
    --username venkatesh --password-file ${user.home}/.password
[Warning]Warning

Sqoop will read entire content of the password file and use it asa password. This will include any trailing white space characters such asnew line characters that are added by default by most of the text editors.You need to make sure that your password file contains only charactersthat belongs to your password. On the command line you can use commandecho with switch-n to store password without any trailing white spacecharacters. For example to store passwordsecret you would callecho -n "secret" > password.file.

Another way of supplying passwords is using the -P argument which willread a password from a console prompt.

Protecting password from preying eyes. Hadoop 2.6.0 provides an API to separate password storage from applications.This API is called the credential provided API and there is a newcredential command line tool to manage passwords and their aliases.The passwords are stored with their aliases in a keystore that is passwordprotected. The keystore password can be the provided to a password prompton the command line, via an environment variable or defaulted to a softwaredefined constant. Please check the Hadoop documentation on the usageof this facility.

Once the password is stored using the Credential Provider facility andthe Hadoop configuration has been suitably updated, all applications canoptionally use the alias in place of the actual password and at runtimeresolve the alias for the password to use.

Since the keystore or similar technology used for storing the credentialprovider is shared across components, passwords for various applications,various database and other passwords can be securely stored in them and onlythe alias needs to be exposed in configuration files, protecting the passwordfrom being visible.

Sqoop has been enhanced to allow usage of this funcionality if it isavailable in the underlying Hadoop version being used. One new optionhas been introduced to provide the alias on the command line instead of theactual password (--password-alias). The argument value this option isthe alias on the storage associated with the actual password.Example usage is as follows:

$ sqoop import --connect jdbc:mysql://database.example.com/employees \
    --username dbuser --password-alias mydb.password.alias

Similarly, if the command line option is not preferred, the alias can be savedin the file provided with --password-file option. Along with this, theSqoop configuration parameter org.apache.sqoop.credentials.loader.classshould be set to the classname that provides the alias resolution:org.apache.sqoop.util.password.CredentialProviderPasswordLoader

Example usage is as follows (assuming .password.alias has the alias forthe real password) :

$ sqoop import --connect jdbc:mysql://database.example.com/employees \
    --username dbuser --password-file ${user.home}/.password-alias
[Warning]Warning

The --password parameter is insecure, as other users maybe able to read your password from the command-line arguments viathe output of programs such asps. The -P argument is the preferredmethod over using the--password argument. Credentials may still betransferred between nodes of the MapReduce cluster using insecure means.For example:

$ sqoop import --connect jdbc:mysql://database.example.com/employees \
    --username aaron --password 12345

Sqoop automatically supports several databases, including MySQL. Connectstrings beginning withjdbc:mysql:// are handled automatically in Sqoop. (Afull list of databases with built-in support is provided in the "SupportedDatabases" section. For some, you may need to install the JDBC driveryourself.)

You can use Sqoop with any otherJDBC-compliant database. First, download the appropriate JDBCdriver for the type of database you want to import, and install the .jarfile in the$SQOOP_HOME/lib directory on your client machine. (This willbe/usr/lib/sqoop/lib if you installed from an RPM or Debian package.)Each driver.jar file also has a specific driver class which definesthe entry-point to the driver. For example, MySQL’s Connector/J library hasa driver class ofcom.mysql.jdbc.Driver. Refer to your databasevendor-specific documentation to determine the main driver class.This class must be provided as an argument to Sqoop with--driver.

For example, to connect to a SQLServer database, first download the driver frommicrosoft.com and install it in your Sqoop lib path.

Then run Sqoop. For example:

$ sqoop import --driver com.microsoft.jdbc.sqlserver.SQLServerDriver \
    --connect <connect-string> ...

When connecting to a database using JDBC, you can optionally specify extraJDBC parameters via a property file using the option--connection-param-file. The contents of this file are parsed as standardJava properties and passed into the driver while creating a connection.

[Note]Note

The parameters specified via the optional property file are onlyapplicable to JDBC connections. Any fastpath connectors that use connectionsother than JDBC will ignore these parameters.

Table 2. Validation arguments More Details

ArgumentDescription
--validateEnable validation of data copied, supports single table copy only.
--validator <class-name>Specify validator class to use.
--validation-threshold <class-name>Specify validation threshold class to use.
--validation-failurehandler <class-name>Specify validation failure handler class to use.

Table 3. Import control arguments:

ArgumentDescription
--appendAppend data to an existing dataset in HDFS
--as-avrodatafileImports data to Avro Data Files
--as-sequencefileImports data to SequenceFiles
--as-textfileImports data as plain text (default)
--as-parquetfileImports data to Parquet Files
--boundary-query <statement>Boundary query to use for creating splits
--columns <col,col,col…>Columns to import from table
--delete-target-dirDelete the import target directory if it exists
--directUse direct connector if exists for the database
--fetch-size <n>Number of entries to read from database at once.
--inline-lob-limit <n>Set the maximum size for an inline LOB
-m,--num-mappers <n>Use n map tasks to import in parallel
-e,--query <statement>Import the results of statement.
--split-by <column-name>Column of the table used to split work units. Cannot be used with--autoreset-to-one-mapper option.
--autoreset-to-one-mapperImport should use one mapper if a table has no primary key and no split-by column is provided. Cannot be used with--split-by <col> option.
--table <table-name>Table to read
--target-dir <dir>HDFS destination dir
--warehouse-dir <dir>HDFS parent for table destination
--where <where clause>WHERE clause to use during import
-z,--compressEnable compression
--compression-codec <c>Use Hadoop codec (default gzip)
--null-string <null-string>The string to be written for a null value for string columns
--null-non-string <null-string>The string to be written for a null value for non-string columns

The --null-string and --null-non-string arguments are optional.\If not specified, then the string "null" will be used.

7.2.2. Selecting the Data to Import

Sqoop typically imports data in a table-centric fashion. Use the--table argument to select the table to import. For example,--tableemployees. This argument can also identify a VIEW or other table-likeentity in a database.

By default, all columns within a table are selected for import.Imported data is written to HDFS in its "natural order;" that is, atable containing columns A, B, and C result in an import of data suchas:

A1,B1,C1
A2,B2,C2
...

You can select a subset of columns and control their ordering by usingthe --columns argument. This should include a comma-delimited listof columns to import. For example:--columns "name,employee_id,jobtitle".

You can control which rows are imported by adding a SQL WHERE clauseto the import statement. By default, Sqoop generates statements of theformSELECT <column list> FROM <table name>. You can append aWHERE clause to this with the--where argument. For example: --where"id > 400". Only rows where the id column has a value greater than400 will be imported.

By default sqoop will use query select min(<split-by>), max(<split-by>) from<table name> to find out boundaries for creating splits. In some cases this queryis not the most optimal so you can specify any arbitrary query returning twonumeric columns using --boundary-query argument.

7.2.3. Free-form Query Imports

Sqoop can also import the result set of an arbitrary SQL query. Instead ofusing the--table, --columns and --where arguments, you can specifya SQL statement with the --query argument.

When importing a free-form query, you must specify a destination directorywith--target-dir.

If you want to import the results of a query in parallel, then each map taskwill need to execute a copy of the query, with results partitioned by boundingconditions inferred by Sqoop. Your query must include the token$CONDITIONSwhich each Sqoop process will replace with a unique condition expression.You must also select a splitting column with--split-by.

For example:

$ sqoop import \
  --query 'SELECT a.*, b.* FROM a JOIN b on (a.id == b.id) WHERE $CONDITIONS' \
  --split-by a.id --target-dir /user/foo/joinresults

Alternately, the query can be executed once and imported serially, byspecifying a single map task with-m 1:

$ sqoop import \
  --query 'SELECT a.*, b.* FROM a JOIN b on (a.id == b.id) WHERE $CONDITIONS' \
  -m 1 --target-dir /user/foo/joinresults
[Note]Note

If you are issuing the query wrapped with double quotes ("),you will have to use\$CONDITIONS instead of just $CONDITIONSto disallow your shell from treating it as a shell variable.For example, a double quoted query may look like:"SELECT * FROM x WHERE a='foo' AND \$CONDITIONS"

[Note]Note

The facility of using free-form query in the current version of Sqoopis limited to simple queries where there are no ambiguous projections andnoOR conditions in the WHERE clause. Use of complex queries such asqueries that have sub-queries or joins leading to ambiguous projections canlead to unexpected results.

7.2.4. Controlling Parallelism

Sqoop imports data in parallel from most database sources. You canspecify the numberof map tasks (parallel processes) to use to perform the import byusing the-m or --num-mappers argument. Each of these argumentstakes an integer value which corresponds to the degree of parallelismto employ. By default, four tasks are used. Some databases may seeimproved performance by increasing this value to 8 or 16. Do notincrease the degree of parallelism greater than that available withinyour MapReduce cluster; tasks will run serially and will likelyincrease the amount of time required to perform the import. Likewise,do not increase the degree of parallism higher than that which yourdatabase can reasonably support. Connecting 100 concurrent clients toyour database may increase the load on the database server to a pointwhere performance suffers as a result.

When performing parallel imports, Sqoop needs a criterion by which itcan split the workload. Sqoop uses asplitting column to split theworkload. By default, Sqoop will identify the primary key column (ifpresent) in a table and use it as the splitting column. The low andhigh values for the splitting column are retrieved from the database,and the map tasks operate on evenly-sized components of the totalrange. For example, if you had a table with a primary key column ofid whose minimum value was 0 and maximum value was 1000, and Sqoopwas directed to use 4 tasks, Sqoop would run four processes which eachexecute SQL statements of the formSELECT * FROM sometable WHERE id>= lo AND id < hi, with(lo, hi) set to (0, 250), (250, 500),(500, 750), and (750, 1001) in the different tasks.

If the actual values for the primary key are not uniformly distributedacross its range, then this can result in unbalanced tasks. You shouldexplicitly choose a different column with the--split-by argument.For example, --split-by employee_id. Sqoop cannot currently split onmulti-column indices. If your table has no index column, or has amulti-column key, then you must also manually choose a splittingcolumn.

If a table does not have a primary key defined and the --split-by <col>is not provided, then import will fail unless the numberof mappers is explicitly set to one with the--num-mappers 1 optionor the --autoreset-to-one-mapper option is used. The option--autoreset-to-one-mapper is typically used with the import-all-tablestool to automatically handle tables without a primary key in a schema.

7.2.5. Controlling Distributed Cache

Sqoop will copy the jars in $SQOOP_HOME/lib folder to job cache everytime when start a Sqoop job. When launched by Oozie this is unnecessarysince Oozie use its own Sqoop share lib which keeps Sqoop dependenciesin the distributed cache. Oozie will do the localization on eachworker node for the Sqoop dependencies only once during the first Sqoopjob and reuse the jars on worker node for subsquencial jobs. Usingoption--skip-dist-cache in Sqoop command when launched by Oozie willskip the step which Sqoop copies its dependencies to job cache and savemassive I/O.

7.2.6. Controlling the Import Process

By default, the import process will use JDBC which provides areasonable cross-vendor import channel. Some databases can performimports in a more high-performance fashion by using database-specificdata movement tools. For example, MySQL provides themysqldump toolwhich can export data from MySQL to other systems very quickly. Bysupplying the--direct argument, you are specifying that Sqoopshould attempt the direct import channel. This channel may behigher performance than using JDBC.

Details about use of direct mode with each specific RDBMS, installation requirements, availableoptions and limitations can be found inSection 25, “Notes for specific connectors”.

By default, Sqoop will import a table named foo to a directory namedfoo inside your home directory in HDFS. For example, if yourusername issomeuser, then the import tool will write to/user/someuser/foo/(files). You can adjust the parent directory ofthe import with the--warehouse-dir argument. For example:

$ sqoop import --connnect <connect-str> --table foo --warehouse-dir /shared \
    ...

This command would write to a set of files in the /shared/foo/ directory.

You can also explicitly choose the target directory, like so:

$ sqoop import --connnect <connect-str> --table foo --target-dir /dest \
    ...

This will import the files into the /dest directory.--target-dir isincompatible with --warehouse-dir.

When using direct mode, you can specify additional arguments whichshould be passed to the underlying tool. If the argument-- is given on the command-line, then subsequent arguments are sentdirectly to the underlying tool. For example, the following adjuststhe character set used by mysqldump:

$ sqoop import --connect jdbc:mysql://server.foo.com/db --table bar \
    --direct -- --default-character-set=latin1

By default, imports go to a new target location. If the destination directoryalready exists in HDFS, Sqoop will refuse to import and overwrite thatdirectory’s contents. If you use the--append argument, Sqoop will importdata to a temporary directory and then rename the files into the normaltarget directory in a manner that does not conflict with existing filenamesin that directory.

7.2.7. Controlling transaction isolation

By default, Sqoop uses the read committed transaction isolation in the mappersto import data. This may not be the ideal in all ETL workflows and it maydesired to reduce the isolation guarantees. The--relaxed-isolation optioncan be used to instruct Sqoop to use read uncommitted isolation level.

The read-uncommitted isolation level is not supported on all databases(for example, Oracle), so specifying the option--relaxed-isolationmay not be supported on all databases.

7.2.8. Controlling type mapping

Sqoop is preconfigured to map most SQL types to appropriate Java or Hiverepresentatives. However the default mapping might not be suitable foreveryone and might be overridden by--map-column-java (for changingmapping to Java) or --map-column-hive (for changing Hive mapping).

Table 4. Parameters for overriding mapping

ArgumentDescription
--map-column-java <mapping>Override mapping from SQL to Java type for configured columns.
--map-column-hive <mapping>Override mapping from SQL to Hive type for configured columns.

Sqoop is expecting comma separated list of mapping in form <name of column>=<new type>. For example:

$ sqoop import ... --map-column-java id=String,value=Integer

Sqoop will rise exception in case that some configured mapping will not be used.

7.2.9. Incremental Imports

Sqoop provides an incremental import mode which can be used to retrieveonly rows newer than some previously-imported set of rows.

The following arguments control incremental imports:

Table 5. Incremental import arguments:

ArgumentDescription
--check-column (col)Specifies the column to be examined when determining which rows to import. (the column should not be of type CHAR/NCHAR/VARCHAR/VARNCHAR/ LONGVARCHAR/LONGNVARCHAR)
--incremental (mode)Specifies how Sqoop determines which rows are new. Legal values formode include append andlastmodified.
--last-value (value)Specifies the maximum value of the check column from the previous import.

Sqoop supports two types of incremental imports: append andlastmodified.You can use the --incremental argument to specify the type of incrementalimport to perform.

You should specify append mode when importing a table where new rows arecontinually being added with increasing row id values. You specify the columncontaining the row’s id with--check-column. Sqoop imports rows where thecheck column has a value greater than the one specified with--last-value.

An alternate table update strategy supported by Sqoop is called lastmodifiedmode. You should use this when rows of the source table may be updated, andeach such update will set the value of a last-modified column to the currenttimestamp. Rows where the check column holds a timestamp more recent than thetimestamp specified with --last-value are imported.

At the end of an incremental import, the value which should be specified as--last-value for a subsequent import is printed to the screen. When runninga subsequent import, you should specify--last-value in this way to ensureyou import only the new or updated data. This is handled automatically bycreating an incremental import as a saved job, which is the preferredmechanism for performing a recurring incremental import. See the section onsaved jobs later in this document for more information.

7.2.10. File Formats

You can import data in one of two file formats: delimited text orSequenceFiles.

Delimited text is the default import format. You can also specify itexplicitly by using the--as-textfile argument. This argument will writestring-based representations of each record to the output files, withdelimiter characters between individual columns and rows. Thesedelimiters may be commas, tabs, or other characters. (The delimiterscan be selected; see "Output line formatting arguments.") Thefollowing is the results of an example text-based import:

1,here is a message,2010-05-01
2,happy new year!,2010-01-01
3,another message,2009-11-12

Delimited text is appropriate for most non-binary data types. It alsoreadily supports further manipulation by other tools, such as Hive.

SequenceFiles are a binary format that store individual records incustom record-specific data types. These data types are manifested asJava classes. Sqoop will automatically generate these data types foryou. This format supports exact storage of all data in binaryrepresentations, and is appropriate for storing binary data(for example,VARBINARY columns), or data that will be principlymanipulated by custom MapReduce programs (reading from SequenceFilesis higher-performance than reading from text files, as records do notneed to be parsed).

Avro data files are a compact, efficient binary format that providesinteroperability with applications written in other programminglanguages. Avro also supports versioning, so that when, e.g., columnsare added or removed from a table, previously imported data files canbe processed along with new ones.

By default, data is not compressed. You can compress your data byusing the deflate (gzip) algorithm with the-z or --compressargument, or specify any Hadoop compression codec using the--compression-codec argument. This applies to SequenceFile, text,and Avro files.

7.2.11. Large Objects

Sqoop handles large objects (BLOB and CLOB columns) in particularways. If this data is truly large, then these columns should not befully materialized in memory for manipulation, as most columns are.Instead, their data is handled in a streaming fashion. Large objectscan be stored inline with the rest of the data, in which case they arefully materialized in memory on every access, or they can be stored ina secondary storage file linked to the primary data storage. Bydefault, large objects less than 16 MB in size are stored inline withthe rest of the data. At a larger size, they are stored in files inthe _lobs subdirectory of the import target directory. These filesare stored in a separate format optimized for large record storage,which can accomodate records of up to 2^63 bytes each. The size atwhich lobs spill into separate files is controlled by the--inline-lob-limit argument, which takes a parameter specifying thelargest lob size to keep inline, in bytes. If you set the inline LOBlimit to 0, all large objects will be placed in externalstorage.

Table 6. Output line formatting arguments:

ArgumentDescription
--enclosed-by <char>Sets a required field enclosing character
--escaped-by <char>Sets the escape character
--fields-terminated-by <char>Sets the field separator character
--lines-terminated-by <char>Sets the end-of-line character
--mysql-delimitersUses MySQL’s default delimiter set: fields:, lines: \n escaped-by:\ optionally-enclosed-by: '
--optionally-enclosed-by <char>Sets a field enclosing character

When importing to delimited files, the choice of delimiter isimportant. Delimiters which appear inside string-based fields maycause ambiguous parsing of the imported data by subsequent analysispasses. For example, the string"Hello, pleased to meet you" shouldnot be imported with the end-of-field delimiter set to a comma.

Delimiters may be specified as:

  • a character (--fields-terminated-by X)
  • an escape character (--fields-terminated-by \t). Supported escape characters are:

    • \b (backspace)
    • \n (newline)
    • \r (carriage return)
    • \t (tab)
    • \" (double-quote)
    • \\' (single-quote)
    • \\ (backslash)
    • \0 (NUL) - This will insert NUL characters between fields or lines, or will disable enclosing/escaping if used for one of the--enclosed-by, --optionally-enclosed-by, or--escaped-by arguments.
  • The octal representation of a UTF-8 character’s code point. This should be of the form\0ooo, where ooo is the octal value. For example,--fields-terminated-by \001 would yield the ^A character.
  • The hexadecimal representation of a UTF-8 character’s code point. This should be of the form\0xhhh, where hhh is the hex value. For example,--fields-terminated-by \0x10 would yield the carriage return character.

The default delimiters are a comma (,) for fields, a newline (\n) for records, no quotecharacter, and no escape character. Note that this can lead toambiguous/unparsible records if you import database records containingcommas or newlines in the field data. For unambiguous parsing, both mustbe enabled. For example, via--mysql-delimiters.

If unambiguous delimiters cannot be presented, then use enclosing andescaping characters. The combination of (optional)enclosing and escaping characters will allow unambiguous parsing oflines. For example, suppose one column of a dataset contained thefollowing values:

Some string, with a comma.
Another "string with quotes"

The following arguments would provide delimiters which can beunambiguously parsed:

$ sqoop import --fields-terminated-by , --escaped-by \\ --enclosed-by '\"' ...

(Note that to prevent the shell from mangling the enclosing character,we have enclosed that argument itself in single-quotes.)

The result of the above arguments applied to the above dataset wouldbe:

"Some string, with a comma.","1","2","3"...
"Another \"string with quotes\"","4","5","6"...

Here the imported strings are shown in the context of additionalcolumns ("1","2","3", etc.) to demonstrate the full effect of enclosingand escaping. The enclosing character is only strictly necessary whendelimiter characters appear in the imported text. The enclosingcharacter can therefore be specified as optional:

$ sqoop import --optionally-enclosed-by '\"' (the rest as above)...

Which would result in the following import:

"Some string, with a comma.",1,2,3...
"Another \"string with quotes\"",4,5,6...
[Note]Note

Even though Hive supports escaping characters, it does nothandle escaping of new-line character. Also, it does not supportthe notion of enclosing characters that may include field delimitersin the enclosed string. It is therefore recommended that you chooseunambiguous field and record-terminating delimiters without the helpof escaping and enclosing characters when working with Hive; this isdue to limitations of Hive’s input parsing abilities.

The --mysql-delimiters argument is a shorthand argument which usesthe default delimiters for themysqldump program.If you use the mysqldump delimiters in conjunction with adirect-mode import (with --direct), very fast imports can beachieved.

While the choice of delimiters is most important for a text-modeimport, it is still relevant if you import to SequenceFiles with--as-sequencefile. The generated class'toString() methodwill use the delimiters you specify, so subsequent formatting ofthe output data will rely on the delimiters you choose.

Table 7. Input parsing arguments:

ArgumentDescription
--input-enclosed-by <char>Sets a required field encloser
--input-escaped-by <char>Sets the input escape character
--input-fields-terminated-by <char>Sets the input field separator
--input-lines-terminated-by <char>Sets the input end-of-line character
--input-optionally-enclosed-by <char>Sets a field enclosing character

When Sqoop imports data to HDFS, it generates a Java class which canreinterpret the text files that it creates when doing adelimited-format import. The delimiters are chosen with arguments suchas--fields-terminated-by; this controls both how the data iswritten to disk, and how the generatedparse() method reinterpretsthis data. The delimiters used by theparse() method can be chosenindependently of the output arguments, by using--input-fields-terminated-by, and so on. This is useful, for example, togenerate classes which can parse records created with one set ofdelimiters, and emit the records to a different set of files using aseparate set of delimiters.

Table 8. Hive arguments:

ArgumentDescription
--hive-home <dir>Override $HIVE_HOME
--hive-importImport tables into Hive (Uses Hive’s default delimiters if none are set.)
--hive-overwriteOverwrite existing data in the Hive table.
--create-hive-tableIf set, then the job will fail if the target hive
 table exits. By default this property is false.
--hive-table <table-name>Sets the table name to use when importing to Hive.
--hive-drop-import-delimsDrops \n,\r, and \01 from string fields when importing to Hive.
--hive-delims-replacementReplace \n, \r, and \01 from string fields with user defined string when importing to Hive.
--hive-partition-keyName of a hive field to partition are sharded on
--hive-partition-value <v>String-value that serves as partition key for this imported into hive in this job.
--map-column-hive <map>Override default mapping from SQL type to Hive type for configured columns.

7.2.12. Importing Data Into Hive

Sqoop’s import tool’s main function is to upload your data into filesin HDFS. If you have a Hive metastore associated with your HDFScluster, Sqoop can also import the data into Hive by generating andexecuting aCREATE TABLE statement to define the data’s layout inHive. Importing data into Hive is as simple as adding the--hive-import option to your Sqoop command line.

If the Hive table already exists, you can specify the--hive-overwrite option to indicate that existing table in hive mustbe replaced. After your data is imported into HDFS or this step isomitted, Sqoop will generate a Hive script containing a CREATE TABLEoperation defining your columns using Hive’s types, and a LOAD DATA INPATHstatement to move the data files into Hive’s warehouse directory.

The script will be executed by callingthe installed copy of hive on the machine where Sqoop is run. If you havemultiple Hive installations, orhive is not in your $PATH, use the--hive-home option to identify the Hive installation directory.Sqoop will use$HIVE_HOME/bin/hive from here.

[Note]Note

This function is incompatible with --as-avrodatafile and--as-sequencefile.

Even though Hive supports escaping characters, it does nothandle escaping of new-line character. Also, it does not supportthe notion of enclosing characters that may include field delimitersin the enclosed string. It is therefore recommended that you chooseunambiguous field and record-terminating delimiters without the helpof escaping and enclosing characters when working with Hive; this isdue to limitations of Hive’s input parsing abilities. If you do use--escaped-by,--enclosed-by, or --optionally-enclosed-by whenimporting data into Hive, Sqoop will print a warning message.

Hive will have problems using Sqoop-imported data if your database’srows contain string fields that have Hive’s default row delimiters(\n and\r characters) or column delimiters (\01 characters)present in them. You can use the--hive-drop-import-delims optionto drop those characters on import to give Hive-compatible text data.Alternatively, you can use the--hive-delims-replacement optionto replace those characters with a user-defined string on import to giveHive-compatible text data. These options should only be used if you useHive’s default delimiters and should not be used if different delimitersare specified.

Sqoop will pass the field and record delimiters through to Hive. If you donot set any delimiters and do use--hive-import, the field delimiter willbe set to ^A and the record delimiter will be set to \n to be consistentwith Hive’s defaults.

Sqoop will by default import NULL values as string null. Hive is howeverusing string\N to denote NULL values and therefore predicates dealingwithNULL (like IS NULL) will not work correctly. You should appendparameters--null-string and --null-non-string in case of import job or--input-null-string and--input-null-non-string in case of an export job ifyou wish to properly preserveNULL values. Because sqoop is using thoseparameters in generated code, you need to properly escape value\N to \\N:

$ sqoop import  ... --null-string '\\N' --null-non-string '\\N'

The table name used in Hive is, by default, the same as that of thesource table. You can control the output table name with the--hive-tableoption.

Hive can put data into partitions for more efficient queryperformance. You can tell a Sqoop job to import data for Hive into aparticular partition by specifying the--hive-partition-key and--hive-partition-value arguments. The partition value must be astring. Please see the Hive documentation for more details onpartitioning.

You can import compressed tables into Hive using the --compress and--compression-codec options. One downside to compressing tables importedinto Hive is that many codecs cannot be split for processing by parallel maptasks. The lzop codec, however, does support splitting. When importing tableswith this codec, Sqoop will automatically index the files for splitting andconfiguring a new Hive table with the correct InputFormat. This featurecurrently requires that all partitions of a table be compressed with the lzopcodec.

Table 9. HBase arguments:

ArgumentDescription
--column-family <family>Sets the target column family for the import
--hbase-create-tableIf specified, create missing HBase tables
--hbase-row-key <col>Specifies which input column to use as the row key
 In case, if input table contains composite
 key, then <col> must be in the form of a
 comma-separated list of composite key
 attributes
--hbase-table <table-name>Specifies an HBase table to use as the target instead of HDFS
--hbase-bulkloadEnables bulk loading

7.2.13. Importing Data Into HBase

Sqoop supports additional import targets beyond HDFS and Hive. Sqoopcan also import records into a table in HBase.

By specifying --hbase-table, you instruct Sqoop to importto a table in HBase rather than a directory in HDFS. Sqoop willimport data to the table specified as the argument to--hbase-table.Each row of the input table will be transformed into an HBasePut operation to a row of the output table. The key for each row istaken from a column of the input. By default Sqoop will use the split-bycolumn as the row key column. If that is not specified, it will try toidentify the primary key column, if any, of the source table. You canmanually specify the row key column with--hbase-row-key. Each outputcolumn will be placed in the same column family, which must be specifiedwith--column-family.

[Note]Note

This function is incompatible with direct import (parameter--direct).

If the input table has composite key, the --hbase-row-key must bein the form of a comma-separated list of composite key attributes.In this case, the row key for HBase row will be generated by combiningvalues of composite key attributes using underscore as a separator.NOTE: Sqoop import for a table with composite key will work only ifparameter--hbase-row-key has been specified.

If the target table and column family do not exist, the Sqoop job willexit with an error. You should create the target table and column familybefore running an import. If you specify--hbase-create-table, Sqoopwill create the target table and column family if they do not exist,using the default parameters from your HBase configuration.

Sqoop currently serializes all values to HBase by converting each fieldto its string representation (as if you were importing to HDFS in textmode), and then inserts the UTF-8 bytes of this string in the targetcell. Sqoop will skip all rows containing null values in all columnsexcept the row key column.

To decrease the load on hbase, Sqoop can do bulk loading as opposed todirect writes. To use bulk loading, enable it using--hbase-bulkload.

Table 10. Accumulo arguments:

ArgumentDescription
--accumulo-table <table-nam>Specifies an Accumulo table to use as the target instead of HDFS
--accumulo-column-family <family>Sets the target column family for the import
--accumulo-create-tableIf specified, create missing Accumulo tables
--accumulo-row-key <col>Specifies which input column to use as the row key
--accumulo-visibility <vis>(Optional) Specifies a visibility token to apply to all rows inserted into Accumulo. Default is the empty string.
--accumulo-batch-size <size>(Optional) Sets the size in bytes of Accumulo’s write buffer. Default is 4MB.
--accumulo-max-latency <ms>(Optional) Sets the max latency in milliseconds for the Accumulo batch writer. Default is 0.
--accumulo-zookeepers <host:port>Comma-separated list of Zookeeper servers used by the Accumulo instance
--accumulo-instance <table-name>Name of the target Accumulo instance
--accumulo-user <username>Name of the Accumulo user to import as
--accumulo-password <password>Password for the Accumulo user

7.2.14. Importing Data Into Accumulo

Sqoop supports importing records into a table in Accumulo

By specifying --accumulo-table, you instruct Sqoop to importto a table in Accumulo rather than a directory in HDFS. Sqoop willimport data to the table specified as the argument to--accumulo-table.Each row of the input table will be transformed into an AccumuloMutation operation to a row of the output table. The key for each row istaken from a column of the input. By default Sqoop will use the split-bycolumn as the row key column. If that is not specified, it will try toidentify the primary key column, if any, of the source table. You canmanually specify the row key column with--accumulo-row-key. Each outputcolumn will be placed in the same column family, which must be specifiedwith--accumulo-column-family.

[Note]Note

This function is incompatible with direct import (parameter--direct), and cannot be used in the same operation as an HBase import.

If the target table does not exist, the Sqoop job willexit with an error, unless the--accumulo-create-table parameter isspecified. Otherwise, you should create the target table before runningan import.

Sqoop currently serializes all values to Accumulo by converting each fieldto its string representation (as if you were importing to HDFS in textmode), and then inserts the UTF-8 bytes of this string in the targetcell.

By default, no visibility is applied to the resulting cells in Accumulo,so the data will be visible to any Accumulo user. Use the--accumulo-visibility parameter to specify a visibility token toapply to all rows in the import job.

For performance tuning, use the optional --accumulo-buffer-size\ and--accumulo-max-latency parameters. See Accumulo’s documentation foran explanation of the effects of these parameters.

In order to connect to an Accumulo instance, you must specify the locationof a Zookeeper ensemble using the--accumulo-zookeepers parameter,the name of the Accumulo instance (--accumulo-instance), and theusername and password to connect with (--accumulo-user and--accumulo-password respectively).

Table 11. Code generation arguments:

ArgumentDescription
--bindir <dir>Output directory for compiled objects
--class-name <name>Sets the generated class name. This overrides--package-name. When combined with --jar-file, sets the input class.
--jar-file <file>Disable code generation; use specified jar
--outdir <dir>Output directory for generated code
--package-name <name>Put auto-generated classes in this package
--map-column-java <m>Override default mapping from SQL type to Java type for configured columns.

As mentioned earlier, a byproduct of importing a table to HDFS is aclass which can manipulate the imported data. If the data is stored inSequenceFiles, this class will be used for the data’s serializationcontainer. Therefore, you should use this class in your subsequentMapReduce processing of the data.

The class is typically named after the table; a table named foo willgenerate a class named foo. You may want to override this classname. For example, if your table is namedEMPLOYEES, you may want tospecify --class-name Employee instead. Similarly, you can specifyjust the package name with--package-name. The following importgenerates a class namedcom.foocorp.SomeTable:

$ sqoop import --connect <connect-str> --table SomeTable --package-name com.foocorp

The .java source file for your class will be written to the currentworking directory when you runsqoop. You can control the outputdirectory with --outdir. For example, --outdir src/generated/.

The import process compiles the source into .class and.jar files;these are ordinarily stored under /tmp. You can select an alternatetarget directory with --bindir. For example, --bindir /scratch.

If you already have a compiled class that can be used to perform theimport and want to suppress the code-generation aspect of the importprocess, you can use an existing jar and class byproviding the--jar-file and --class-name options. For example:

$ sqoop import --table SomeTable --jar-file mydatatypes.jar \
    --class-name SomeTableType

This command will load the SomeTableType class out ofmydatatypes.jar.

7.2.15. Additional Import Configuration Properties

There are some additional properties which can be configured by modifyingconf/sqoop-site.xml. Properties can be specified the same as in Hadoopconfiguration files, for example:

  <property>
    <name>property.name</name>
    <value>property.value</value>
  </property>

They can also be specified on the command line in the generic arguments, forexample:

sqoop import -D property.name=property.value ...

Table 12. Additional import configuration properties:

ArgumentDescription
sqoop.bigdecimal.format.stringControls how BigDecimal columns will formatted when stored as a String. A value oftrue (default) will use toPlainString to store them without an exponent component (0.0000001); while a value offalse will use toString which may include an exponent (1E-7)
sqoop.hbase.add.row.keyWhen set to false (default), Sqoop will not add the column used as a row key into the row data in HBase. When set totrue, the column used as a row key will be added to the row data in HBase.

7.3. Example Invocations

The following examples illustrate how to use the import tool in a varietyof situations.

A basic import of a table named EMPLOYEES in the corp database:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES

A basic import requiring a login:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES \
    --username SomeUser -P
Enter password: (hidden)

Selecting specific columns from the EMPLOYEES table:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES \
    --columns "employee_id,first_name,last_name,job_title"

Controlling the import parallelism (using 8 parallel tasks):

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES \
    -m 8

Storing data in SequenceFiles, and setting the generated class name tocom.foocorp.Employee:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES \
    --class-name com.foocorp.Employee --as-sequencefile

Specifying the delimiters to use in a text-mode import:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES \
    --fields-terminated-by '\t' --lines-terminated-by '\n' \
    --optionally-enclosed-by '\"'

Importing the data to Hive:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES \
    --hive-import

Importing only new employees:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES \
    --where "start_date > '2010-01-01'"

Changing the splitting column from the default:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES \
    --split-by dept_id

Verifying that an import was successful:

$ hadoop fs -ls EMPLOYEES
Found 5 items
drwxr-xr-x   - someuser somegrp          0 2010-04-27 16:40 /user/someuser/EMPLOYEES/_logs
-rw-r--r--   1 someuser somegrp    2913511 2010-04-27 16:40 /user/someuser/EMPLOYEES/part-m-00000
-rw-r--r--   1 someuser somegrp    1683938 2010-04-27 16:40 /user/someuser/EMPLOYEES/part-m-00001
-rw-r--r--   1 someuser somegrp    7245839 2010-04-27 16:40 /user/someuser/EMPLOYEES/part-m-00002
-rw-r--r--   1 someuser somegrp    7842523 2010-04-27 16:40 /user/someuser/EMPLOYEES/part-m-00003

$ hadoop fs -cat EMPLOYEES/part-m-00000 | head -n 10
0,joe,smith,engineering
1,jane,doe,marketing
...

Performing an incremental import of new data, after having alreadyimported the first 100,000 rows of a table:

$ sqoop import --connect jdbc:mysql://db.foo.com/somedb --table sometable \
    --where "id > 100000" --target-dir /incremental_dataset --append

An import of a table named EMPLOYEES in the corp database that usesvalidation to validate the import using the table row count and number ofrows copied into HDFS:More Details

$ sqoop import --connect jdbc:mysql://db.foo.com/corp \
    --table EMPLOYEES --validate

8. sqoop-import-all-tables

8.1. Purpose

The import-all-tables tool imports a set of tables from an RDBMS to HDFS.Data from each table is stored in a separate directory in HDFS.

For the import-all-tables tool to be useful, the following conditionsmust be met:

  • Each table must have a single-column primary key or --autoreset-to-one-mapper option must be used.
  • You must intend to import all columns of each table.
  • You must not intend to use non-default splitting column, nor impose any conditions via aWHERE clause.

8.2. Syntax

$ sqoop import-all-tables (generic-args) (import-args)
$ sqoop-import-all-tables (generic-args) (import-args)

Although the Hadoop generic arguments must preceed any import arguments,the import arguments can be entered in any order with respect to oneanother.

Table 13. Common arguments

ArgumentDescription
--connect <jdbc-uri>Specify JDBC connect string
--connection-manager <class-name>Specify connection manager class to use
--driver <class-name>Manually specify JDBC driver class to use
--hadoop-mapred-home <dir>Override $HADOOP_MAPRED_HOME
--helpPrint usage instructions
--password-fileSet path for a file containing the authentication password
-PRead password from console
--password <password>Set authentication password
--username <username>Set authentication username
--verbosePrint more information while working
--connection-param-file <filename>Optional properties file that provides connection parameters
--relaxed-isolationSet connection transaction isolation to read uncommitted for the mappers.

Table 14. Import control arguments:

ArgumentDescription
--as-avrodatafileImports data to Avro Data Files
--as-sequencefileImports data to SequenceFiles
--as-textfileImports data as plain text (default)
--as-parquetfileImports data to Parquet Files
--directUse direct import fast path
--inline-lob-limit <n>Set the maximum size for an inline LOB
-m,--num-mappers <n>Use n map tasks to import in parallel
--warehouse-dir <dir>HDFS parent for table destination
-z,--compressEnable compression
--compression-codec <c>Use Hadoop codec (default gzip)
--exclude-tables <tables>Comma separated list of tables to exclude from import process
--autoreset-to-one-mapperImport should use one mapper if a table with no primary key is encountered

These arguments behave in the same manner as they do when used for thesqoop-import tool, but the--table, --split-by, --columns,and --where arguments are invalid forsqoop-import-all-tables.The --exclude-tables argument is for +sqoop-import-all-tables only.

Table 15. Output line formatting arguments:

ArgumentDescription
--enclosed-by <char>Sets a required field enclosing character
--escaped-by <char>Sets the escape character
--fields-terminated-by <char>Sets the field separator character
--lines-terminated-by <char>Sets the end-of-line character
--mysql-delimitersUses MySQL’s default delimiter set: fields:, lines: \n escaped-by:\ optionally-enclosed-by: '
--optionally-enclosed-by <char>Sets a field enclosing character

Table 16. Input parsing arguments:

ArgumentDescription
--input-enclosed-by <char>Sets a required field encloser
--input-escaped-by <char>Sets the input escape character
--input-fields-terminated-by <char>Sets the input field separator
--input-lines-terminated-by <char>Sets the input end-of-line character
--input-optionally-enclosed-by <char>Sets a field enclosing character

Table 17. Hive arguments:

ArgumentDescription
--hive-home <dir>Override $HIVE_HOME
--hive-importImport tables into Hive (Uses Hive’s default delimiters if none are set.)
--hive-overwriteOverwrite existing data in the Hive table.
--create-hive-tableIf set, then the job will fail if the target hive
 table exits. By default this property is false.
--hive-table <table-name>Sets the table name to use when importing to Hive.
--hive-drop-import-delimsDrops \n,\r, and \01 from string fields when importing to Hive.
--hive-delims-replacementReplace \n, \r, and \01 from string fields with user defined string when importing to Hive.
--hive-partition-keyName of a hive field to partition are sharded on
--hive-partition-value <v>String-value that serves as partition key for this imported into hive in this job.
--map-column-hive <map>Override default mapping from SQL type to Hive type for configured columns.

Table 18. Code generation arguments:

ArgumentDescription
--bindir <dir>Output directory for compiled objects
--jar-file <file>Disable code generation; use specified jar
--outdir <dir>Output directory for generated code
--package-name <name>Put auto-generated classes in this package

The import-all-tables tool does not support the --class-name argument.You may, however, specify a package with --package-name in which allgenerated classes will be placed.

8.3. Example Invocations

Import all tables from the corp database:

$ sqoop import-all-tables --connect jdbc:mysql://db.foo.com/corp

Verifying that it worked:

$ hadoop fs -ls
Found 4 items
drwxr-xr-x   - someuser somegrp       0 2010-04-27 17:15 /user/someuser/EMPLOYEES
drwxr-xr-x   - someuser somegrp       0 2010-04-27 17:15 /user/someuser/PAYCHECKS
drwxr-xr-x   - someuser somegrp       0 2010-04-27 17:15 /user/someuser/DEPARTMENTS
drwxr-xr-x   - someuser somegrp       0 2010-04-27 17:15 /user/someuser/OFFICE_SUPPLIES

9. sqoop-import-mainframe

9.1. Purpose

The import-mainframe tool imports all sequential datasetsin a partitioned dataset(PDS) on a mainframe to HDFS. A PDS isakin to a directory on the open systems.The records in a dataset can contain only character data.Records will be stored with the entire record as a single text field.

9.2. Syntax

$ sqoop import-mainframe (generic-args) (import-args)
$ sqoop-import-mainframe (generic-args) (import-args)

While the Hadoop generic arguments must precede any import arguments,you can type the import arguments in any order with respect to oneanother.

Table 19. Common arguments

ArgumentDescription
--connect <hostname>Specify mainframe host to connect
--connection-manager <class-name>Specify connection manager class to use
--hadoop-mapred-home <dir>Override $HADOOP_MAPRED_HOME
--helpPrint usage instructions
--password-fileSet path for a file containing the authentication password
-PRead password from console
--password <password>Set authentication password
--username <username>Set authentication username
--verbosePrint more information while working
--connection-param-file <filename>Optional properties file that provides connection parameters

9.2.1. Connecting to a Mainframe

Sqoop is designed to import mainframe datasets into HDFS. To doso, you must specify a mainframe host name in the Sqoop--connect argument.

$ sqoop import-mainframe --connect z390

This will connect to the mainframe host z390 via ftp.

You might need to authenticate against the mainframe host toaccess it. You can use the--username to supply a username to the mainframe.Sqoop provides couple of different ways to supply a password,secure and non-secure, to the mainframe which is detailed below.

Secure way of supplying password to the mainframe. You should save the password in a file on the users home directory with 400permissions and specify the path to that file using the--password-fileargument, and is the preferred method of entering credentials. Sqoop willthen read the password from the file and pass it to the MapReduce clusterusing secure means with out exposing the password in the job configuration.The file containing the password can either be on the Local FS or HDFS.

Example:

$ sqoop import-mainframe --connect z390 \
    --username david --password-file ${user.home}/.password

Another way of supplying passwords is using the -P argument which willread a password from a console prompt.

[Warning]Warning

The --password parameter is insecure, as other users maybe able to read your password from the command-line arguments viathe output of programs such asps. The -P argument is the preferredmethod over using the--password argument. Credentials may still betransferred between nodes of the MapReduce cluster using insecure means.

Example:

$ sqoop import-mainframe --connect z390 --username david --password 12345

Table 20. Import control arguments:

ArgumentDescription
--as-avrodatafileImports data to Avro Data Files
--as-sequencefileImports data to SequenceFiles
--as-textfileImports data as plain text (default)
--as-parquetfileImports data to Parquet Files
--delete-target-dirDelete the import target directory if it exists
-m,--num-mappers <n>Use n map tasks to import in parallel
--target-dir <dir>HDFS destination dir
--warehouse-dir <dir>HDFS parent for table destination
-z,--compressEnable compression
--compression-codec <c>Use Hadoop codec (default gzip)

9.2.2. Selecting the Files to Import

You can use the --dataset argument to specify a partitioned dataset name.All sequential datasets in the partitioned dataset will be imported.

9.2.3. Controlling Parallelism

Sqoop imports data in parallel by making multiple ftp connections to themainframe to transfer multiple files simultaneously. You can specify thenumber of map tasks (parallel processes) to use to perform the import byusing the-m or --num-mappers argument. Each of these argumentstakes an integer value which corresponds to the degree of parallelismto employ. By default, four tasks are used. You can adjust this value tomaximize the data transfer rate from the mainframe.

9.2.4. Controlling Distributed Cache

Sqoop will copy the jars in $SQOOP_HOME/lib folder to job cache everytime when start a Sqoop job. When launched by Oozie this is unnecessarysince Oozie use its own Sqoop share lib which keeps Sqoop dependenciesin the distributed cache. Oozie will do the localization on eachworker node for the Sqoop dependencies only once during the first Sqoopjob and reuse the jars on worker node for subsquencial jobs. Usingoption--skip-dist-cache in Sqoop command when launched by Oozie willskip the step which Sqoop copies its dependencies to job cache and savemassive I/O.

9.2.5. Controlling the Import Process

By default, Sqoop will import all sequential files in a partitioned datasetpds to a directory namedpds inside your home directory in HDFS. Forexample, if your username issomeuser, then the import tool will write to/user/someuser/pds/(files). You can adjust the parent directory ofthe import with the--warehouse-dir argument. For example:

$ sqoop import-mainframe --connnect <host> --dataset foo --warehouse-dir /shared \
    ...

This command would write to a set of files in the /shared/pds/ directory.

You can also explicitly choose the target directory, like so:

$ sqoop import-mainframe --connnect <host> --dataset foo --target-dir /dest \
    ...

This will import the files into the /dest directory.--target-dir isincompatible with --warehouse-dir.

By default, imports go to a new target location. If the destination directoryalready exists in HDFS, Sqoop will refuse to import and overwrite thatdirectory’s contents.

9.2.6. File Formats

By default, each record in a dataset is storedas a text record with a newline at the end. Each record is assumed to containa single text field with the name DEFAULT_COLUMN.When Sqoop imports data to HDFS, it generates a Java class which canreinterpret the text files that it creates.

You can also import mainframe records to Sequence, Avro, or Parquet files.

By default, data is not compressed. You can compress your data byusing the deflate (gzip) algorithm with the-z or --compressargument, or specify any Hadoop compression codec using the--compression-codec argument.

Table 21. Output line formatting arguments:

ArgumentDescription
--enclosed-by <char>Sets a required field enclosing character
--escaped-by <char>Sets the escape character
--fields-terminated-by <char>Sets the field separator character
--lines-terminated-by <char>Sets the end-of-line character
--mysql-delimitersUses MySQL’s default delimiter set: fields:, lines: \n escaped-by:\ optionally-enclosed-by: '
--optionally-enclosed-by <char>Sets a field enclosing character

Since mainframe record contains only one field, importing to delimited fileswill not contain any field delimiter. However, the field may be enclosed withenclosing character or escaped by an escaping character.

Table 22. Input parsing arguments:

ArgumentDescription
--input-enclosed-by <char>Sets a required field encloser
--input-escaped-by <char>Sets the input escape character
--input-fields-terminated-by <char>Sets the input field separator
--input-lines-terminated-by <char>Sets the input end-of-line character
--input-optionally-enclosed-by <char>Sets a field enclosing character

When Sqoop imports data to HDFS, it generates a Java class which canreinterpret the text files that it creates when doing adelimited-format import. The delimiters are chosen with arguments suchas--fields-terminated-by; this controls both how the data iswritten to disk, and how the generatedparse() method reinterpretsthis data. The delimiters used by theparse() method can be chosenindependently of the output arguments, by using--input-fields-terminated-by, and so on. This is useful, for example, togenerate classes which can parse records created with one set ofdelimiters, and emit the records to a different set of files using aseparate set of delimiters.

Table 23. Hive arguments:

ArgumentDescription
--hive-home <dir>Override $HIVE_HOME
--hive-importImport tables into Hive (Uses Hive’s default delimiters if none are set.)
--hive-overwriteOverwrite existing data in the Hive table.
--create-hive-tableIf set, then the job will fail if the target hive
 table exits. By default this property is false.
--hive-table <table-name>Sets the table name to use when importing to Hive.
--hive-drop-import-delimsDrops \n,\r, and \01 from string fields when importing to Hive.
--hive-delims-replacementReplace \n, \r, and \01 from string fields with user defined string when importing to Hive.
--hive-partition-keyName of a hive field to partition are sharded on
--hive-partition-value <v>String-value that serves as partition key for this imported into hive in this job.
--map-column-hive <map>Override default mapping from SQL type to Hive type for configured columns.

9.2.7. Importing Data Into Hive

Sqoop’s import tool’s main function is to upload your data into filesin HDFS. If you have a Hive metastore associated with your HDFScluster, Sqoop can also import the data into Hive by generating andexecuting aCREATE TABLE statement to define the data’s layout inHive. Importing data into Hive is as simple as adding the--hive-import option to your Sqoop command line.

If the Hive table already exists, you can specify the--hive-overwrite option to indicate that existing table in hive mustbe replaced. After your data is imported into HDFS or this step isomitted, Sqoop will generate a Hive script containing a CREATE TABLEoperation defining your columns using Hive’s types, and a LOAD DATA INPATHstatement to move the data files into Hive’s warehouse directory.

The script will be executed by callingthe installed copy of hive on the machine where Sqoop is run. If you havemultiple Hive installations, orhive is not in your $PATH, use the--hive-home option to identify the Hive installation directory.Sqoop will use$HIVE_HOME/bin/hive from here.

[Note]Note

This function is incompatible with --as-avrodatafile and--as-sequencefile.

Even though Hive supports escaping characters, it does nothandle escaping of new-line character. Also, it does not supportthe notion of enclosing characters that may include field delimitersin the enclosed string. It is therefore recommended that you chooseunambiguous field and record-terminating delimiters without the helpof escaping and enclosing characters when working with Hive; this isdue to limitations of Hive’s input parsing abilities. If you do use--escaped-by,--enclosed-by, or --optionally-enclosed-by whenimporting data into Hive, Sqoop will print a warning message.

Hive will have problems using Sqoop-imported data if your database’srows contain string fields that have Hive’s default row delimiters(\n and\r characters) or column delimiters (\01 characters)present in them. You can use the--hive-drop-import-delims optionto drop those characters on import to give Hive-compatible text data.Alternatively, you can use the--hive-delims-replacement optionto replace those characters with a user-defined string on import to giveHive-compatible text data. These options should only be used if you useHive’s default delimiters and should not be used if different delimitersare specified.

Sqoop will pass the field and record delimiters through to Hive. If you donot set any delimiters and do use--hive-import, the field delimiter willbe set to ^A and the record delimiter will be set to \n to be consistentwith Hive’s defaults.

Sqoop will by default import NULL values as string null. Hive is howeverusing string\N to denote NULL values and therefore predicates dealingwithNULL (like IS NULL) will not work correctly. You should appendparameters--null-string and --null-non-string in case of import job or--input-null-string and--input-null-non-string in case of an export job ifyou wish to properly preserveNULL values. Because sqoop is using thoseparameters in generated code, you need to properly escape value\N to \\N:

$ sqoop import  ... --null-string '\\N' --null-non-string '\\N'

The table name used in Hive is, by default, the same as that of thesource table. You can control the output table name with the--hive-tableoption.

Hive can put data into partitions for more efficient queryperformance. You can tell a Sqoop job to import data for Hive into aparticular partition by specifying the--hive-partition-key and--hive-partition-value arguments. The partition value must be astring. Please see the Hive documentation for more details onpartitioning.

You can import compressed tables into Hive using the --compress and--compression-codec options. One downside to compressing tables importedinto Hive is that many codecs cannot be split for processing by parallel maptasks. The lzop codec, however, does support splitting. When importing tableswith this codec, Sqoop will automatically index the files for splitting andconfiguring a new Hive table with the correct InputFormat. This featurecurrently requires that all partitions of a table be compressed with the lzopcodec.

Table 24. HBase arguments:

ArgumentDescription
--column-family <family>Sets the target column family for the import
--hbase-create-tableIf specified, create missing HBase tables
--hbase-row-key <col>Specifies which input column to use as the row key
 In case, if input table contains composite
 key, then <col> must be in the form of a
 comma-separated list of composite key
 attributes
--hbase-table <table-name>Specifies an HBase table to use as the target instead of HDFS
--hbase-bulkloadEnables bulk loading

9.2.8. Importing Data Into HBase

Sqoop supports additional import targets beyond HDFS and Hive. Sqoopcan also import records into a table in HBase.

By specifying --hbase-table, you instruct Sqoop to importto a table in HBase rather than a directory in HDFS. Sqoop willimport data to the table specified as the argument to--hbase-table.Each row of the input table will be transformed into an HBasePut operation to a row of the output table. The key for each row istaken from a column of the input. By default Sqoop will use the split-bycolumn as the row key column. If that is not specified, it will try toidentify the primary key column, if any, of the source table. You canmanually specify the row key column with--hbase-row-key. Each outputcolumn will be placed in the same column family, which must be specifiedwith--column-family.

[Note]Note

This function is incompatible with direct import (parameter--direct).

If the input table has composite key, the --hbase-row-key must bein the form of a comma-separated list of composite key attributes.In this case, the row key for HBase row will be generated by combiningvalues of composite key attributes using underscore as a separator.NOTE: Sqoop import for a table with composite key will work only ifparameter--hbase-row-key has been specified.

If the target table and column family do not exist, the Sqoop job willexit with an error. You should create the target table and column familybefore running an import. If you specify--hbase-create-table, Sqoopwill create the target table and column family if they do not exist,using the default parameters from your HBase configuration.

Sqoop currently serializes all values to HBase by converting each fieldto its string representation (as if you were importing to HDFS in textmode), and then inserts the UTF-8 bytes of this string in the targetcell. Sqoop will skip all rows containing null values in all columnsexcept the row key column.

To decrease the load on hbase, Sqoop can do bulk loading as opposed todirect writes. To use bulk loading, enable it using--hbase-bulkload.

Table 25. Accumulo arguments:

ArgumentDescription
--accumulo-table <table-nam>Specifies an Accumulo table to use as the target instead of HDFS
--accumulo-column-family <family>Sets the target column family for the import
--accumulo-create-tableIf specified, create missing Accumulo tables
--accumulo-row-key <col>Specifies which input column to use as the row key
--accumulo-visibility <vis>(Optional) Specifies a visibility token to apply to all rows inserted into Accumulo. Default is the empty string.
--accumulo-batch-size <size>(Optional) Sets the size in bytes of Accumulo’s write buffer. Default is 4MB.
--accumulo-max-latency <ms>(Optional) Sets the max latency in milliseconds for the Accumulo batch writer. Default is 0.
--accumulo-zookeepers <host:port>Comma-separated list of Zookeeper servers used by the Accumulo instance
--accumulo-instance <table-name>Name of the target Accumulo instance
--accumulo-user <username>Name of the Accumulo user to import as
--accumulo-password <password>Password for the Accumulo user

9.2.9. Importing Data Into Accumulo

Sqoop supports importing records into a table in Accumulo

By specifying --accumulo-table, you instruct Sqoop to importto a table in Accumulo rather than a directory in HDFS. Sqoop willimport data to the table specified as the argument to--accumulo-table.Each row of the input table will be transformed into an AccumuloMutation operation to a row of the output table. The key for each row istaken from a column of the input. By default Sqoop will use the split-bycolumn as the row key column. If that is not specified, it will try toidentify the primary key column, if any, of the source table. You canmanually specify the row key column with--accumulo-row-key. Each outputcolumn will be placed in the same column family, which must be specifiedwith--accumulo-column-family.

[Note]Note

This function is incompatible with direct import (parameter--direct), and cannot be used in the same operation as an HBase import.

If the target table does not exist, the Sqoop job willexit with an error, unless the--accumulo-create-table parameter isspecified. Otherwise, you should create the target table before runningan import.

Sqoop currently serializes all values to Accumulo by converting each fieldto its string representation (as if you were importing to HDFS in textmode), and then inserts the UTF-8 bytes of this string in the targetcell.

By default, no visibility is applied to the resulting cells in Accumulo,so the data will be visible to any Accumulo user. Use the--accumulo-visibility parameter to specify a visibility token toapply to all rows in the import job.

For performance tuning, use the optional --accumulo-buffer-size\ and--accumulo-max-latency parameters. See Accumulo’s documentation foran explanation of the effects of these parameters.

In order to connect to an Accumulo instance, you must specify the locationof a Zookeeper ensemble using the--accumulo-zookeepers parameter,the name of the Accumulo instance (--accumulo-instance), and theusername and password to connect with (--accumulo-user and--accumulo-password respectively).

Table 26. Code generation arguments:

ArgumentDescription
--bindir <dir>Output directory for compiled objects
--class-name <name>Sets the generated class name. This overrides--package-name. When combined with --jar-file, sets the input class.
--jar-file <file>Disable code generation; use specified jar
--outdir <dir>Output directory for generated code
--package-name <name>Put auto-generated classes in this package
--map-column-java <m>Override default mapping from SQL type to Java type for configured columns.

As mentioned earlier, a byproduct of importing a table to HDFS is aclass which can manipulate the imported data.You should use this class in your subsequentMapReduce processing of the data.

The class is typically named after the partitioned dataset name; apartitioned dataset namedfoo willgenerate a class named foo. You may want to override this classname. For example, if your partitioned datasetis namedEMPLOYEES, you may want tospecify --class-name Employee instead. Similarly, you can specifyjust the package name with--package-name. The following importgenerates a class namedcom.foocorp.SomePDS:

$ sqoop import-mainframe --connect <host> --dataset SomePDS --package-name com.foocorp

The .java source file for your class will be written to the currentworking directory when you runsqoop. You can control the outputdirectory with --outdir. For example, --outdir src/generated/.

The import process compiles the source into .class and.jar files;these are ordinarily stored under /tmp. You can select an alternatetarget directory with --bindir. For example, --bindir /scratch.

If you already have a compiled class that can be used to perform theimport and want to suppress the code-generation aspect of the importprocess, you can use an existing jar and class byproviding the--jar-file and --class-name options. For example:

$ sqoop import-mainframe --dataset SomePDS --jar-file mydatatypes.jar \
    --class-name SomePDSType

This command will load the SomePDSType class out ofmydatatypes.jar.

9.2.10. Additional Import Configuration Properties

There are some additional properties which can be configured by modifyingconf/sqoop-site.xml. Properties can be specified the same as in Hadoopconfiguration files, for example:

  <property>
    <name>property.name</name>
    <value>property.value</value>
  </property>

They can also be specified on the command line in the generic arguments, forexample:

sqoop import -D property.name=property.value ...

9.3. Example Invocations

The following examples illustrate how to use the import tool in a varietyof situations.

A basic import of all sequential files in a partitioned dataset namedEMPLOYEES in the mainframe host z390:

$ sqoop import-mainframe --connect z390 --dataset EMPLOYEES \
    --username SomeUser -P
Enter password: (hidden)

Controlling the import parallelism (using 8 parallel tasks):

$ sqoop import-mainframe --connect z390 --dataset EMPLOYEES \
    --username SomeUser --password-file mypassword -m 8

Importing the data to Hive:

$ sqoop import-mainframe --connect z390 --dataset EMPLOYEES \
    --hive-import

10. sqoop-export

10.1. Purpose

The export tool exports a set of files from HDFS back to an RDBMS.The target table must already exist in the database. The input filesare read and parsed into a set of records according to theuser-specified delimiters.

The default operation is to transform these into a set of INSERTstatements that inject the records into the database. In "update mode,"Sqoop will generateUPDATE statements that replace existing recordsin the database, and in "call mode" Sqoop will make a stored procedurecall for each record.

10.2. Syntax

$ sqoop export (generic-args) (export-args)
$ sqoop-export (generic-args) (export-args)

Although the Hadoop generic arguments must preceed any export arguments,the export arguments can be entered in any order with respect to oneanother.

Table 27. Common arguments

ArgumentDescription
--connect <jdbc-uri>Specify JDBC connect string
--connection-manager <class-name>Specify connection manager class to use
--driver <class-name>Manually specify JDBC driver class to use
--hadoop-mapred-home <dir>Override $HADOOP_MAPRED_HOME
--helpPrint usage instructions
--password-fileSet path for a file containing the authentication password
-PRead password from console
--password <password>Set authentication password
--username <username>Set authentication username
--verbosePrint more information while working
--connection-param-file <filename>Optional properties file that provides connection parameters
--relaxed-isolationSet connection transaction isolation to read uncommitted for the mappers.

Table 28. Validation arguments More Details

ArgumentDescription
--validateEnable validation of data copied, supports single table copy only.
--validator <class-name>Specify validator class to use.
--validation-threshold <class-name>Specify validation threshold class to use.
--validation-failurehandler <class-name>Specify validation failure handler class to use.

Table 29. Export control arguments:

ArgumentDescription
--columns <col,col,col…>Columns to export to table
--directUse direct export fast path
--export-dir <dir>HDFS source path for the export
-m,--num-mappers <n>Use n map tasks to export in parallel
--table <table-name>Table to populate
--call <stored-proc-name>Stored Procedure to call
--update-key <col-name>Anchor column to use for updates. Use a comma separated list of columns if there are more than one column.
--update-mode <mode>Specify how updates are performed when new rows are found with non-matching keys in database.
 Legal values for mode include updateonly (default) and allowinsert.
--input-null-string <null-string>The string to be interpreted as null for string columns
--input-null-non-string <null-string>The string to be interpreted as null for non-string columns
--staging-table <staging-table-name>The table in which data will be staged before being inserted into the destination table.
--clear-staging-tableIndicates that any data present in the staging table can be deleted.
--batchUse batch mode for underlying statement execution.

The --export-dir argument and one of --table or --call arerequired. These specify the table to populate in the database (or thestored procedure to call), and the directory in HDFS that containsthe source data.

By default, all columns within a table are selected for export. Youcan select a subset of columns and control their ordering by using the--columns argument. This should include a comma-delimited listof columns to export. For example: --columns "col1,col2,col3". Notethat columns that are not included in the--columns parameter needto have either defined default value or allowNULL values. Otherwiseyour database will reject the imported data which in turn will makeSqoop job fail.

You can control the number of mappers independently from the number offiles present in the directory. Export performance depends on thedegree of parallelism. By default, Sqoop will use four tasks inparallel for the export process. This may not be optimal; you willneed to experiment with your own particular setup. Additional tasksmay offer better concurrency, but if the database is alreadybottlenecked on updating indices, invoking triggers, and so on, thenadditional load may decrease performance. The--num-mappers or -marguments control the number of map tasks, which is the degree ofparallelism used.

Some databases provides a direct mode for exports as well. Use the --direct argumentto specify this codepath. This may be higher-performance than the standard JDBC codepath.Details about use of direct mode with each specific RDBMS, installation requirements, availableoptions and limitations can be found inSection 25, “Notes for specific connectors”.

The --input-null-string and --input-null-non-string arguments areoptional. If --input-null-string is not specified, then the string"null" will be interpreted as null for string-type columns.If--input-null-non-string is not specified, then both the string"null" and the empty string will be interpreted as null for non-stringcolumns. Note that, the empty string will be always interpreted as nullfor non-string columns, in addition to other string if specified by--input-null-non-string.

Since Sqoop breaks down export process into multiple transactions, itis possible that a failed export job may result in partial data beingcommitted to the database. This can further lead to subsequent jobsfailing due to insert collisions in some cases, or lead to duplicated datain others. You can overcome this problem by specifying a staging table viathe--staging-table option which acts as an auxiliary table that is usedto stage exported data. The staged data is finally moved to the destinationtable in a single transaction.

In order to use the staging facility, you must create the staging tableprior to running the export job. This table must be structurallyidentical to the target table. This table should either be empty beforethe export job runs, or the--clear-staging-table option must be specified.If the staging table contains data and the--clear-staging-table option isspecified, Sqoop will delete all of the data before starting the export job.

[Note]Note

Support for staging data prior to pushing it into the destinationtable is not always available for--direct exports. It is also not available whenexport is invoked using the--update-key option for updating existing data,and when stored procedures are used to insert the data. It is best to check theSection 25, “Notes for specific connectors” section to validate.

10.3. Inserts vs. Updates

By default, sqoop-export appends new rows to a table; each inputrecord is transformed into anINSERT statement that adds a row to thetarget database table. If your table has constraints (e.g., a primarykey column whose values must be unique) and already contains data, youmust take care to avoid inserting records that violate theseconstraints. The export process will fail if an INSERT statementfails. This mode is primarily intended for exporting records to a new,empty table intended to receive these results.

If you specify the --update-key argument, Sqoop will instead modifyan existing dataset in the database. Each input record is treated asanUPDATE statement that modifies an existing row. The row astatement modifies is determined by the column name(s) specified with--update-key. For example, consider the following tabledefinition:

CREATE TABLE foo(
    id INT NOT NULL PRIMARY KEY,
    msg VARCHAR(32),
    bar INT);

Consider also a dataset in HDFS containing records like these:

0,this is a test,42
1,some more data,100
...

Running sqoop-export --table foo --update-key id --export-dir/path/to/data --connect … will run an export job that executes SQLstatements based on the data like so:

UPDATE foo SET msg='this is a test', bar=42 WHERE id=0;
UPDATE foo SET msg='some more data', bar=100 WHERE id=1;
...

If an UPDATE statement modifies no rows, this is not considered anerror; the export will silently continue. (In effect, this means thatan update-based export will not insert new rows into the database.)Likewise, if the column specified with --update-key does notuniquely identify rows and multiple rows are updated by a singlestatement, this condition is also undetected.

The argument --update-key can also be given a comma separated list ofcolumn names. In which case, Sqoop will match all keys from this list beforeupdating any existing record.

Depending on the target database, you may also specify the --update-modeargument with allowinsert mode if you want to update rows if they existin the database already or insert rows if they do not exist yet.

Table 30. Input parsing arguments:

ArgumentDescription
--input-enclosed-by <char>Sets a required field encloser
--input-escaped-by <char>Sets the input escape character
--input-fields-terminated-by <char>Sets the input field separator
--input-lines-terminated-by <char>Sets the input end-of-line character
--input-optionally-enclosed-by <char>Sets a field enclosing character

Table 31. Output line formatting arguments:

ArgumentDescription
--enclosed-by <char>Sets a required field enclosing character
--escaped-by <char>Sets the escape character
--fields-terminated-by <char>Sets the field separator character
--lines-terminated-by <char>Sets the end-of-line character
--mysql-delimitersUses MySQL’s default delimiter set: fields:, lines: \n escaped-by:\ optionally-enclosed-by: '
--optionally-enclosed-by <char>Sets a field enclosing character

Sqoop automatically generates code to parse and interpret records of thefiles containing the data to be exported back to the database. Ifthese files were created with non-default delimiters (comma-separatedfields with newline-separated records), you should specifythe same delimiters again so that Sqoop can parse your files.

If you specify incorrect delimiters, Sqoop will fail to find enoughcolumns per line. This will cause export map tasks to fail by throwingParseExceptions.

Table 32. Code generation arguments:

ArgumentDescription
--bindir <dir>Output directory for compiled objects
--class-name <name>Sets the generated class name. This overrides--package-name. When combined with --jar-file, sets the input class.
--jar-file <file>Disable code generation; use specified jar
--outdir <dir>Output directory for generated code
--package-name <name>Put auto-generated classes in this package
--map-column-java <m>Override default mapping from SQL type to Java type for configured columns.

If the records to be exported were generated as the result of aprevious import, then the original generated class can be used to readthe data back. Specifying--jar-file and --class-name obviatethe need to specify delimiters in this case.

The use of existing generated code is incompatible with--update-key; an update-mode export requires new code generation toperform the update. You cannot use--jar-file, and must fully specifyany non-default delimiters.

10.4. Exports and Transactions

Exports are performed by multiple writers in parallel. Each writeruses a separate connection to the database; these have separatetransactions from one another. Sqoop uses the multi-rowINSERTsyntax to insert up to 100 records per statement. Every 100statements, the current transaction within a writer task is committed,causing a commit every 10,000 rows. This ensures that transactionbuffers do not grow without bound, and cause out-of-memory conditions.Therefore, an export is not an atomic process. Partial results fromthe export will become visible before the export is complete.

10.5. Failed Exports

Exports may fail for a number of reasons:

  • Loss of connectivity from the Hadoop cluster to the database (either due to hardware fault, or server software crashes)
  • Attempting to INSERT a row which violates a consistency constraint (for example, inserting a duplicate primary key value)
  • Attempting to parse an incomplete or malformed record from the HDFS source data
  • Attempting to parse records using incorrect delimiters
  • Capacity issues (such as insufficient RAM or disk space)

If an export map task fails due to these or other reasons, it willcause the export job to fail. The results of a failed export areundefined. Each export map task operates in a separate transaction.Furthermore, individual map taskscommit their current transactionperiodically. If a task fails, the current transaction will be rolledback. Any previously-committed transactions will remain durable in thedatabase, leading to a partially-complete export.

10.6. Example Invocations

A basic export to populate a table named bar:

$ sqoop export --connect jdbc:mysql://db.example.com/foo --table bar  \
    --export-dir /results/bar_data

This example takes the files in /results/bar_data and injects theircontents in to thebar table in the foo database ondb.example.com.The target table must already exist in the database. Sqoop performsa set ofINSERT INTO operations, without regard for existing content. IfSqoop attempts to insert rows which violate constraints in the database(for example, a particular primary key value already exists), then the exportfails.

Alternatively, you can specify the columns to be exported by providing--columns "col1,col2,col3". Please note that columns that are not includedin the--columns parameter need to have either defined default value orallowNULL values. Otherwise your database will reject the imported datawhich in turn will make Sqoop job fail.

Another basic export to populate a table named bar with validation enabled:More Details

$ sqoop export --connect jdbc:mysql://db.example.com/foo --table bar  \
    --export-dir /results/bar_data --validate

An export that calls a stored procedure named barproc for every record in/results/bar_data would look like:

$ sqoop export --connect jdbc:mysql://db.example.com/foo --call barproc \
    --export-dir /results/bar_data

11. validation

11.1. Purpose

Validate the data copied, either import or export by comparing the rowcounts from the source and the target post copy.

11.2. Introduction

There are 3 basic interfaces:ValidationThreshold - Determines if the error margin between the source andtarget are acceptable: Absolute, Percentage Tolerant, etc.Default implementation is AbsoluteValidationThreshold which ensures the rowcounts from source and targets are the same.

ValidationFailureHandler - Responsible for handling failures: log anerror/warning, abort, etc.Default implementation is LogOnFailureHandler that logs a warning message tothe configured logger.

Validator - Drives the validation logic by delegating the decision toValidationThreshold and delegating failure handling to ValidationFailureHandler.The default implementation is RowCountValidator which validates the rowcounts from source and the target.

11.3. Syntax

$ sqoop import (generic-args) (import-args)
$ sqoop export (generic-args) (export-args)

Validation arguments are part of import and export arguments.

11.4. Configuration

The validation framework is extensible and pluggable. It comes with defaultimplementations but the interfaces can be extended to allow customimplementations by passing them as part of the command line arguments asdescribed below.

Validator. 

Property:         validator
Description:      Driver for validation,
                  must implement org.apache.sqoop.validation.Validator
Supported values: The value has to be a fully qualified class name.
Default value:    org.apache.sqoop.validation.RowCountValidator

Validation Threshold. 

Property:         validation-threshold
Description:      Drives the decision based on the validation meeting the
                  threshold or not. Must implement
                  org.apache.sqoop.validation.ValidationThreshold
Supported values: The value has to be a fully qualified class name.
Default value:    org.apache.sqoop.validation.AbsoluteValidationThreshold

Validation Failure Handler. 

Property:         validation-failurehandler
Description:      Responsible for handling failures, must implement
                  org.apache.sqoop.validation.ValidationFailureHandler
Supported values: The value has to be a fully qualified class name.
Default value:    org.apache.sqoop.validation.AbortOnFailureHandler

11.5. Limitations

Validation currently only validates data copied from a single table into HDFS.The following are the limitations in the current implementation:

  • all-tables option
  • free-form query option
  • Data imported into Hive, HBase or Accumulo
  • table import with --where argument
  • incremental imports

11.6. Example Invocations

A basic import of a table named EMPLOYEES in the corp database that usesvalidation to validate the row counts:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp  \
    --table EMPLOYEES --validate

A basic export to populate a table named bar with validation enabled:

$ sqoop export --connect jdbc:mysql://db.example.com/foo --table bar  \
    --export-dir /results/bar_data --validate

Another example that overrides the validation args:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES \
    --validate --validator org.apache.sqoop.validation.RowCountValidator \
    --validation-threshold \
          org.apache.sqoop.validation.AbsoluteValidationThreshold \
    --validation-failurehandler \
          org.apache.sqoop.validation.AbortOnFailureHandler

12. Saved Jobs

Imports and exports can be repeatedly performed by issuing the same commandmultiple times. Especially when using the incremental import capability,this is an expected scenario.

Sqoop allows you to define saved jobs which make this process easier. Asaved job records the configuration information required to execute aSqoop command at a later time. The section on thesqoop-job tooldescribes how to create and work with saved jobs.

By default, job descriptions are saved to a private repository storedin $HOME/.sqoop/. You can configure Sqoop to instead use a sharedmetastore, which makes saved jobs available to multiple users across ashared cluster. Starting the metastore is covered by the section on thesqoop-metastore tool.

13. sqoop-job

13.1. Purpose

The job tool allows you to create and work with saved jobs. Saved jobsremember the parameters used to specify a job, so they can bere-executed by invoking the job by its handle.

If a saved job is configured to perform an incremental import, state regardingthe most recently imported rows is updated in the saved job to allow the jobto continually import only the newest rows.

13.2. Syntax

$ sqoop job (generic-args) (job-args) [-- [subtool-name] (subtool-args)]
$ sqoop-job (generic-args) (job-args) [-- [subtool-name] (subtool-args)]

Although the Hadoop generic arguments must preceed any job arguments,the job arguments can be entered in any order with respect to oneanother.

Table 33. Job management options:

ArgumentDescription
--create <job-id>Define a new saved job with the specified job-id (name). A second Sqoop command-line, separated by a-- should be specified; this defines the saved job.
--delete <job-id>Delete a saved job.
--exec <job-id>Given a job defined with --create, run the saved job.
--show <job-id>Show the parameters for a saved job.
--listList all saved jobs

Creating saved jobs is done with the --create action. This operationrequires a-- followed by a tool name and its arguments. The tool andits arguments will form the basis of the saved job. Consider:

$ sqoop job --create myjob -- import --connect jdbc:mysql://example.com/db \
    --table mytable

This creates a job named myjob which can be executed later. The job is notrun. This job is now available in the list of saved jobs:

$ sqoop job --list
Available jobs:
  myjob

We can inspect the configuration of a job with the show action:

 $ sqoop job --show myjob
 Job: myjob
 Tool: import
 Options:
 ----------------------------
 direct.import = false
 codegen.input.delimiters.record = 0
 hdfs.append.dir = false
 db.table = mytable
 ...

And if we are satisfied with it, we can run the job with exec:

$ sqoop job --exec myjob
10/08/19 13:08:45 INFO tool.CodeGenTool: Beginning code generation
...

The exec action allows you to override arguments of the saved jobby supplying them after a--. For example, if the database werechanged to require a username, we could specify the username andpassword with:

$ sqoop job --exec myjob -- --username someuser -P
Enter password:
...

Table 34. Metastore connection options:

ArgumentDescription
--meta-connect <jdbc-uri>Specifies the JDBC connect string used to connect to the metastore

By default, a private metastore is instantiated in $HOME/.sqoop. Ifyou have configured a hosted metastore with thesqoop-metastoretool, you can connect to it by specifying the--meta-connectargument. This is a JDBC connect string just like the ones used toconnect to databases for import.

In conf/sqoop-site.xml, you can configuresqoop.metastore.client.autoconnect.url with this address, so you do not haveto supply--meta-connect to use a remote metastore. This parameter canalso be modified to move the private metastore to a location on yourfilesystem other than your home directory.

If you configure sqoop.metastore.client.enable.autoconnect with thevaluefalse, then you must explicitly supply --meta-connect.

Table 35. Common options:

ArgumentDescription
--helpPrint usage instructions
--verbosePrint more information while working

13.3. Saved jobs and passwords

The Sqoop metastore is not a secure resource. Multiple users can accessits contents. For this reason, Sqoop does not store passwords in themetastore. If you create a job that requires a password, you will beprompted for that password each time you execute the job.

You can enable passwords in the metastore by settingsqoop.metastore.client.record.password totrue in the configuration.

Note that you have to set sqoop.metastore.client.record.password totrueif you are executing saved jobs via Oozie because Sqoop cannot prompt the userto enter passwords while being executed as Oozie tasks.

13.4. Saved jobs and incremental imports

Incremental imports are performed by comparing the values in a check columnagainst a reference value for the most recent import. For example, if the--incremental append argument was specified, along with--check-columnid and --last-value 100, all rows withid > 100 will be imported.If an incremental import is run from the command line, the value whichshould be specified as--last-value in a subsequent incremental importwill be printed to the screen for your reference. If an incremental import isrun from a saved job, this value will be retained in the saved job. Subsequentruns ofsqoop job --exec someIncrementalJob will continue to import onlynewer rows than those previously imported.

14. sqoop-metastore

14.1. Purpose

The metastore tool configures Sqoop to host a shared metadata repository.Multiple users and/or remote users can define and execute saved jobs (createdwithsqoop job) defined in this metastore.

Clients must be configured to connect to the metastore in sqoop-site.xml orwith the --meta-connect argument.

14.2. Syntax

$ sqoop metastore (generic-args) (metastore-args)
$ sqoop-metastore (generic-args) (metastore-args)

Although the Hadoop generic arguments must preceed any metastore arguments,the metastore arguments can be entered in any order with respect to oneanother.

Table 36. Metastore management options:

ArgumentDescription
--shutdownShuts down a running metastore instance on the same machine.

Running sqoop-metastore launches a shared HSQLDB database instance onthe current machine. Clients can connect to this metastore and create jobswhich can be shared between users for execution.

The location of the metastore’s files on disk is controlled by thesqoop.metastore.server.location property inconf/sqoop-site.xml.This should point to a directory on the local filesystem.

The metastore is available over TCP/IP. The port is controlled by thesqoop.metastore.server.port configuration parameter, and defaults to 16000.

Clients should connect to the metastore by specifyingsqoop.metastore.client.autoconnect.url or--meta-connect with thevalue jdbc:hsqldb:hsql://<server-name>:<port>/sqoop. For example,jdbc:hsqldb:hsql://metaserver.example.com:16000/sqoop.

This metastore may be hosted on a machine within the Hadoop cluster, orelsewhere on the network.

15. sqoop-merge

15.1. Purpose

The merge tool allows you to combine two datasets where entries in onedataset should overwrite entries of an older dataset. For example, anincremental import run in last-modified mode will generate multiple datasetsin HDFS where successively newer data appears in each dataset. The mergetool will "flatten" two datasets into one, taking the newest availablerecords for each primary key.

15.2. Syntax

$ sqoop merge (generic-args) (merge-args)
$ sqoop-merge (generic-args) (merge-args)

Although the Hadoop generic arguments must preceed any merge arguments,the job arguments can be entered in any order with respect to oneanother.

Table 37. Merge options:

ArgumentDescription
--class-name <class>Specify the name of the record-specific class to use during the merge job.
--jar-file <file>Specify the name of the jar to load the record class from.
--merge-key <col>Specify the name of a column to use as the merge key.
--new-data <path>Specify the path of the newer dataset.
--onto <path>Specify the path of the older dataset.
--target-dir <path>Specify the target path for the output of the merge job.

The merge tool runs a MapReduce job that takes two directories asinput: a newer dataset, and an older one. These are specified with--new-data and--onto respectively. The output of the MapReducejob will be placed in the directory in HDFS specified by--target-dir.

When merging the datasets, it is assumed that there is a unique primarykey value in each record. The column for the primary key is specifiedwith--merge-key. Multiple rows in the same dataset should nothave the same primary key, or else data loss may occur.

To parse the dataset and extract the key column, the auto-generatedclass from a previous import must be used. You should specify theclass name and jar file with--class-name and --jar-file. Ifthis is not availab,e you can recreate the class using thecodegentool.

The merge tool is typically run after an incremental import with thedate-last-modified mode (sqoop import --incremental lastmodified …).

Supposing two incremental imports were performed, where some older datais in an HDFS directory namedolder and newer data is in an HDFSdirectory named newer, these could be merged like so:

$ sqoop merge --new-data newer --onto older --target-dir merged \
    --jar-file datatypes.jar --class-name Foo --merge-key id

This would run a MapReduce job where the value in the id columnof each row is used to join rows; rows in thenewer dataset willbe used in preference to rows in theolder dataset.

This can be used with both SequenceFile-, Avro- and text-basedincremental imports. The file types of the newer and older datasetsmust be the same.

16. sqoop-codegen

16.1. Purpose

The codegen tool generates Java classes which encapsulate andinterpret imported records. The Java definition of a record isinstantiated as part of the import process, but can also be performedseparately. For example, if Java source is lost, it can be recreated.New versions of a class can be created which use different delimitersbetween fields, and so on.

16.2. Syntax

$ sqoop codegen (generic-args) (codegen-args)
$ sqoop-codegen (generic-args) (codegen-args)

Although the Hadoop generic arguments must preceed any codegen arguments,the codegen arguments can be entered in any order with respect to oneanother.

Table 38. Common arguments

ArgumentDescription
--connect <jdbc-uri>Specify JDBC connect string
--connection-manager <class-name>Specify connection manager class to use
--driver <class-name>Manually specify JDBC driver class to use
--hadoop-mapred-home <dir>Override $HADOOP_MAPRED_HOME
--helpPrint usage instructions
--password-fileSet path for a file containing the authentication password
-PRead password from console
--password <password>Set authentication password
--username <username>Set authentication username
--verbosePrint more information while working
--connection-param-file <filename>Optional properties file that provides connection parameters
--relaxed-isolationSet connection transaction isolation to read uncommitted for the mappers.

Table 39. Code generation arguments:

ArgumentDescription
--bindir <dir>Output directory for compiled objects
--class-name <name>Sets the generated class name. This overrides--package-name. When combined with --jar-file, sets the input class.
--jar-file <file>Disable code generation; use specified jar
--outdir <dir>Output directory for generated code
--package-name <name>Put auto-generated classes in this package
--map-column-java <m>Override default mapping from SQL type to Java type for configured columns.

Table 40. Output line formatting arguments:

ArgumentDescription
--enclosed-by <char>Sets a required field enclosing character
--escaped-by <char>Sets the escape character
--fields-terminated-by <char>Sets the field separator character
--lines-terminated-by <char>Sets the end-of-line character
--mysql-delimitersUses MySQL’s default delimiter set: fields:, lines: \n escaped-by:\ optionally-enclosed-by: '
--optionally-enclosed-by <char>Sets a field enclosing character

Table 41. Input parsing arguments:

ArgumentDescription
--input-enclosed-by <char>Sets a required field encloser
--input-escaped-by <char>Sets the input escape character
--input-fields-terminated-by <char>Sets the input field separator
--input-lines-terminated-by <char>Sets the input end-of-line character
--input-optionally-enclosed-by <char>Sets a field enclosing character

Table 42. Hive arguments:

ArgumentDescription
--hive-home <dir>Override $HIVE_HOME
--hive-importImport tables into Hive (Uses Hive’s default delimiters if none are set.)
--hive-overwriteOverwrite existing data in the Hive table.
--create-hive-tableIf set, then the job will fail if the target hive
 table exits. By default this property is false.
--hive-table <table-name>Sets the table name to use when importing to Hive.
--hive-drop-import-delimsDrops \n,\r, and \01 from string fields when importing to Hive.
--hive-delims-replacementReplace \n, \r, and \01 from string fields with user defined string when importing to Hive.
--hive-partition-keyName of a hive field to partition are sharded on
--hive-partition-value <v>String-value that serves as partition key for this imported into hive in this job.
--map-column-hive <map>Override default mapping from SQL type to Hive type for configured columns.

If Hive arguments are provided to the code generation tool, Sqoopgenerates a file containing the HQL statements to create a table andload data.

16.3. Example Invocations

Recreate the record interpretation code for the employees table of acorporate database:

$ sqoop codegen --connect jdbc:mysql://db.example.com/corp \
    --table employees

17. sqoop-create-hive-table

17.1. Purpose

The create-hive-table tool populates a Hive metastore with adefinition for a table based on a database table previously importedto HDFS, or one planned to be imported. This effectively performs the"--hive-import" step of sqoop-import without running thepreceeding import.

If data was already loaded to HDFS, you can use this tool to finishthe pipeline of importing the data to Hive. You can also create Hive tableswith this tool; data then can be imported and populated intothe target after a preprocessing step run by the user.

17.2. Syntax

$ sqoop create-hive-table (generic-args) (create-hive-table-args)
$ sqoop-create-hive-table (generic-args) (create-hive-table-args)

Although the Hadoop generic arguments must preceed any create-hive-tablearguments, the create-hive-table arguments can be entered in any orderwith respect to one another.

Table 43. Common arguments

ArgumentDescription
--connect <jdbc-uri>Specify JDBC connect string
--connection-manager <class-name>Specify connection manager class to use
--driver <class-name>Manually specify JDBC driver class to use
--hadoop-mapred-home <dir>Override $HADOOP_MAPRED_HOME
--helpPrint usage instructions
--password-fileSet path for a file containing the authentication password
-PRead password from console
--password <password>Set authentication password
--username <username>Set authentication username
--verbosePrint more information while working
--connection-param-file <filename>Optional properties file that provides connection parameters
--relaxed-isolationSet connection transaction isolation to read uncommitted for the mappers.

Table 44. Hive arguments:

ArgumentDescription
--hive-home <dir>Override $HIVE_HOME
--hive-overwriteOverwrite existing data in the Hive table.
--create-hive-tableIf set, then the job will fail if the target hive
 table exits. By default this property is false.
--hive-table <table-name>Sets the table name to use when importing to Hive.
--tableThe database table to read the definition from.

Table 45. Output line formatting arguments:

ArgumentDescription
--enclosed-by <char>Sets a required field enclosing character
--escaped-by <char>Sets the escape character
--fields-terminated-by <char>Sets the field separator character
--lines-terminated-by <char>Sets the end-of-line character
--mysql-delimitersUses MySQL’s default delimiter set: fields:, lines: \n escaped-by:\ optionally-enclosed-by: '
--optionally-enclosed-by <char>Sets a field enclosing character

Do not use enclosed-by or escaped-by delimiters with output formattingarguments used to import to Hive. Hive cannot currently parse them.

17.3. Example Invocations

Define in Hive a table named emps with a definition based on adatabase table namedemployees:

$ sqoop create-hive-table --connect jdbc:mysql://db.example.com/corp \
    --table employees --hive-table emps

18. sqoop-eval

18.1. Purpose

The eval tool allows users to quickly run simple SQL queries againsta database; results are printed to the console. This allows users topreview their import queries to ensure they import the data theyexpect.

[Warning]Warning

The eval tool is provided for evaluation purpose only. You can use it to verify database connection from within the Sqoop or to test simple queries. It’s not suppose to be used in production workflows.

18.2. Syntax

$ sqoop eval (generic-args) (eval-args)
$ sqoop-eval (generic-args) (eval-args)

Although the Hadoop generic arguments must preceed any eval arguments,the eval arguments can be entered in any order with respect to oneanother.

Table 46. Common arguments

ArgumentDescription
--connect <jdbc-uri>Specify JDBC connect string
--connection-manager <class-name>Specify connection manager class to use
--driver <class-name>Manually specify JDBC driver class to use
--hadoop-mapred-home <dir>Override $HADOOP_MAPRED_HOME
--helpPrint usage instructions
--password-fileSet path for a file containing the authentication password
-PRead password from console
--password <password>Set authentication password
--username <username>Set authentication username
--verbosePrint more information while working
--connection-param-file <filename>Optional properties file that provides connection parameters
--relaxed-isolationSet connection transaction isolation to read uncommitted for the mappers.

Table 47. SQL evaluation arguments:

ArgumentDescription
-e,--query <statement>Execute statement in SQL.

18.3. Example Invocations

Select ten records from the employees table:

$ sqoop eval --connect jdbc:mysql://db.example.com/corp \
    --query "SELECT * FROM employees LIMIT 10"

Insert a row into the foo table:

$ sqoop eval --connect jdbc:mysql://db.example.com/corp \
    -e "INSERT INTO foo VALUES(42, 'bar')"

19. sqoop-list-databases

19.1. Purpose

List database schemas present on a server.

19.2. Syntax

$ sqoop list-databases (generic-args) (list-databases-args)
$ sqoop-list-databases (generic-args) (list-databases-args)

Although the Hadoop generic arguments must preceed any list-databasesarguments, the list-databases arguments can be entered in any orderwith respect to one another.

Table 48. Common arguments

ArgumentDescription
--connect <jdbc-uri>Specify JDBC connect string
--connection-manager <class-name>Specify connection manager class to use
--driver <class-name>Manually specify JDBC driver class to use
--hadoop-mapred-home <dir>Override $HADOOP_MAPRED_HOME
--helpPrint usage instructions
--password-fileSet path for a file containing the authentication password
-PRead password from console
--password <password>Set authentication password
--username <username>Set authentication username
--verbosePrint more information while working
--connection-param-file <filename>Optional properties file that provides connection parameters
--relaxed-isolationSet connection transaction isolation to read uncommitted for the mappers.

19.3. Example Invocations

List database schemas available on a MySQL server:

$ sqoop list-databases --connect jdbc:mysql://database.example.com/
information_schema
employees
[Note]Note

This only works with HSQLDB, MySQL and Oracle. When using with Oracle,it is necessary that the user connecting to the database has DBA privileges.

20. sqoop-list-tables

20.1. Purpose

List tables present in a database.

20.2. Syntax

$ sqoop list-tables (generic-args) (list-tables-args)
$ sqoop-list-tables (generic-args) (list-tables-args)

Although the Hadoop generic arguments must preceed any list-tablesarguments, the list-tables arguments can be entered in any orderwith respect to one another.

Table 49. Common arguments

ArgumentDescription
--connect <jdbc-uri>Specify JDBC connect string
--connection-manager <class-name>Specify connection manager class to use
--driver <class-name>Manually specify JDBC driver class to use
--hadoop-mapred-home <dir>Override $HADOOP_MAPRED_HOME
--helpPrint usage instructions
--password-fileSet path for a file containing the authentication password
-PRead password from console
--password <password>Set authentication password
--username <username>Set authentication username
--verbosePrint more information while working
--connection-param-file <filename>Optional properties file that provides connection parameters
--relaxed-isolationSet connection transaction isolation to read uncommitted for the mappers.

20.3. Example Invocations

List tables available in the "corp" database:

$ sqoop list-tables --connect jdbc:mysql://database.example.com/corp
employees
payroll_checks
job_descriptions
office_supplies

In case of postgresql, list tables command with common arguments fetches only "public" schema. For custom schema, use --schema argument to list tables of particular schemaExample

$ sqoop list-tables --connect jdbc:postgresql://localhost/corp  --username name -P -- --schema payrolldept
employees
expenses

21. sqoop-help

21.1. Purpose

List tools available in Sqoop and explain their usage.

21.2. Syntax

$ sqoop help [tool-name]
$ sqoop-help [tool-name]

If no tool name is provided (for example, the user runs sqoop help), thenthe available tools are listed. With a tool name, the usageinstructions for that specific tool are presented on the console.

21.3. Example Invocations

List available tools:

$ sqoop help
usage: sqoop COMMAND [ARGS]

Available commands:
  codegen            Generate code to interact with database records
  create-hive-table  Import a table definition into Hive
  eval               Evaluate a SQL statement and display the results
  export             Export an HDFS directory to a database table

...

See 'sqoop help COMMAND' for information on a specific command.

Display usage instructions for the import tool:

$ bin/sqoop help import
usage: sqoop import [GENERIC-ARGS] [TOOL-ARGS]

Common arguments:
   --connect <jdbc-uri>     Specify JDBC connect string
   --connection-manager <class-name>     Specify connection manager class to use
   --driver <class-name>    Manually specify JDBC driver class to use
   --hadoop-mapred-home <dir>            Override $HADOOP_MAPRED_HOME
   --help                   Print usage instructions
   --password-file          Set path for file containing authentication password
   -P                       Read password from console
   --password <password>    Set authentication password
   --username <username>    Set authentication username
   --verbose                Print more information while working
   --hadoop-home <dir>      Deprecated. Override $HADOOP_HOME

Import control arguments:
   --as-avrodatafile             Imports data to Avro Data Files
   --as-sequencefile             Imports data to SequenceFiles
   --as-textfile                 Imports data as plain text (default)
   --as-parquetfile              Imports data to Parquet Data Files
...

22. sqoop-version

22.1. Purpose

Display version information for Sqoop.

22.2. Syntax

$ sqoop version
$ sqoop-version

22.3. Example Invocations

Display the version:

$ sqoop version
Sqoop {revnumber}
git commit id 46b3e06b79a8411320d77c984c3030db47dd1c22
Compiled by aaron@jargon on Mon May 17 13:43:22 PDT 2010

23. Sqoop-HCatalog Integration

23.1. HCatalog Background

HCatalog is a table and storage management service for Hadoop that enablesusers with different data processing tools Pig, MapReduce, and Hiveto more easily read and write data on the grid. HCatalog’s table abstractionpresents users with a relational view of data in the Hadoop distributedfile system (HDFS) and ensures that users need not worry about where orin what format their data is stored: RCFile format, text files, orSequenceFiles.

HCatalog supports reading and writing files in any format for which a HiveSerDe (serializer-deserializer) has been written. By default, HCatalogsupports RCFile, CSV, JSON, and SequenceFile formats. To use a customformat, you must provide the InputFormat and OutputFormat as well as the SerDe.

The ability of HCatalog to abstract various storage formats is used inproviding the RCFile (and future file types) support to Sqoop.

23.2. Exposing HCatalog Tables to Sqoop

HCatalog integration with Sqoop is patterned on an existing feature set thatsupports Avro and Hive tables. Seven new command line options are introduced,and some command line options defined for Hive have been reused.

23.2.1. New Command Line Options
--hcatalog-database
Specifies the database name for the HCatalog table. If not specified,the default database name default is used. Providing the --hcatalog-database option without --hcatalog-table is an error.This is not a required option.
--hcatalog-table
The argument value for this option is the HCatalog tablename.The presence of the --hcatalog-table option signifies that the importor export job is done using HCatalog tables, and it is a required option forHCatalog jobs.
--hcatalog-home
The home directory for the HCatalog installation. The directory isexpected to have a lib subdirectory and a share/hcatalog subdirectorywith necessary HCatalog libraries. If not specified, the system property hcatalog.home will be checked and failing that, a system environmentvariable HCAT_HOME will be checked. If none of these are set, thedefault value will be used and currently the default is set to /usr/lib/hcatalog.This is not a required option.
--create-hcatalog-table
This option specifies whether an HCatalog table should be createdautomatically when importing data. By default, HCatalog tables are assumedto exist. The table name will be the same as the database table nametranslated to lower case. Further described in Automatic Table Creationbelow.
--hcatalog-storage-stanza
This option specifies the storage stanza to be appended to the table.Further described in Automatic Table Creation below.
--hcatalog-partition-keys and--hcatalog-partition-values
These two options are used to specify multiple static partition key/valuepairs. In the prior releases, --hive-partition-key and --hive-partition-value options were used to specify the static partitionkey/value pair, but only one level of static partition keys could be provided.The options --hcatalog-partition-keys and --hcatalog-partition-valuesallow multiple keys and values to be provided as static partitioning keys.Multiple option values are to be separated by , (comma).

For example, if the hive partition keys for the table to export/import from aredefined with partition key names year, month and date and a specific partitionwith year=1999, month=12, day=31 is the desired partition, then the valuesfor the two options will be as follows:

  • --hcatalog-partition-keys year,month,day
  • --hcatalog-partition-values 1999,12,31

To provide backward compatibility, if --hcatalog-partition-keys or--hcatalog-partition-values options are not provided, then--hive-partitition-key and--hive-partition-value will be used if provided.

It is an error to specify only one of --hcatalog-partition-keys or--hcatalog-partition-values options. Either both of the options should beprovided or neither of the options should be provided.

23.2.2. Supported Sqoop Hive Options

The following Sqoop options are also used along with the --hcatalog-tableoption to provide additional input to the HCatalog jobs. Some of the existingHive import job options are reused with HCatalog jobs instead of creatingHCatalog-specific options for the same purpose.

--map-column-hive
This option maps a database column to HCatalog with a specific HCatalogtype.
--hive-home
The Hive home location.
--hive-partition-key
Used for static partitioning filter. The partitioning key should be oftype STRING. There can be only one static partitioning key.Please see the discussion about --hcatalog-partition-keys and --hcatalog-partition-values options.
--hive-partition-value
The value associated with the partition.Please see the discussion about --hcatalog-partition-keys and --hcatalog-partition-values options.
23.2.3. Direct Mode support

HCatalog integration in Sqoop has been enhanced to support direct modeconnectors (which are high performance connectors specific to a database).Netezza direct mode connector has been enhanced to take advatange of thisfeature.

[Important]Important

Only Netezza direct mode connector is currently enabled to workwith HCatalog.

23.2.4. Unsupported Sqoop Options
23.2.4.1. Unsupported Sqoop Hive Import Options

The following Sqoop Hive import options are not supported with HCatalog jobs.

  • --hive-import
  • --hive-overwrite
23.2.4.2. Unsupported Sqoop Export and Import Options

The following Sqoop export and import options are not supported with HCatalog jobs.

  • --export-dir
  • --target-dir
  • --warehouse-dir
  • --append
  • --as-sequencefile
  • --as-avrodatafile
  • --as-parquetfile
23.2.5. Ignored Sqoop Options

The following options are ignored with HCatalog jobs.

  • All input delimiter options are ignored.
  • Output delimiters are generally ignored unless either--hive-drop-import-delims or--hive-delims-replacement is used. When the--hive-drop-import-delims or--hive-delims-replacement option isspecified, all CHAR type database table columns will be post-processedto either remove or replace the delimiters, respectively. SeeDelimited TextFormats and Field and Line Delimiter Characters below. This is only neededif the HCatalog table uses text formats.

23.3. Automatic Table Creation

One of the key features of Sqoop is to manage and create the table metadatawhen importing into Hadoop. HCatalog import jobs also provide for thisfeature with the option--create-hcatalog-table. Furthermore, one of theimportant benefits of the HCatalog integration is to provide storageagnosticism to Sqoop data movement jobs. To provide for that feature,HCatalog import jobs provide an option that lets a user specifiy thestorage format for the created table.

The option --create-hcatalog-table is used as an indicator that a tablehas to be created as part of the HCatalog import job. If the option--create-hcatalog-table is specified and the table exists, then thetable creation will fail and the job will be aborted.

The option --hcatalog-storage-stanza can be used to specify the storageformat of the newly created table. The default value for this option isstored as rcfile. The value specified for this option is assumed to be avalid Hive storage format expression. It will be appended to thecreate tablecommand generated by the HCatalog import job as part of automatic tablecreation. Any error in the storage stanza will cause the table creation tofail and the import job will be aborted.

Any additional resources needed to support the storage format referenced inthe option--hcatalog-storage-stanza should be provided to the job eitherby placing them in$HIVE_HOME/lib or by providing them in HADOOP_CLASSPATHand LIBJAR files.

If the option --hive-partition-key is specified, then the value of thisoption is used as the partitioning key for the newly created table. Onlyone partitioning key can be specified with this option.

Object names are mapped to the lowercase equivalents as specified belowwhen mapped to an HCatalog table. This includes the table name (whichis the same as the external store table name converted to lower case)and field names.

23.4. Delimited Text Formats and Field and Line Delimiter Characters

HCatalog supports delimited text format as one of the table storage formats.But when delimited text is used and the imported data has fields that containthose delimiters, then the data may be parsed into a different number offields and records by Hive, thereby losing data fidelity.

For this case, one of these existing Sqoop import options can be used:

  • --hive-delims-replacement
  • --hive-drop-import-delims

If either of these options is provided for import, then any column of typeSTRING will be formatted with the Hive delimiter processing and then writtento the HCatalog table.

23.5. HCatalog Table Requirements

The HCatalog table should be created before using it as part of a Sqoop jobif the default table creation options (with optional storage stanza) are notsufficient. All storage formats supported by HCatalog can be used with thecreation of the HCatalog tables. This makes this feature readily adopt newstorage formats that come into the Hive project, such as ORC files.

23.6. Support for Partitioning

The Sqoop HCatalog feature supports the following table types:

  • Unpartitioned tables
  • Partitioned tables with a static partitioning key specified
  • Partitioned tables with dynamic partition keys from the databaseresult set
  • Partitioned tables with a combination of a static key and additionaldynamic partitioning keys

23.7. Schema Mapping

Sqoop currently does not support column name mapping. However, the useris allowed to override the type mapping. Type mapping loosely followsthe Hive type mapping already present in Sqoop except that SQL typesFLOAT and REAL are mapped to HCatalog type float. In the Sqoop typemapping for Hive, these two are mapped to double. Type mapping is primarilyused for checking the column definition correctness only and can be overriddenwith the --map-column-hive option.

All types except binary are assignable to a String type.

Any field of number type (int, shortint, tinyint, bigint and bigdecimal,float and double) is assignable to another field of any number type duringexports and imports. Depending on the precision and scale of the target typeof assignment, truncations can occur.

Furthermore, date/time/timestamps are mapped to date/timestamp hive types.(the full date/time/timestamp representation). Date/time/timstamp columnscan also be mapped to bigint Hive type in which case the value will bethe number of milliseconds since epoch.

BLOBs and CLOBs are only supported for imports. The BLOB/CLOB objects whenimported are stored in a Sqoop-specific format and knowledge of this formatis needed for processing these objects in a Pig/Hive job or another Map Reducejob.

Database column names are mapped to their lowercase equivalents when mappedto the HCatalog fields. Currently, case-sensitive database object names arenot supported.

Projection of a set of columns from a table to an HCatalog table or loadingto a column projection is allowed, subject to table constraints. The dynamicpartitioning columns, if any, must be part of the projection when importingdata into HCatalog tables.

Dynamic partitioning fields should be mapped to database columns that aredefined with the NOT NULL attribute (although this is not enforced duringschema mapping). A null value during import for a dynamic partitioningcolumn will abort the Sqoop job.

23.8. Support for HCatalog Data Types

All the primitive Hive types that are part of Hive 0.13 version are supported.Currently all the complex HCatalog types are not supported.

BLOB/CLOB database types are only supported for imports.

23.9. Providing Hive and HCatalog Libraries for the Sqoop Job

With the support for HCatalog added to Sqoop, any HCatalog job depends on aset of jar files being available both on the Sqoop client host and where theMap/Reduce tasks run. To run HCatalog jobs, the environment variableHADOOP_CLASSPATH must be set up as shown below before launching the SqoopHCatalog jobs.

HADOOP_CLASSPATH=$(hcat -classpath)export HADOOP_CLASSPATH

The necessary HCatalog dependencies will be copied to the distributed cacheautomatically by the Sqoop job.

23.10. Examples

Create an HCatalog table, such as:

hcat -e "create table txn(txn_date string, cust_id string, amount float,store_id int) partitioned by (cust_id string) stored as rcfile;"

Then Sqoop import and export of the "txn" HCatalog table can be invoked asfollows:

23.11. Import

$SQOOP_HOME/bin/sqoop import --connect <jdbc-url> -table <table-name> --hcatalog-table txn <other sqoop options>

23.12. Export

$SQOOP_HOME/bin/sqoop export --connect <jdbc-url> -table <table-name> --hcatalog-table txn <other sqoop options>

24. Compatibility Notes

Sqoop uses JDBC to connect to databases and adheres topublished standards as much as possible. For databases which do notsupport standards-compliant SQL, Sqoop uses alternate codepaths toprovide functionality. In general, Sqoop is believed to be compatiblewith a large number of databases, but it is tested with only a few.

Nonetheless, several database-specific decisions were made in theimplementation of Sqoop, and some databases offer additional settingswhich are extensions to the standard.

This section describes the databases tested with Sqoop, anyexceptions in Sqoop’s handling of each database relative to thenorm, and any database-specific settings available in Sqoop.

24.1. Supported Databases

While JDBC is a compatibility layer that allows a program to accessmany different databases through a common API, slight differences inthe SQL language spoken by each database may mean that Sqoop can’t useevery database out of the box, or that some databases may be used inan inefficient manner.

When you provide a connect string to Sqoop, it inspects the protocol scheme todetermine appropriate vendor-specific logic to use. If Sqoop knows abouta given database, it will work automatically. If not, you may need tospecify the driver class to load via--driver. This will use a genericcode path which will use standard SQL to access the database. Sqoop providessome databases with faster, non-JDBC-based access mechanisms. These can beenabled by specfying the--direct parameter.

Sqoop includes vendor-specific support for the following databases:

Databaseversion--direct support?connect string matches
HSQLDB1.8.0+Nojdbc:hsqldb:*//
MySQL5.0+Yesjdbc:mysql://
Oracle10.2.0+Nojdbc:oracle:*//
PostgreSQL8.3+Yes (import only)jdbc:postgresql://
CUBRID9.2+NOjdbc:cubrid:*

Sqoop may work with older versions of the databases listed, but we haveonly tested it with the versions specified above.

Even if Sqoop supports a database internally, you may still need toinstall the database vendor’s JDBC driver in your$SQOOP_HOME/libpath on your client. Sqoop can load classes from any jars in$SQOOP_HOME/lib on the client and will use them as part of anyMapReduce jobs it runs; unlike older versions, you no longer need toinstall JDBC jars in the Hadoop library path on your servers.

24.2. MySQL

JDBC Driver: MySQLConnector/J

MySQL v5.0 and above offers very thorough coverage by Sqoop. Sqoophas been tested withmysql-connector-java-5.1.13-bin.jar.

24.2.1. zeroDateTimeBehavior

MySQL allows values of '0000-00-00\' for DATE columns, which is anon-standard extension to SQL. When communicated via JDBC, thesevalues are handled in one of three different ways:

  • Convert to NULL.
  • Throw an exception in the client.
  • Round to the nearest legal date ('0001-01-01\').

You specify the behavior by using the zeroDateTimeBehaviorproperty of the connect string. If azeroDateTimeBehavior propertyis not specified, Sqoop uses theconvertToNull behavior.

You can override this behavior. For example:

$ sqoop import --table foo \
    --connect jdbc:mysql://db.example.com/someDb?zeroDateTimeBehavior=round
24.2.2. UNSIGNED columns

Columns with type UNSIGNED in MySQL can hold values between 0 and2^32 (4294967295), but the database will report the data type to SqoopasINTEGER, which will can hold values between -2147483648 and\+2147483647. Sqoop cannot currently importUNSIGNED values above2147483647.

24.2.3. BLOB andCLOB columns

Sqoop’s direct mode does not support imports of BLOB,CLOB, orLONGVARBINARY columns. Use JDBC-based imports for thesecolumns; do not supply the--direct argument to the import tool.

24.2.4. Importing views in direct mode

Sqoop is currently not supporting import from view in direct mode. UseJDBC based (non direct) mode in case that you need to import view (simplyomit--direct parameter).

24.3. PostgreSQL

Sqoop supports JDBC-based connector for PostgreSQL: http://jdbc.postgresql.org/

The connector has been tested using JDBC driver version "9.1-903 JDBC 4" withPostgreSQL server 9.1.

24.3.1. Importing views in direct mode

Sqoop is currently not supporting import from view in direct mode. UseJDBC based (non direct) mode in case that you need to import view (simplyomit--direct parameter).

24.4. Oracle

JDBC Driver:OracleJDBC Thin Driver - Sqoop is compatible withojdbc6.jar.

Sqoop has been tested with Oracle 10.2.0 Express Edition. Oracle isnotable in its different approach to SQL from the ANSI standard, andits non-standard JDBC driver. Therefore, several features workdifferently.

24.4.1. Dates and Times

Oracle JDBC represents DATE and TIME SQL types as TIMESTAMPvalues. Any DATE columns in an Oracle database will be imported as aTIMESTAMP in Sqoop, and Sqoop-generated code will store these valuesinjava.sql.Timestamp fields.

When exporting data back to a database, Sqoop parses text fields asTIMESTAMP types (with the formyyyy-mm-dd HH:MM:SS.ffffffff) evenif you expect these fields to be formatted with the JDBC date escapeformat ofyyyy-mm-dd. Dates exported to Oracle should be formattedas full timestamps.

Oracle also includes the additional date/time types TIMESTAMP WITHTIMEZONE andTIMESTAMP WITH LOCAL TIMEZONE. To support these types,the user’s session timezone must be specified. By default, Sqoop willspecify the timezone"GMT" to Oracle. You can override this settingby specifying a Hadoop propertyoracle.sessionTimeZone on thecommand-line when running a Sqoop job. For example:

$ sqoop import -D oracle.sessionTimeZone=America/Los_Angeles \
    --connect jdbc:oracle:thin:@//db.example.com/foo --table bar

Note that Hadoop parameters (-D …) are generic arguments andmust appear before the tool-specific arguments (--connect,--table, and so on).

Legal values for the session timezone string are enumerated athttp://download-west.oracle.com/docs/cd/B19306_01/server.102/b14225/applocaledata.htm#i637736.

24.5. Schema Definition in Hive

Hive users will note that there is not a one-to-one mapping betweenSQL types and Hive types. In general, SQL types that do not have adirect mapping (for example,DATE, TIME, and TIMESTAMP) will be coerced toSTRING in Hive. TheNUMERIC and DECIMAL SQL types will be coerced toDOUBLE. In these cases, Sqoop will emit a warning in its log messagesinforming you of the loss of precision.

24.6. CUBRID

Sqoop supports JDBC-based connector for Cubrid: http://www.cubrid.org/?mid=downloads&item=jdbc_driver

The connector has been tested using JDBC driver version "JDBC-9.2.0.0155-cubrid.jar" with Cubrid 9.2.

25. Notes for specific connectors

25.1. MySQL JDBC Connector
25.1.1. Upsert functionality
25.2. MySQL Direct Connector
25.2.1. Requirements 25.2.2. Limitations 25.2.3. Direct-mode Transactions
25.3. Microsoft SQL Connector
25.3.1. Extra arguments 25.3.2. Allow identity inserts 25.3.3. Non-resilient operations 25.3.4. Schema support 25.3.5. Table hints
25.4. PostgreSQL Connector
25.4.1. Extra arguments 25.4.2. Schema support
25.5. PostgreSQL Direct Connector
25.5.1. Requirements 25.5.2. Limitations
25.6. pg_bulkload connector
25.6.1. Purpose 25.6.2. Requirements 25.6.3. Syntax 25.6.4. Data Staging
25.7. Netezza Connector
25.7.1. Extra arguments 25.7.2. Direct Mode 25.7.3. Null string handling
25.8. Data Connector for Oracle and Hadoop
25.8.1. About
25.8.1.1. Jobs 25.8.1.2. How The Standard Oracle Manager Works for Imports 25.8.1.3. How The Data Connector for Oracle and Hadoop Works for Imports 25.8.1.4. Data Connector for Oracle and Hadoop Exports
25.8.2. Requirements
25.8.2.1. Ensure The Oracle Database JDBC Driver Is Setup Correctly 25.8.2.2. Oracle Roles and Privileges 25.8.2.3. Additional Oracle Roles And Privileges Required for Export 25.8.2.4. Supported Data Types
25.8.3. Execute Sqoop With Data Connector for Oracle and Hadoop
25.8.3.1. Connect to Oracle / Oracle RAC 25.8.3.2. Connect to An Oracle Database Instance 25.8.3.3. Connect to An Oracle RAC 25.8.3.4. Login to The Oracle Instance 25.8.3.5. Kill Data Connector for Oracle and Hadoop Jobs
25.8.4. Import Data from Oracle
25.8.4.1. Match Hadoop Files to Oracle Table Partitions 25.8.4.2. Specify The Partitions To Import 25.8.4.3. Consistent Read: All Mappers Read From The Same Point In Time
25.8.5. Export Data into Oracle
25.8.5.1. Insert-Export 25.8.5.2. Update-Export 25.8.5.3. Merge-Export 25.8.5.4. Create Oracle Tables 25.8.5.5. NOLOGGING 25.8.5.6. Partitioning 25.8.5.7. Match Rows Via Multiple Columns 25.8.5.8. Storage Clauses
25.8.6. Manage Date And Timestamp Data Types
25.8.6.1. Import Date And Timestamp Data Types from Oracle 25.8.6.2. The Data Connector for Oracle and Hadoop Does Not Apply A Time Zone to DATE / TIMESTAMP Data Types 25.8.6.3. The Data Connector for Oracle and Hadoop Retains Time Zone Information in TIMEZONE Data Types 25.8.6.4. Data Connector for Oracle and Hadoop Explicitly States Time Zone for LOCAL TIMEZONE Data Types 25.8.6.5. java.sql.Timestamp 25.8.6.6. Export Date And Timestamp Data Types into Oracle
25.8.7. Configure The Data Connector for Oracle and Hadoop
25.8.7.1. oraoop-site-template.xml 25.8.7.2. oraoop.oracle.session.initialization.statements 25.8.7.3. oraoop.table.import.where.clause.location 25.8.7.4. oracle.row.fetch.size 25.8.7.5. oraoop.import.hint 25.8.7.6. oraoop.oracle.append.values.hint.usage 25.8.7.7. mapred.map.tasks.speculative.execution 25.8.7.8. oraoop.block.allocation 25.8.7.9. oraoop.import.omit.lobs.and.long 25.8.7.10. oraoop.locations 25.8.7.11. sqoop.connection.factories 25.8.7.12. Expressions in oraoop-site.xml
25.8.8. Troubleshooting The Data Connector for Oracle and Hadoop
25.8.8.1. Quote Oracle Owners And Tables 25.8.8.2. Quote Oracle Columns 25.8.8.3. Confirm The Data Connector for Oracle and Hadoop Can Initialize The Oracle Session 25.8.8.4. Check The Sqoop Debug Logs for Error Messages 25.8.8.5. Export: Check Tables Are Compatible 25.8.8.6. Export: Parallelization 25.8.8.7. Export: Check oraoop.oracle.append.values.hint.usage 25.8.8.8. Turn On Verbose

25.1. MySQL JDBC Connector

This section contains information specific to MySQL JDBC Connector.

25.1.1. Upsert functionality

MySQL JDBC Connector is supporting upsert functionality using argument--update-mode allowinsert. To achieve that Sqoop is using MySQL clause INSERT INTO… ON DUPLICATE KEY UPDATE. This clause do not allow user to specify which columnsshould be used to distinct whether we should update existing row or add new row. Insteadthis clause relies on table’s unique keys (primary key belongs to this set). MySQLwill try to insert new row and if the insertion fails with duplicate unique key errorit will update appropriate row instead. As a result, Sqoop is ignoring values specifiedin parameter--update-key, however user needs to specify at least one valid columnto turn on update mode itself.

25.2. MySQL Direct Connector

MySQL Direct Connector allows faster import and export to/from MySQL using mysqldump and mysqlimport tools functionalityinstead of SQL selects and inserts.

To use the MySQL Direct Connector, specify the --direct argument for your import or export job.

Example:

$ sqoop import --connect jdbc:mysql://db.foo.com/corp --table EMPLOYEES \
    --direct

Passing additional parameters to mysqldump:

$ sqoop import --connect jdbc:mysql://server.foo.com/db --table bar \
    --direct -- --default-character-set=latin1
25.2.1. Requirements

Utilities mysqldump and mysqlimport should be present in the shell path of the user running the Sqoop command onall nodes. To validate SSH as this user to all nodes and execute these commands. If you get an error, so will Sqoop.

25.2.2. Limitations
  • Currently the direct connector does not support import of large object columns (BLOB and CLOB).
  • Importing to HBase and Accumulo is not supported
  • Use of a staging table when exporting data is not supported
  • Import of views is not supported
25.2.3. Direct-mode Transactions

For performance, each writer will commit the current transactionapproximately every 32 MB of exported data. You can control thisby specifying the following argumentbefore any tool-specific arguments: -Dsqoop.mysql.export.checkpoint.bytes=size, where size is a value inbytes. Setsize to 0 to disable intermediate checkpoints,but individual files being exported will continue to be committedindependently of one another.

Sometimes you need to export large data with Sqoop to a live MySQL cluster thatis under a high load serving random queries from the users of your application.While data consistency issues during the export can be easily solved with astaging table, there is still a problem with the performance impact caused bythe heavy export.

First off, the resources of MySQL dedicated to the import process can affectthe performance of the live product, both on the master and on the slaves.Second, even if the servers can handle the import with no significantperformance impact (mysqlimport should be relatively "cheap"), importing bigtables can cause serious replication lag in the cluster risking datainconsistency.

With -D sqoop.mysql.export.sleep.ms=time, where time is a value inmilliseconds, you can let the server relax between checkpoints and the replicascatch up by pausing the export process after transferring the number of bytesspecified insqoop.mysql.export.checkpoint.bytes. Experiment with differentsettings of these two parameters to archieve an export pace that doesn’tendanger the stability of your MySQL cluster.

[Important]Important

Note that any arguments to Sqoop that are of the form -Dparameter=value are Hadoopgeneric arguments and must appear beforeany tool-specific arguments (for example,--connect, --table, etc).Don’t forget that these parameters are only supported with the--directflag set.

25.3. Microsoft SQL Connector

25.3.1. Extra arguments

List of all extra arguments supported by Microsoft SQL Connector is shown below:

Table 50. Supported Microsoft SQL Connector extra arguments:

ArgumentDescription
+--identity-insertSet IDENTITY_INSERT to ON before export insert.
--non-resilientDon’t attempt to recover failed export operations.
--schema <name>Scheme name that sqoop should use. Default is "dbo".
--table-hints <hints>Table hints that Sqoop should use for data movement.

25.3.2. Allow identity inserts

You can allow inserts on columns that have identity. For example:

$ sqoop export ... --export-dir custom_dir --table custom_table -- --identity-insert
25.3.3. Non-resilient operations

You can override the default and not use resilient operations during export.This will avoid retrying failed operations. For example:

$ sqoop export ... --export-dir custom_dir --table custom_table -- --non-resilient
25.3.4. Schema support

If you need to work with tables that are located in non-default schemas, you canspecify schema names via the--schema argument. Custom schemas are supported forboth import and export jobs. For example:

$ sqoop import ... --table custom_table -- --schema custom_schema
25.3.5. Table hints

Sqoop supports table hints in both import and export jobs. Table hints are used onlyfor queries that move data from/to Microsoft SQL Server, but they cannot be used formeta data queries. You can specify a comma-separated list of table hints in the--table-hints argument. For example:

$ sqoop import ... --table custom_table -- --table-hints NOLOCK

25.4. PostgreSQL Connector

25.4.1. Extra arguments

List of all extra arguments supported by PostgreSQL Connector is shown below:

Table 51. Supported PostgreSQL extra arguments:

ArgumentDescription
--schema <name>Scheme name that sqoop should use. Default is "public".

25.4.2. Schema support

If you need to work with table that is located in schema other than default one,you need to specify extra argument--schema. Custom schemas are supported forboth import and export job (optional staging table however must be present in thesame schema as target table). Example invocation:

$ sqoop import ... --table custom_table -- --schema custom_schema

25.5. PostgreSQL Direct Connector

PostgreSQL Direct Connector allows faster import and export to/from PostgresSQL "COPY" command.

To use the PostgreSQL Direct Connector, specify the --direct argument for your import or export job.

When importing from PostgreSQL in conjunction with direct mode, youcan split the import into separate files afterindividual files reach a certain size. This size limit is controlledwith the--direct-split-size argument.

The direct connector offers also additional extra arguments:

Table 52. Additional supported PostgreSQL extra arguments in direct mode:

ArgumentDescription
--boolean-true-string <str>String that will be used to encodetrue value of boolean columns.
 Default is "TRUE".
--boolean-false-string <str>String that will be used to encodefalse value of boolean columns.
 Default is "FALSE".

25.5.1. Requirements

Utility psql should be present in the shell path of the user running the Sqoop command onall nodes. To validate SSH as this user to all nodes and execute these commands. If you get an error, so will Sqoop.

25.5.2. Limitations
  • Currently the direct connector does not support import of large object columns (BLOB and CLOB).
  • Importing to HBase and Accumulo is not supported
  • Import of views is not supported

25.6. pg_bulkload connector

25.6.1. Purpose

pg_bulkload connector is a direct connector for exporting data into PostgreSQL.This connector usespg_bulkload.Users benefit from functionality of pg_bulkload such asfast exports bypassing shared bufferes and WAL,flexible error records handling,and ETL feature with filter functions.

25.6.2. Requirements

pg_bulkload connector requires following conditions for export job execution:

  • The pg_bulkload must be installed on DB server and all slave nodes. RPM for RedHat or CentOS is available in thendownload page.
  • The PostgreSQL JDBC is required on client node.
  • Superuser role of PostgreSQL database is required for execution of pg_bulkload.
25.6.3. Syntax

Use --connection-manager option to specify connection manager classname.

$ sqoop export (generic-args) --connection-manager org.apache.sqoop.manager.PGBulkloadManager (export-args)
$ sqoop-export (generic-args) --connection-manager org.apache.sqoop.manager.PGBulkloadManager (export-args)

This connector supports export arguments shown below.

Table 53. Supported export control arguments:

ArgumentDescription
--export-dir <dir>HDFS source path for the export
-m,--num-mappers <n>Use n map tasks to export in parallel
--table <table-name>Table to populate
--input-null-string <null-string>The string to be interpreted as null for string columns

There are additional configuration for pg_bulkload executionspecified via Hadoop Configuration propertieswhich can be given with-D <property=value> option.Because Hadoop Configuration properties are generic arguments of the sqoop,it must preceed any export control arguments.

Table 54. Supported export control properties:

PropertyDescription
mapred.reduce.tasksNumber of reduce tasks for staging. The defalt value is 1. Each tasks do staging in a single transaction.
pgbulkload.binPath of the pg_bulkoad binary installed on each slave nodes.
pgbulkload.check.constraintsSpecify whether CHECK constraints are checked during the loading. The default value is YES.
pgbulkload.parse.errorsThe maximum mumber of ingored records that cause errors during parsing, encoding, filtering, constraints checking, and data type conversion. Error records are recorded in the PARSE BADFILE. The default value is INFINITE.
pgbulkload.duplicate.errorsNumber of ingored records that violate unique constraints. Duplicated records are recorded in the DUPLICATE BADFILE on DB server. The default value is INFINITE.
pgbulkload.filterSpecify the filter function to convert each row in the input file. See the pg_bulkload documentation to know how to write FILTER functions.
pgbulkload.clear.staging.tableIndicates that any data present in the staging table can be dropped.

Here is a example of complete command line.

$ sqoop export \
    -Dmapred.reduce.tasks=2
    -Dpgbulkload.bin="/usr/local/bin/pg_bulkload" \
    -Dpgbulkload.input.field.delim=$'\t' \
    -Dpgbulkload.check.constraints="YES" \
    -Dpgbulkload.parse.errors="INFINITE" \
    -Dpgbulkload.duplicate.errors="INFINITE" \
    --connect jdbc:postgresql://pgsql.example.net:5432/sqooptest \
    --connection-manager org.apache.sqoop.manager.PGBulkloadManager \
    --table test --username sqooptest --export-dir=/test -m 2
25.6.4. Data Staging

Each map tasks of pg_bulkload connector’s export job createtheir own staging table on the fly.The Name of staging tables is decided based on the destination tableand the task attempt ids.For example, the name of staging table for the "test" table is liketest_attempt_1345021837431_0001_m_000000_0 .

Staging tables are automatically dropped if tasks successfully completeor map tasks fail.When reduce task fails,staging table for the task are left for manual retry andusers must take care of it.

25.7. Netezza Connector

25.7.1. Extra arguments

List of all extra arguments supported by Netezza Connector is shown below:

Table 55. Supported Netezza extra arguments:

ArgumentDescription
--partitioned-accessWhether each mapper acts on a subset of data slices of a table or all Default is "false" for standard mode and "true" for direct mode.
--max-errorsApplicable only in direct mode. This option specifies the error threshold per mapper while transferring data. If the number of errors encountered exceed this threshold then the job will fail.
 Default value is 1.
--log-dirApplicable only in direct mode. Specifies the directory where Netezza external table operation logs are stored on the hadoop filesystem. Logs are stored under this directory with one directory for the job and sub-directories for each task number and attempt. Default value is the user home directory.
--trunc-stringApplicable only in direct mode. Specifies whether the system truncates strings to the declared storage and loads the data. By default truncation of strings is reported as an error.
--ctrl-charsApplicable only in direct mode. Specifies whether control characters (ASCII chars 1 - 31) can be allowed to be part of char/nchar/varchar/nvarchar columns. Default is false.

25.7.2. Direct Mode

Netezza connector supports an optimized data transfer facility using theNetezza external tables feature. Each map tasks of Netezza connector’s importjob will work on a subset of the Netezza partitions and transparently createand use an external table to transport data. Similarly, export jobs will usethe external table to push data fast onto the NZ system. Direct mode doesnot support staging tables, upsert options etc.

Here is an example of complete command line for import using the Netezzaexternal table feature.

$ sqoop import \
    --direct \
    --connect jdbc:netezza://nzhost:5480/sqoop \
    --table nztable \
    --username nzuser \
    --password nzpass \
    --target-dir hdfsdir

Here is an example of complete command line for export with tab as the fieldterminator character.

$ sqoop export \
    --direct \
    --connect jdbc:netezza://nzhost:5480/sqoop \
    --table nztable \
    --username nzuser \
    --password nzpass \
    --export-dir hdfsdir \
    --input-fields-terminated-by "\t"
25.7.3. Null string handling

Netezza direct connector supports the null-string features of Sqoop. The nullstring values are converted to appropriate external table options during exportand import operations.

Table 56. Supported export control arguments:

ArgumentDescription
--input-null-string <null-string>The string to be interpreted as null for string columns.
--input-null-non-string <null-string>The string to be interpreted as null for non string columns.

In the case of Netezza direct mode connector, both the arguments must beleft to the default values or explicitly set to the same value. Furthermorethe null string value is restricted to 0-4 utf8 characters.

On export, for non-string columns, if the chosen null value is a validrepresentation in the column domain, then the column might not be loaded asnull. For example, if the null string value is specified as "1", then onexport, any occurrence of "1" in the input file will be loaded as value 1instead of NULL for int columns.

It is suggested that the null value be specified as empty string forperformance and consistency.

Table 57. Supported import control arguments:

ArgumentDescription
--null-string <null-string>The string to be interpreted as null for string columns.
--null-non-string <null-string>The string to be interpreted as null for non string columns.

In the case of Netezza direct mode connector, both the arguments must beleft to the default values or explicitly set to the same value. Furthermorethe null string value is restricted to 0-4 utf8 characters.

On import, for non-string columns, the chosen null value in currentimplementations the null value representation is ignored for non charactercolumns. For example, if the null string value is specified as "\N", then onimport, any occurrence of NULL for non-char columns in the table will beimported as an empty string instead of \N, the chosen null stringrepresentation.

It is suggested that the null value be specified as empty string forperformance and consistency.

25.8. Data Connector for Oracle and Hadoop

25.8.1. About
25.8.1.1. Jobs 25.8.1.2. How The Standard Oracle Manager Works for Imports 25.8.1.3. How The Data Connector for Oracle and Hadoop Works for Imports 25.8.1.4. Data Connector for Oracle and Hadoop Exports
25.8.2. Requirements
25.8.2.1. Ensure The Oracle Database JDBC Driver Is Setup Correctly 25.8.2.2. Oracle Roles and Privileges 25.8.2.3. Additional Oracle Roles And Privileges Required for Export 25.8.2.4. Supported Data Types
25.8.3. Execute Sqoop With Data Connector for Oracle and Hadoop
25.8.3.1. Connect to Oracle / Oracle RAC 25.8.3.2. Connect to An Oracle Database Instance 25.8.3.3. Connect to An Oracle RAC 25.8.3.4. Login to The Oracle Instance 25.8.3.5. Kill Data Connector for Oracle and Hadoop Jobs
25.8.4. Import Data from Oracle
25.8.4.1. Match Hadoop Files to Oracle Table Partitions 25.8.4.2. Specify The Partitions To Import 25.8.4.3. Consistent Read: All Mappers Read From The Same Point In Time
25.8.5. Export Data into Oracle
25.8.5.1. Insert-Export 25.8.5.2. Update-Export 25.8.5.3. Merge-Export 25.8.5.4. Create Oracle Tables 25.8.5.5. NOLOGGING 25.8.5.6. Partitioning 25.8.5.7. Match Rows Via Multiple Columns 25.8.5.8. Storage Clauses
25.8.6. Manage Date And Timestamp Data Types
25.8.6.1. Import Date And Timestamp Data Types from Oracle 25.8.6.2. The Data Connector for Oracle and Hadoop Does Not Apply A Time Zone to DATE / TIMESTAMP Data Types 25.8.6.3. The Data Connector for Oracle and Hadoop Retains Time Zone Information in TIMEZONE Data Types 25.8.6.4. Data Connector for Oracle and Hadoop Explicitly States Time Zone for LOCAL TIMEZONE Data Types 25.8.6.5. java.sql.Timestamp 25.8.6.6. Export Date And Timestamp Data Types into Oracle
25.8.7. Configure The Data Connector for Oracle and Hadoop
25.8.7.1. oraoop-site-template.xml 25.8.7.2. oraoop.oracle.session.initialization.statements 25.8.7.3. oraoop.table.import.where.clause.location 25.8.7.4. oracle.row.fetch.size 25.8.7.5. oraoop.import.hint 25.8.7.6. oraoop.oracle.append.values.hint.usage 25.8.7.7. mapred.map.tasks.speculative.execution 25.8.7.8. oraoop.block.allocation 25.8.7.9. oraoop.import.omit.lobs.and.long 25.8.7.10. oraoop.locations 25.8.7.11. sqoop.connection.factories 25.8.7.12. Expressions in oraoop-site.xml
25.8.8. Troubleshooting The Data Connector for Oracle and Hadoop
25.8.8.1. Quote Oracle Owners And Tables 25.8.8.2. Quote Oracle Columns 25.8.8.3. Confirm The Data Connector for Oracle and Hadoop Can Initialize The Oracle Session 25.8.8.4. Check The Sqoop Debug Logs for Error Messages 25.8.8.5. Export: Check Tables Are Compatible 25.8.8.6. Export: Parallelization 25.8.8.7. Export: Check oraoop.oracle.append.values.hint.usage 25.8.8.8. Turn On Verbose
25.8.1. About

The Data Connector for Oracle and Hadoop is now included in Sqoop.

It can be enabled by specifying the --direct argument for your import orexport job.

25.8.1.1. Jobs

The Data Connector for Oracle and Hadoop inspects each Sqoop job and assumesresponsibility for the ones it can perform better than the Oracle manager builtinto Sqoop.

Data Connector for Oracle and Hadoop accepts responsibility for the followingSqoop Job types:

  • Import jobs that areNon-Incremental.
  • Export jobs
  • Data Connector for Oracle and Hadoop does not accept responsibility for otherSqoop job types. For example Data Connector for Oracle and Hadoop does notaccepteval jobs etc.

Data Connector for Oracle and Hadoop accepts responsibility for those Sqoop Jobswith the following attributes:

  • Oracle-related
  • Table-Based - Jobs where the table argument is used and the specified objectis a table.

    [Note]Note

    Data Connector for Oracle and Hadoop does not process index-organizedtables unless the table is partitioned andoraoop.chunk.method is setto PARTITION

  • There are at least 2 mappers — Jobs where the Sqoop command-line does notinclude:--num-mappers 1
25.8.1.2. How The Standard Oracle Manager Works for Imports

The Oracle manager built into Sqoop uses a range-based query for each mapper.Each mapper executes a query of the form:

SELECT * FROM sometable WHERE id >= lo AND id < hi

The lo and hi values are based on the number of mappers and the minimum andmaximum values of the data in the column the table is being split by.

If no suitable index exists on the table then these queries result in fulltable-scans within Oracle. Even with a suitable index, multiple mappers mayfetch data stored within the same Oracle blocks, resulting in redundant IOcalls.

25.8.1.3. How The Data Connector for Oracle and Hadoop Works for Imports

The Data Connector for Oracle and Hadoop generates queries for the mappers ofthe form:

SELECT *
  FROM sometable
 WHERE rowid >= dbms_rowid.rowid_create(1, 893, 1, 279, 0) AND
       rowid <= dbms_rowid.rowid_create(1, 893, 1, 286, 32767)

The Data Connector for Oracle and Hadoop queries ensure that:

  • No two mappers read data from the same Oracle block. This minimizesredundant IO.
  • The table does not require indexes.
  • The Sqoop command line does not need to specify a --split-by column.
25.8.1.4. Data Connector for Oracle and Hadoop Exports

Benefits of the Data Connector for Oracle and Hadoop:

  • Merge-Export facility - Update Oracle tables by modifying changed rows ANDinserting rows from the HDFS file that did not previously exist in the Oracletable. The Connector for Oracle and Hadoop’s Merge-Export is unique - there isno Sqoop equivalent.
  • Lower impact on the Oracle database - Update the rows in the Oracle tablethat have changed, not all rows in the Oracle table. This has performancebenefits and reduces the impact of the query on Oracle (for example, the Oracleredo logs).
  • Improved performance - With partitioned tables, mappers utilize temporaryOracle tables which allow parallel inserts and direct path writes.
25.8.2. Requirements
25.8.2.1. Ensure The Oracle Database JDBC Driver Is Setup Correctly

You may want to ensure the Oracle Database 11g Release 2 JDBC driver is setupcorrectly on your system. This driver is required for Sqoop to work with Oracle.

The Oracle Database 11g Release 2 JDBC driver file is ojdbc6.jar (3.2Mb).

If this file is not on your system then download it from:http://www.oracle.com/technetwork/database/features/jdbc/index-091264.html

This file should be put into the $SQOOP_HOME/lib directory.

25.8.2.2. Oracle Roles and Privileges

The Oracle user for The Data Connector for Oracle and Hadoop requires thefollowing roles and privileges:

  • create session

In addition, the user must have the select any dictionary privilege orselect_catalog_role role or all of the following object privileges:

  • select on v_$instance
  • select on dba_tables
  • select on dba_tab_columns
  • select on dba_objects
  • select on dba_extents
  • select on dba_segments — Required for Sqoop imports only
  • select on dba_constraints — Required for Sqoop imports only
  • select on v_$database — Required for Sqoop imports only
  • select on v_$parameter — Required for Sqoop imports only
[Note]Note

The user also requires the alter session privilege to make use of sessiontracing functionality. See "oraoop.oracle.session.initialization.statements"for more information.

25.8.2.3. Additional Oracle Roles And Privileges Required for Export

The Oracle user for Data Connector for Oracle and Hadoop requires:

  • Quota on the tablespace in which the Oracle export tables are located.

    An example Oracle command to achieve this is

    alter user username quota unlimited on tablespace
  • The following privileges:

    Type of ExportPrivileges Required
    All Exportcreate table
     select on dba_tab_partitions
     select on dba_tab_subpartitions
     select on dba_indexes
     select on dba_ind_columns
    Insert-Export with a template table into another schemaselect any table
     create any table
     insert any table
     alter any table (partitioning)
    Insert-Export without a template table into another schemaselect,insert on table (no partitioning)
     select,alter on table (partitioning)
    Update-Export into another schemaselect,update on table (no partitioning)
     select,delete,alter,insert on table (partitioning)
    Merge-Export into another schemaselect,insert,update on table (no partitioning)
     select,insert,delete,alter on table (partitioning)
25.8.2.4. Supported Data Types

The following Oracle data types are supported by the Data Connector forOracle and Hadoop:

BINARY_DOUBLENCLOB
BINARY_FLOATNUMBER
BLOBNVARCHAR2
CHARRAW
CLOBROWID
DATETIMESTAMP
FLOATTIMESTAMP WITH TIME ZONE
INTERVAL DAY TO SECONDTIMESTAMP WITH LOCAL TIME ZONE
INTERVAL YEAR TO MONTHURITYPE
LONGVARCHAR2
NCHAR 

All other Oracle column types are NOT supported. Example Oracle column types NOTsupported by Data Connector for Oracle and Hadoop include:

All of the ANY typesBFILE
All of the MEDIA typesLONG RAW
All of the SPATIAL typesMLSLABEL
Any type referred to as UNDEFINEDUROWID
All custom (user-defined) URI typesXMLTYPE
[Note]Note

Data types RAW, LONG and LOB (BLOB, CLOB and NCLOB) are supported forData Connector for Oracle and Hadoop imports. They are not supported for DataConnector for Oracle and Hadoop exports.

25.8.3. Execute Sqoop With Data Connector for Oracle and Hadoop
25.8.3.1. Connect to Oracle / Oracle RAC

The Sqoop --connect parameter defines the Oracle instance or Oracle RAC toconnect to. It is required with all Sqoop import and export commands.

Data Connector for Oracle and Hadoop expects the associated connection stringto be of a specific format dependent on whether the Oracle SID, Serviceor TNS name is defined. The TNS name based URL scheme can be used to enableauthentication using Oracle wallets.

--connect jdbc:oracle:thin:@OracleServer:OraclePort:OracleSID

--connect jdbc:oracle:thin:@//OracleServer:OraclePort/OracleService

--connect jdbc:oracle:thin:@TNSName

25.8.3.2. Connect to An Oracle Database Instance
Parameter / ComponentDescription
jdbc:oracle:thinThe Data Connector for Oracle and Hadoop requires the connection string starts with jdbc:oracle.
  
 The Data Connector for Oracle and Hadoop has been tested with the thin driver however it should work equally well with other drivers such as OCI.
OracleServerThe host name of the Oracle server.
OraclePortThe port to connect to the Oracle server.
OracleSIDThe Oracle instance.
OracleServiceThe Oracle Service.
TNSNameThe TNS name for the entry describing the connection to the Oracle server.
[Note]Note

The Hadoop mappers connect to the Oracle database using a dynamicallygenerated JDBC URL. This is designed to improve performance however it can bedisabled by specifying:

-D oraoop.jdbc.url.verbatim=true

25.8.3.3. Connect to An Oracle RAC

Use the --connect parameter as above. The connection string should point toone instance of the Oracle RAC. The listener of the host of this Oracleinstance will locate the other instances of the Oracle RAC.

[Note]Note

To improve performance, The Data Connector for Oracle and Hadoopidentifies the active instances of the Oracle RAC and connects each Hadoopmapper to them in a roundrobin manner.

If services are defined for this Oracle RAC then use the following parameterto specify the service name:

-D oraoop.oracle.rac.service.name=ServiceName

Parameter / ComponentDescription
OracleServer:OraclePort:OracleInstanceName one instance of the Oracle RAC. The Data Connector for Oracle and Hadoop assumes the same port number for all instances of the Oracle RAC.
  
 The listener of the host of this Oracle instance is used to locate other instances of the Oracle RAC. For more information enter this command on the host command line:
  
 lsnrctl status
-D oraoop.oracle.rac.service.name=ServiceNameThe service to connect to in the Oracle RAC.
  
 A connection is made to all instances of the Oracle RAC associated with the service given byServiceName.
  
 If omitted, a connection is made to all instances of the Oracle RAC.
  
 The listener of the host of this Oracle instance needs to know theServiceName and all instances of the Oracle RAC. For more information enter this command on the host command line:
  
 lsnrctl status
25.8.3.4. Login to The Oracle Instance

Login to the Oracle instance on the Sqoop command line:

--connect jdbc:oracle:thin:@OracleServer:OraclePort:OracleInstance --usernameUserName -P

Parameter / ComponentDescription
--username UserNameThe username to login to the Oracle instance (SID).
-PYou will be prompted for the password to login to the Oracle instance.
25.8.3.5. Kill Data Connector for Oracle and Hadoop Jobs

Use the Hadoop Job Tracker to kill the Sqoop job, just as you would kill anyother Map-Reduce job.

$ hadoop job -kill jobid

To allow an Oracle DBA to kill a Data Connector for Oracle and Hadoopjob (via killing the sessions in Oracle) you need to prevent Map-Reduce fromre-attempting failed jobs. This is done via the following Sqoopcommand-line switch:

-D mapred.map.max.attempts=1

This sends instructions similar to the following to the console:

14/07/07 15:24:51 INFO oracle.OraOopManagerFactory:
Note: This Data Connector for Oracle and Hadoop job can be killed via Oracle
by executing the following statement:
  begin
    for row in (select sid,serial# from v$session where
                module='Data Connector for Oracle and Hadoop' and
                action='import 20140707152451EST') loop
      execute immediate 'alter system kill session ''' || row.sid ||
                        ',' || row.serial# || '''';
    end loop;
  end;
25.8.4. Import Data from Oracle

Execute Sqoop. Following is an example command:

$ sqoop import --direct --connect … --table OracleTableName

If The Data Connector for Oracle and Hadoop accepts the job then the followingtext is output:

**************************************************
*** Using Data Connector for Oracle and Hadoop ***
**************************************************
[Note]Note
  • More information is available on the --connect parameter. See "Connect toOracle / Oracle RAC" for more information.
  • If Java runs out of memory the workaround is to specify each mapper’sJVM memory allocation. Add the following parameter for example to allocate 4GB:

    -Dmapred.child.java.opts=-Xmx4000M

  • An Oracle optimizer hint is included in the SELECT statement by default.See "oraoop.import.hint" for more information.

    You can alter the hint on the command line as follows:

    -Doraoop.import.hint="NO_INDEX(t)"

    You can turn off the hint on the command line as follows (notice the spacebetween the double quotes):

    -Doraoop.import.hint=" "

25.8.4.1. Match Hadoop Files to Oracle Table Partitions

-Doraoop.chunk.method={ROWID|PARTITION}

To import data from a partitioned table in such a way that the resulting HDFSfolder structure in Hadoop will match the table’s partitions, set the chunkmethod to PARTITION. The alternative (default) chunk method is ROWID.

[Note]Note
  • For the number of Hadoop files to match the number of Oracle partitions,set the number of mappers to be greater than or equal to the number ofpartitions.
  • If the table is not partitioned then value PARTITION will lead to an error.
25.8.4.2. Specify The Partitions To Import

-Doraoop.import.partitions=PartitionA,PartitionB --table OracleTableName

Imports PartitionA and PartitionB ofOracleTableName.

[Note]Note
  • You can enclose an individual partition name in double quotes to retain theletter case or if the name has special characters.

    -Doraoop.import.partitions='"PartitionA",PartitionB' --table OracleTableName

    If the partition name is not double quoted then its name will be automaticallyconverted to upper case, PARTITIONB for above.

    When using double quotes the entire list of partition names must be enclosed insingle quotes.

    If the last partition name in the list is double quoted then there must be acomma at the end of the list.

    -Doraoop.import.partitions='"PartitionA","PartitionB",' --tableOracleTableName

  • Name each partition to be included. There is no facility to provide a range ofpartition names.
  • There is no facility to define sub partitions. The entire partition isincluded/excluded as per the filter.
25.8.4.3. Consistent Read: All Mappers Read From The Same Point In Time

-Doraoop.import.consistent.read={true|false}

When set to false (by default) each mapper runs a select query. This willreturn potentially inconsistent data if there are a lot of DML operations onthe table at the time of import.

Set to true to ensure all mappers read from the same point in time. TheSystem Change Number (SCN) is passed down to all mappers, which use the OracleFlashback Query to query the table as at that SCN.

[Note]Note
  • Values true | false are case sensitive.
  • By default the SCN is taken from V$database. You can specify the SCN in thefollowing command

    -Doraoop.import.consistent.read.scn=12345

25.8.5. Export Data into Oracle

Execute Sqoop. Following is an example command:

$ sqoop export --direct --connect … --table OracleTableName --export-dir/user/username/tablename

The Data Connector for Oracle and Hadoop accepts all jobs that export data toOracle. You can verify The Data Connector for Oracle and Hadoop is in use bychecking the following text is output:

**************************************************
*** Using Data Connector for Oracle and Hadoop ***
**************************************************
[Note]Note
  • OracleTableName is the Oracle table the data will export into.
  • OracleTableName can be in a schema other than that for the connecting user.Prefix the table name with the schema, for exampleSchemaName.OracleTableName.
  • Hadoop tables are picked up from the /user/username/tablename directory.
  • The export will fail if the Hadoop file contains any fields of a data typenot supported by The Data Connector for Oracle and Hadoop. See"Supported Data Types" for more information.
  • The export will fail if the column definitions in the Hadoop table do notexactly match the column definitions in the Oracle table.
  • The Data Connector for Oracle and Hadoop indicates if it finds temporarytables that it created more than a day ago that still exist. Usually thesetables can be dropped. The only circumstance when these tables should not bedropped is when an The Data Connector for Oracle and Hadoop job has beenrunning for more than 24 hours and is still running.
  • More information is available on the --connect parameter. See"Connect to Oracle / Oracle RAC" for more information.
25.8.5.1. Insert-Export

Appends data to OracleTableName. It does not modify existing data inOracleTableName.

Insert-Export is the default method, executed in the absence of the--update-key parameter. All rows in the HDFS file in/user/UserName/TableName are inserted intoOracleTableName. Nochange is made to pre-existing data inOracleTableName.

$ sqoop export --direct --connect … --table OracleTableName --export-dir/user/username/tablename

[Note]Note
  • If OracleTableName was previously created by The Data Connector for Oracleand Hadoop with partitions then this export will create a new partition for thedata being inserted.
  • When creating OracleTableName specify a template. See"Create Oracle Tables" for more information.
25.8.5.2. Update-Export

--update-key OBJECT

Updates existing rows in OracleTableName.

Rows in the HDFS file in /user/UserName/TableName are matched to rows inOracleTableName by theOBJECT column. Rows that match are copied from theHDFS file to the Oracle table. No action is taken on rows that do not match.

$ sqoop export --direct --connect … --update-key OBJECT --tableOracleTableName --export-dir /user/username/tablename

[Note]Note
  • If OracleTableName was previously created by The Data Connector for Oracleand Hadoop with partitions then this export will create a new partition for thedata being inserted. Updated rows will be moved to the new partition that wascreated for the export.
  • For performance reasons it is strongly recommended that where more than a fewrows are involved columnOBJECT be an index column of OracleTableName.
  • Ensure the column name defined with --update-key OBJECT is specified in thecorrect letter case. Sqoop will show an error if the letter case is incorrect.
  • It is possible to match rows via multiple columns. See "Match Rows ViaMultiple Columns" for more information.
25.8.5.3. Merge-Export

--update-key OBJECT -Doraoop.export.merge=true

Updates existing rows in OracleTableName. Copies across rows from the HDFSfile that do not exist within the Oracle table.

Rows in the HDFS file in /user/UserName/TableName are matched to rows inOracleTableName by theOBJECT column. Rows that match are copied from theHDFS file to the Oracle table. Rows in the HDFS file that do not exist inOracleTableName are added toOracleTableName.

$ sqoop export --direct --connect … --update-key OBJECT-Doraoop.export.merge=true --table OracleTableName --export-dir/user/username/tablename

[Note]Note
  • Merge-Export is unique to The Data Connector for Oracle and Hadoop. It isnot a standard Sqoop feature.
  • If OracleTableName was previously created by The Data Connector for Oracleand Hadoop with partitions, then this export will create a new partition forthe data being inserted. Updated rows will be moved to the new partition thatwas created for the export.
  • For performance reasons it is strongly recommended that where more than afew rows are involved columnOBJECT be an index column of OracleTableName.
  • Ensure the column name defined with --update-key OBJECT is specified in thecorrect letter case. Sqoop will show an error if the letter case is incorrect.
  • It is possible to match rows via multiple columns. See "Match Rows ViaMultiple Columns" for more information.
25.8.5.4. Create Oracle Tables

-Doraoop.template.table=TemplateTableName

Creates OracleTableName by replicating the structure and data types ofTemplateTableName.TemplateTableName is a table that exists in Oracle priorto executing the Sqoop command.

[Note]Note
  • The export will fail if the Hadoop file contains any fields of a data typenot supported by The Data Connector for Oracle and Hadoop. See "SupportedData Types" for more information.
  • The export will fail if the column definitions in the Hadoop table do notexactly match the column definitions in the Oracle table.
  • This parameter is specific to creating an Oracle table. The export will failifOracleTableName already exists in Oracle.

Example command:

$ sqoop export --direct --connect.. --table OracleTableName --export-dir/user/username/tablename -Doraoop.template.table=TemplateTableName

25.8.5.5. NOLOGGING

-Doraoop.nologging=true

Assigns the NOLOGGING option to OracleTableName.

NOLOGGING may enhance performance but you will be unable to backup the table.

25.8.5.6. Partitioning

-Doraoop.partitioned=true

Partitions the table with the following benefits:

  • The speed of the export is improved by allowing each mapper to insert datainto a separate Oracle table using direct path writes. (An alter table exchangesubpartition SQL statement is subsequently executed to swap the data into theexport table.)
  • You can selectively query or delete the data inserted by each Sqoop exportjob. For example, you can delete old data by dropping old partitions fromthe table.

The partition value is the SYSDATE of when Sqoop export job was performed.

The partitioned table created by The Data Connector for Oracle and Hadoopincludes the following columns that don’t exist in the template table:

  • oraoop_export_sysdate - This is the Oracle SYSDATE when the Sqoop exportjob was performed. The created table will be partitioned by this column.
  • oraoop_mapper_id - This is the id of the Hadoop mapper that was used toprocess the rows from the HDFS file. Each partition is subpartitioned by thiscolumn. This column exists merely to facilitate the exchange subpartitionmechanism that is performed by each mapper during the export process.
  • oraoop_mapper_row - A unique row id within the mapper / partition.
[Note]Note

If a unique row id is required for the table it can be formed by acombination of oraoop_export_sysdate, oraoop_mapper_id and oraoop_mapper_row.

25.8.5.7. Match Rows Via Multiple Columns

-Doraoop.update.key.extra.columns="ColumnA,ColumnB"

Used with Update-Export and Merge-Export to match on more than one column. Thefirst column to be matched on is--update-key OBJECT. To match on additionalcolumns, specify those columns on this parameter.

[Note]Note
  • Letter case for the column names on this parameter is not important.
  • All columns used for matching should be indexed. The first three items on theindex should beColumnA, ColumnB and the column specified on--update-key - but the order in which the columns are specified is notimportant.
25.8.5.8. Storage Clauses

-Doraoop.temporary.table.storage.clause="StorageClause"

-Doraoop.table.storage.clause="StorageClause"

Use to customize storage with Oracle clauses as in TABLESPACE or COMPRESS

-Doraoop.table.storage.clause applies to the export table that is createdfrom-Doraoop.template.table. See "Create Oracle Tables" for moreinformation.-Doraoop.temporary.table.storage.clause applies to all otherworking tables that are created during the export process and then dropped atthe end of the export job.

25.8.6. Manage Date And Timestamp Data Types
25.8.6.1. Import Date And Timestamp Data Types from Oracle

This section lists known differences in the data obtained by performing anData Connector for Oracle and Hadoop import of an Oracle table versus a nativeSqoop import of the same table.

25.8.6.2. The Data Connector for Oracle and Hadoop Does Not Apply A Time Zone to DATE / TIMESTAMP Data Types

Data stored in a DATE or TIMESTAMP column of an Oracle table is not associatedwith a time zone. Sqoop without the Data Connector for Oracle and Hadoopinappropriately applies time zone information to this data.

Take for example the following timestamp in an Oracle DATE or TIMESTAMP column:2am on 3rd October, 2010.

Request Sqoop without the Data Connector for Oracle and Hadoop import this datausing a system located in Melbourne Australia. The data is adjusted to MelbourneDaylight Saving Time. The data is imported into Hadoop as:3am on 3rd October, 2010.

The Data Connector for Oracle and Hadoop does not apply time zone information tothese Oracle data-types. Even from a system located in Melbourne Australia, TheData Connector for Oracle and Hadoop ensures the Oracle and Hadoop timestampsmatch. The Data Connector for Oracle and Hadoop correctly imports thistimestamp as:2am on 3rd October, 2010.

[Note]Note

In order for The Data Connector for Oracle and Hadoop to ensure dataaccuracy, Oracle DATE and TIMESTAMP values must be represented by a String,even when--as-sequencefile is used on the Sqoop command-line to produce abinary file in Hadoop.

25.8.6.3. The Data Connector for Oracle and Hadoop Retains Time Zone Information in TIMEZONE Data Types

Data stored in a TIMESTAMP WITH TIME ZONE column of an Oracle table isassociated with a time zone. This data consists of two distinct parts: when theevent occurred and where the event occurred.

When Sqoop without The Data Connector for Oracle and Hadoop is used to importdata it converts the timestamp to the time zone of the system running Sqoop andomits the component of the data that specifies where the event occurred.

Take for example the following timestamps (with time zone) in an OracleTIMESTAMP WITH TIME ZONE column:

2:59:00 am on 4th April, 2010. Australia/Melbourne
2:59:00 am on 4th April, 2010. America/New York

Request Sqoop without The Data Connector for Oracle and Hadoop import this datausing a system located in Melbourne Australia. From the data imported intoHadoop we know when the events occurred, assuming we know the Sqoop command wasrun from a system located in the Australia/Melbourne time zone, but we have lostthe information regarding where the event occurred.

2010-04-04 02:59:00.0
2010-04-04 16:59:00.0

Sqoop with The Data Connector for Oracle and Hadoop imports the exampletimestamps as follows. The Data Connector for Oracle and Hadoop retains thetime zone portion of the data.

2010-04-04 02:59:00.0 Australia/Melbourne
2010-04-04 02:59:00.0 America/New_York
25.8.6.4. Data Connector for Oracle and Hadoop Explicitly States Time Zone for LOCAL TIMEZONE Data Types

Data stored in a TIMESTAMP WITH LOCAL TIME ZONE column of an Oracle table isassociated with a time zone. Multiple end-users in differing time zones(locales) will each have that data expressed as a timestamp within theirrespective locale.

When Sqoop without the Data Connector for Oracle and Hadoop is used to importdata it converts the timestamp to the time zone of the system running Sqoop andomits the component of the data that specifies location.

Take for example the following two timestamps (with time zone) in an OracleTIMESTAMP WITH LOCAL TIME ZONE column:

2:59:00 am on 4th April, 2010. Australia/Melbourne
2:59:00 am on 4th April, 2010. America/New York

Request Sqoop without the Data Connector for Oracle and Hadoop import this datausing a system located in Melbourne Australia. The timestamps are importedcorrectly but the local time zone has to be guessed. If multiple systems indifferent locale were executing the Sqoop import it would be very difficult todiagnose the cause of the data corruption.

2010-04-04 02:59:00.0
2010-04-04 16:59:00.0

Sqoop with the Data Connector for Oracle and Hadoop explicitly states the timezone portion of the data imported into Hadoop. The local time zone is GMT bydefault. You can set the local time zone with parameter:

-Doracle.sessionTimeZone=Australia/Melbourne

The Data Connector for Oracle and Hadoop would import these two timestamps as:

2010-04-04 02:59:00.0 Australia/Melbourne
2010-04-04 16:59:00.0 Australia/Melbourne
25.8.6.5. java.sql.Timestamp

To use Sqoop’s handling of date and timestamp data types when importing datafrom Oracle use the following parameter:

-Doraoop.timestamp.string=false

[Note]Note

Sqoop’s handling of date and timestamp data types does not store thetimezone. However, some developers may prefer Sqoop’s handling as the DataConnector for Oracle and Hadoop converts date and timestamp data types tostring. This may not work for some developers as the string will requireparsing later in the workflow.

25.8.6.6. Export Date And Timestamp Data Types into Oracle

Ensure the data in the HDFS file fits the required format exactly before usingSqoop to export the data into Oracle.

[Note]Note
  • The Sqoop export command will fail if the data is not in the required format.
  • ff = Fractional second
  • TZR = Time Zone Region
Oracle Data TypeRequired Format of The Data in the HDFS File
DATEyyyy-mm-dd hh24:mi:ss
TIMESTAMPyyyy-mm-dd hh24:mi:ss.ff
TIMESTAMPTZyyyy-mm-dd hh24:mi:ss.ff TZR
TIMESTAMPLTZyyyy-mm-dd hh24:mi:ss.ff TZR
25.8.7. Configure The Data Connector for Oracle and Hadoop
25.8.7.1. oraoop-site-template.xml

The oraoop-site-template.xml file is supplied with the Data Connector forOracle and Hadoop. It contains a number of ALTER SESSION statements that areused to initialize the Oracle sessions created by the Data Connector for Oracleand Hadoop.

If you need to customize these initializations to your environment then:

  1. Find oraoop-site-template.xml in the Sqoop configuration directory.
  2. Copy oraoop-site-template.xml tooraoop-site.xml.
  3. Edit the ALTER SESSION statements inoraoop-site.xml.
25.8.7.2. oraoop.oracle.session.initialization.statements

The value of this property is a semicolon-delimited list of Oracle SQLstatements. These statements are executed, in order, for each Oracle sessioncreated by the Data Connector for Oracle and Hadoop.

The default statements include:

alter session set time_zone = '{oracle.sessionTimeZone|GMT}';

This statement initializes the timezone of the JDBC client. This ensures thatdata from columns of type TIMESTAMP WITH LOCAL TIMEZONE are correctly adjustedinto the timezone of the client and not kept in the timezone of the Oracledatabase.

[Note]Note
  • There is an explanation to the text within the curly-braces. See"Expressions in oraoop-site.xml" for more information..
  • A list of the time zones supported by your Oracle database is available byexecuting the following query:SELECT TZNAME FROM V$TIMEZONE_NAMES;
alter session disable parallel query;

This statement instructs Oracle to not parallelize SQL statements executed bythe Data Connector for Oracle and Hadoop sessions. This Oracle feature isdisabled because the Map/Reduce job launched by Sqoop is the mechanism usedfor parallelization.

It is recommended that you not enable parallel query because it can have anadverse effect the load on the Oracle instance and on the balance betweenthe Data Connector for Oracle and Hadoop mappers.

Some export operations are performed in parallel where deemed appropriate bythe Data Connector for Oracle and Hadoop. See "Parallelization" formore information.

alter session set "_serial_direct_read"=true;
This statement instructs Oracle to bypass the buffer cache. This is used toprevent Oracle from filling its buffers with the data being read by the DataConnector for Oracle and Hadoop, therefore diminishing its capacity to cachehigher prioritized data. Hence, this statement is intended to minimize theData Connector for Oracle and Hadoop’s impact on the immediate futureperformance of the Oracle database.
--alter session set events '10046 trace name context forever, level 8';
This statement has been commented-out. To allow tracing, remove the commenttoken "--" from the start of the line.
[Note]Note
  • These statements are placed on separate lines for readability. They do notneed to be placed on separate lines.
  • A statement can be commented-out via the standard Oracle double-hyphentoken: "--". The comment takes effect until the next semicolon.
25.8.7.3. oraoop.table.import.where.clause.location
SUBSPLIT (default)

When set to this value, the where clause is applied to each subquery used toretrieve data from the Oracle table.

A Sqoop command like:

sqoop import -D oraoop.table.import.where.clause.location=SUBSPLIT --tableJUNK --where "owner like 'G%'"

Generates SQL query of the form:

SELECT OWNER,OBJECT_NAME
  FROM JUNK
 WHERE ((rowid >=
           dbms_rowid.rowid_create(1, 113320, 1024, 4223664, 0)
         AND rowid <=
             dbms_rowid.rowid_create(1, 113320, 1024, 4223671, 32767)))
   AND (owner like 'G%')
UNION ALL
SELECT OWNER,OBJECT_NAME
  FROM JUNK
 WHERE ((rowid >=
           dbms_rowid.rowid_create(1, 113320, 1024, 4223672, 0)
         AND rowid <=
           dbms_rowid.rowid_create(1, 113320, 1024, 4223679, 32767)))
   AND (owner like 'G%')
SPLIT

When set to this value, the where clause is applied to the entire SQLstatement used by each split/mapper.

A Sqoop command like:

sqoop import -D oraoop.table.import.where.clause.location=SPLIT --tableJUNK --where "rownum ⇐ 10"

Generates SQL query of the form:

SELECT OWNER,OBJECT_NAME
  FROM (
        SELECT OWNER,OBJECT_NAME
          FROM JUNK
         WHERE ((rowid >=
                   dbms_rowid.rowid_create(1, 113320, 1024, 4223664, 0)
                 AND rowid <=
                   dbms_rowid.rowid_create(1, 113320, 1024, 4223671, 32767)))
        UNION ALL
        SELECT OWNER,OBJECT_NAME
          FROM JUNK
         WHERE ((rowid >=
                   dbms_rowid.rowid_create(1, 113320, 1024, 4223672, 0)
                 AND rowid <=
                   dbms_rowid.rowid_create(1, 113320, 1024, 4223679,32767)))
       )
 WHERE rownum <= 10
[Note]Note
  • In this example, there are up to 10 rows imported per mapper.
  • The SPLIT clause may result in greater overhead than the SUBSPLITclause because the UNION statements need to be fully materializedbefore the data can be streamed to the mappers. However, you maywish to use SPLIT in the case where you want to limit the totalnumber of rows processed by each mapper.
25.8.7.4. oracle.row.fetch.size

The value of this property is an integer specifying the number of rows theOracle JDBC driver should fetch in each network round-trip to the database.The default value is 5000.

If you alter this setting, confirmation of thechange is displayed in the logs of the mappers during the Map-Reduce job.

25.8.7.5. oraoop.import.hint

The Oracle optimizer hint is added to the SELECT statement for IMPORT jobsas follows:

SELECT /*+ NO_INDEX(t) */ * FROM employees;

The default hint is NO_INDEX(t)

[Note]Note
  • The hint can be added to the command line. See "Import Data from Oracle" formore information.
  • See the Oracle Database Performance Tuning Guide (Using Optimizer Hints)for more information on Oracle optimizer hints.
  • To turn the hint off, insert a space between the <value> elements.

    <property>
      <name>oraoop.import.hint</name>
      <value> </value>
    </property>
25.8.7.6. oraoop.oracle.append.values.hint.usage

The value of this property is one of: AUTO / ON / OFF.

AUTO

AUTO is the default value.

Currently AUTO is equivalent to OFF.

ON
During export the Data Connector for Oracle and Hadoop uses direct pathwrites to populate the target Oracle table, bypassing the buffer cache.Oracle only allows a single session to perform direct writes against a specifictable at any time, so this has the effect of serializing the writes to thetable. This may reduce throughput, especially if the number of mappers is high.However, for databases where DBWR is very busy, or where the IO bandwidth tothe underlying table is narrow (table resides on a single disk spindle forinstance), then setting oraoop.oracle.append.values.hint.usage to ON mayreduce the load on the Oracle database and possibly increase throughput.
OFF
During export the Data Connector for Oracle and Hadoop does not use the APPEND_VALUES Oracle hint.
[Note]Note

This parameter is only effective on Oracle 11g Release 2 and above.

25.8.7.7. mapred.map.tasks.speculative.execution

By default speculative execution is disabled for the Data Connector forOracle and Hadoop. This avoids placing redundant load on the Oracle database.

If Speculative execution is enabled, then Hadoop may initiate multiple mappersto read the same blocks of data, increasing the overall load on the database.

25.8.7.8. oraoop.block.allocation

This setting determines how Oracle’s data-blocks are assigned to Map-Reduce mappers.

[Note]Note

Applicable to import. Not applicable to export.

ROUNDROBIN (default)

Each chunk of Oracle blocks is allocated to the mappers in a roundrobinmanner. This helps prevent one of the mappers from beingallocated a large proportion of typically small-sized blocks from thestart of Oracle data-files. In doing so it also helps prevent one of theother mappers from being allocated a large proportion of typicallylarger-sized blocks from the end of the Oracle data-files.

Use this method to help ensure all the mappers are allocated a similaramount of work.

RANDOM
The list of Oracle blocks is randomized before being allocated to themappers via a round-robin approach. This has the benefit of increasingthe chance that, at any given instant in time, each mapper is readingfrom a different Oracle data-file. If the Oracle data-files are located onseparate spindles, this should increase the overall IO throughput.
SEQUENTIAL

Each chunk of Oracle blocks is allocated to the mappers sequentially.This produces the tendency for each mapper to sequentially read a large,contiguous proportion of an Oracle data-file. It is unlikely for theperformance of this method to exceed that of the round-robin methodand it is more likely to allocate a large difference in the work betweenthe mappers.

Use of this method is generally not recommended.

25.8.7.9. oraoop.import.omit.lobs.and.long

This setting can be used to omit all LOB columns (BLOB, CLOB and NCLOB) and LONGcolumn from an Oracle table being imported. This is advantageous introubleshooting, as it provides a convenient way to exclude all LOB-based datafrom the import.

25.8.7.10. oraoop.locations
[Note]Note

Applicable to import. Not applicable to export.

By default, four mappers are used for a Sqoop import job. The number of mapperscan be altered via the Sqoop--num-mappers parameter.

If the data-nodes in your Hadoop cluster have 4 task-slots (that is they are4-CPU core machines) it is likely for all four mappers to execute on thesame machine. Therefore, IO may be concentrated between the Oracle databaseand a single machine.

This setting allows you to control which DataNodes in your Hadoop cluster eachmapper executes on. By assigning each mapper to a separate machine you mayimprove the overall IO performance for the job. This will also have theside-effect of the imported data being more diluted across the machines inthe cluster. (HDFS replication will dilute the data across the cluster anyway.)

Specify the machine names as a comma separated list. The locations areallocated to each of the mappers in a round-robin manner.

If using EC2, specify the internal name of the machines. Here is an exampleof using this parameter from the Sqoop command-line:

$ sqoop import -Doraoop.locations=ip-10-250-23-225.ec2.internal,ip-10-250-107-32.ec2.internal,ip-10-250-207-2.ec2.internal,ip-10-250-27-114.ec2.internal--direct --connect…

25.8.7.11. sqoop.connection.factories

This setting determines behavior if the Data Connector for Oracle and Hadoopcannot accept the job. By default Sqoop accepts the jobs that the Data Connectorfor Oracle and Hadoop rejects.

Set the value to org.apache.sqoop.manager.oracle.OraOopManagerFactory when youwant the job to fail if the Data Connector for Oracle and Hadoop cannotaccept the job.

25.8.7.12. Expressions in oraoop-site.xml

Text contained within curly-braces { and } are expressions to be evaluatedprior to the SQL statement being executed. The expression contains the nameof the configuration property optionally followed by a default value to useif the property has not been set. A pipe | character is used to delimit theproperty name and the default value.

For example:

When this Sqoop command is executed
$ sqoop import -D oracle.sessionTimeZone=US/Hawaii --direct --connect
The statement within oraoop-site.xml
alter session set time_zone ='{oracle.sessionTimeZone|GMT}';
Becomes
alter session set time_zone = 'US/Hawaii'
If the oracle.sessionTimeZone property had not been set, then this statement would use the specified default value and would become
alter session set time_zone = 'GMT'
[Note]Note

The oracle.sessionTimeZone property can be specified within thesqoop-site.xml file if you want this setting to be used all the time.

25.8.8. Troubleshooting The Data Connector for Oracle and Hadoop
25.8.8.1. Quote Oracle Owners And Tables

If the owner of the Oracle table needs to be
quoted, use:

sqoop import … --table
"\"\"Scott\".customers\""


This is the equivalent of:
"Scott".customers

If the Oracle table needs to be quoted, use:

sqoop import … --table
"\"scott.\"Customers\"\""


This is the equivalent of:
scott."Customers"

If both the owner of the Oracle table and the
table itself needs to be quoted, use:

sqoop import … --table
"\"\"Scott\".\"Customers\"\""


This is the equivalent of:
"Scott"."Customers"

[Note]Note
  • The HDFS output directory is called something like:/user/username/"Scott"."Customers"
  • If a table name contains a $ character, it may need to be escaped within yourUnix shell. For example, the dr$object table in the ctxsys schema would bereferred to as: $sqoop import … --table "ctxsys.dr\$object"
25.8.8.2. Quote Oracle Columns
If a column name of an Oracle table needs to be quoted, use

$ sqoop import … --table customers --columns "\"\"first name\"\""

This is the equivalent of: select "first name" from customers

25.8.8.3. Confirm The Data Connector for Oracle and Hadoop Can Initialize The Oracle Session

If the Sqoop output includes feedback such as the following then theconfiguration properties contained withinoraoop-site-template.xml andoraoop-site.xml have been loaded by Hadoop and can be accessed by the DataConnector for Oracle and Hadoop.

14/07/08 15:21:13 INFO oracle.OracleConnectionFactory:Initializing Oracle session with SQL

25.8.8.4. Check The Sqoop Debug Logs for Error Messages

For more information about any errors encountered during the Sqoop import,refer to the log files generated by each of the (by default 4) mappers thatperformed the import.

The logs can be obtained via your Map-Reduce Job Tracker’s web page.

Include these log files with any requests you make for assistance on the SqoopUser Group web site.

25.8.8.5. Export: Check Tables Are Compatible

Check tables particularly in the case of a parsing error.

  • Ensure the fields contained with the HDFS file and the columns within theOracle table are identical. If they are not identical, the Java codedynamically generated by Sqoop to parse the HDFS file will throw an error whenreading the file – causing the export to fail. When creating a table in Oracleensure the definitions for the table template are identical to the definitionsfor the HDFS file.
  • Ensure the data types in the table are supported. See "Supported Data Types"for more information.
  • Are date and time zone based data types used? See "Export Date And TimestampData Types into Oracle" for more information.
25.8.8.6. Export: Parallelization

-D oraoop.export.oracle.parallelization.enabled=false

If you see a parallelization error you may decide to disable parallelizationon Oracle queries.

25.8.8.7. Export: Check oraoop.oracle.append.values.hint.usage

The oraoop.oracle.append.values.hint.usage parameter should not be set to ONif the Oracle table contains either a BINARY_DOUBLE or BINARY_FLOAT column andthe HDFS file being exported contains a NULL value in either of these columntypes. Doing so will result in the error: ORA-12838: cannot read/modify anobject after modifying it in parallel.

25.8.8.8. Turn On Verbose

Turn on verbose on the Sqoop command line.

--verbose

Check Sqoop stdout (standard output) and the mapper logs for information as towhere the problem may be.

26. Getting Support

Some general information is available at thehttp://sqoop.apache.org/

Report bugs in Sqoop to the issue tracker athttps://issues.apache.org/jira/browse/SQOOP.

Questions and discussion regarding the usage of Sqoop should be directed to thesqoop-user mailing list.

Before contacting either forum, run your Sqoop job with the--verbose flag to acquire as much debugging information aspossible. Also report the string returned bysqoop version aswell as the version of Hadoop you are running (hadoop version).

27. Troubleshooting

27.1. General Troubleshooting Process

The following steps should be followed to troubleshoot any failure that youencounter while running Sqoop.

  • Turn on verbose output by executing the same command again and specifying the--verbose option. This produces more debug output on the console which can be inspected to identify any obvious errors.
  • Look at the task logs from Hadoop to see if there are any specific failures recorded there. It is possible that the failure that occurs while task execution is not relayed correctly to the console.
  • Make sure that the necessary input files or input/output tables are present and can be accessed by the user that Sqoop is executing as or connecting to the database as. It is possible that the necessary files or tables are present but the specific user that Sqoop connects as does not have the necessary permissions to access these files.
  • If you are doing a compound action such as populating a Hive table or partition, try breaking the job into two separate actions to see where the problem really occurs. For example if an import that creates and populates a Hive table is failing, you can break it down into two steps - first for doing the import alone, and the second to create a Hive table without the import using thecreate-hive-table tool. While this does not address the original use-case of populating the Hive table, it does help narrow down the problem to either regular import or during the creation and population of Hive table.
  • Search the mailing lists archives and JIRA for keywords relating to the problem. It is possible that you may find a solution discussed there that will help you solve or work-around your problem.

27.2. Specific Troubleshooting Tips

27.2.1. Oracle: Connection Reset Errors

Problem: When using the default Sqoop connector for Oracle, some data doesget transferred, but during the map-reduce job a lot of errors are reportedas below:

11/05/26 16:23:47 INFO mapred.JobClient: Task Id : attempt_201105261333_0002_m_000002_0, Status : FAILED
java.lang.RuntimeException: java.lang.RuntimeException: java.sql.SQLRecoverableException: IO Error: Connection reset
at com.cloudera.sqoop.mapreduce.db.DBInputFormat.setConf(DBInputFormat.java:164)
at org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:62)
at org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:117)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:605)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:322)
at org.apache.hadoop.mapred.Child$4.run(Child.java:268)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1115)
at org.apache.hadoop.mapred.Child.main(Child.java:262)
Caused by: java.lang.RuntimeException: java.sql.SQLRecoverableException: IO Error: Connection reset
at com.cloudera.sqoop.mapreduce.db.DBInputFormat.getConnection(DBInputFormat.java:190)
at com.cloudera.sqoop.mapreduce.db.DBInputFormat.setConf(DBInputFormat.java:159)
... 9 more
Caused by: java.sql.SQLRecoverableException: IO Error: Connection reset
at oracle.jdbc.driver.T4CConnection.logon(T4CConnection.java:428)
at oracle.jdbc.driver.PhysicalConnection.<init>(PhysicalConnection.java:536)
at oracle.jdbc.driver.T4CConnection.<init>(T4CConnection.java:228)
at oracle.jdbc.driver.T4CDriverExtension.getConnection(T4CDriverExtension.java:32)
at oracle.jdbc.driver.OracleDriver.connect(OracleDriver.java:521)
at java.sql.DriverManager.getConnection(DriverManager.java:582)
at java.sql.DriverManager.getConnection(DriverManager.java:185)
at com.cloudera.sqoop.mapreduce.db.DBConfiguration.getConnection(DBConfiguration.java:152)
at com.cloudera.sqoop.mapreduce.db.DBInputFormat.getConnection(DBInputFormat.java:184)
... 10 more
Caused by: java.net.SocketException: Connection reset
at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
at oracle.net.ns.DataPacket.send(DataPacket.java:199)
at oracle.net.ns.NetOutputStream.flush(NetOutputStream.java:211)
at oracle.net.ns.NetInputStream.getNextPacket(NetInputStream.java:227)
at oracle.net.ns.NetInputStream.read(NetInputStream.java:175)
at oracle.net.ns.NetInputStream.read(NetInputStream.java:100)
at oracle.net.ns.NetInputStream.read(NetInputStream.java:85)
at oracle.jdbc.driver.T4CSocketInputStreamWrapper.readNextPacket(T4CSocketInputStreamWrapper.java:123)
at oracle.jdbc.driver.T4CSocketInputStreamWrapper.read(T4CSocketInputStreamWrapper.java:79)
at oracle.jdbc.driver.T4CMAREngine.unmarshalUB1(T4CMAREngine.java:1122)
at oracle.jdbc.driver.T4CMAREngine.unmarshalSB1(T4CMAREngine.java:1099)
at oracle.jdbc.driver.T4CTTIfun.receive(T4CTTIfun.java:288)
at oracle.jdbc.driver.T4CTTIfun.doRPC(T4CTTIfun.java:191)
at oracle.jdbc.driver.T4CTTIoauthenticate.doOAUTH(T4CTTIoauthenticate.java:366)
at oracle.jdbc.driver.T4CTTIoauthenticate.doOAUTH(T4CTTIoauthenticate.java:752)
at oracle.jdbc.driver.T4CConnection.logon(T4CConnection.java:366)
... 18 more

Solution: This problem occurs primarily due to the lack of a fast randomnumber generation device on the host where the map tasks execute. Ontypical Linux systems this can be addressed by setting the followingproperty in thejava.security file:

securerandom.source=file:/dev/../dev/urandom

The java.security file can be found under $JAVA_HOME/jre/lib/securitydirectory. Alternatively, this property can also be specified on thecommand line via:

-D mapred.child.java.opts="-Djava.security.egd=file:/dev/../dev/urandom"

Please note that it’s very important to specify this weird path /dev/../dev/urandomas it is due to a Java bug6202721,or/dev/urandom will be ignored and substituted by /dev/random.

27.2.2. Oracle: Case-Sensitive Catalog Query Errors

Problem: While working with Oracle you may encounter problems when Sqoop cannot figure out column names. This happens because the catalog queries thatSqoop uses for Oracle expect the correct case to be specified for theuser name and table name.

One example, using --hive-import and resulting in a NullPointerException:

1/09/21 17:18:49 INFO manager.OracleManager: Time zone has been set to
GMT
11/09/21 17:18:49 DEBUG manager.SqlManager: Using fetchSize for next
query: 1000
11/09/21 17:18:49 INFO manager.SqlManager: Executing SQL statement:
SELECT t.* FROM addlabel_pris t WHERE 1=0
11/09/21 17:18:49 DEBUG manager.OracleManager$ConnCache: Caching
released connection for jdbc:oracle:thin:
11/09/21 17:18:49 ERROR sqoop.Sqoop: Got exception running Sqoop:
java.lang.NullPointerException
java.lang.NullPointerException
at com.cloudera.sqoop.hive.TableDefWriter.getCreateTableStmt(TableDefWriter.java:148)
at com.cloudera.sqoop.hive.HiveImport.importTable(HiveImport.java:187)
at com.cloudera.sqoop.tool.ImportTool.importTable(ImportTool.java:362)
at com.cloudera.sqoop.tool.ImportTool.run(ImportTool.java:423)
at com.cloudera.sqoop.Sqoop.run(Sqoop.java:144)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65)
at com.cloudera.sqoop.Sqoop.runSqoop(Sqoop.java:180)
at com.cloudera.sqoop.Sqoop.runTool(Sqoop.java:219)
at com.cloudera.sqoop.Sqoop.runTool(Sqoop.java:228)
at com.cloudera.sqoop.Sqoop.main(Sqoop.java:237)

Solution:

  1. Specify the user name, which Sqoop is connecting as, in upper case (unlessit was created with mixed/lower case within quotes).
  2. Specify the table name, which you are working with, in upper case (unlessit was created with mixed/lower case within quotes).
27.2.3. MySQL: Connection Failure

Problem: While importing a MySQL table into Sqoop, if you do not havethe necessary permissions to access your MySQL database over the network,you may get the below connection failure.

Caused by: com.mysql.jdbc.exceptions.jdbc4.CommunicationsException: Communications link failure

Solution: First, verify that you can connect to the database from the node whereyou are running Sqoop:

$ mysql --host=<IP Address> --database=test --user=<username> --password=<password>

If this works, it rules out any problem with the client network configurationor security/authentication configuration.

Add the network port for the server to your my.cnf file /etc/my.cnf:

[mysqld]
port = xxxx

Set up a user account to connect via Sqoop.Grant permissions to the user to access the database over the network:(1.) Log into MySQL as rootmysql -u root -p<ThisIsMyPassword>.(2.) Issue the following command:

mysql> grant all privileges on test.* to 'testuser'@'%' identified by 'testpassword'

Note that doing this will enable the testuser to connect to theMySQL server from any IP address. While this will work, it is notadvisable for a production environment. We advise consulting with yourDBA to grant the necessary privileges based on the setup topology.

If the database server’s IP address changes, unless it is bound toa static hostname in your server, the connect string passed into Sqoopwill also need to be changed.

27.2.4. Oracle: ORA-00933 error (SQL command not properly ended)

Problem: While working with Oracle you may encounter the below problemwhen the Sqoop command explicitly specifies the --driver<driver name> option. When the driver option is included inthe Sqoop command, the built-in connection manager selection defaults to thegeneric connection manager, which causes this issue with Oracle. If thedriver option is not specified, the built-in connection manager selectionmechanism selects the Oracle specific connection manager which generatesvalid SQL for Oracle and uses the driver "oracle.jdbc.OracleDriver".

ERROR manager.SqlManager: Error executing statement:
java.sql.SQLSyntaxErrorException: ORA-00933: SQL command not properly ended

Solution: Omit the option --driver oracle.jdbc.driver.OracleDriver and thenre-run the Sqoop command.

27.2.5. MySQL: Import of TINYINT(1) from MySQL behaves strangely

Problem: Sqoop is treating TINYINT(1) columns as booleans, which is for examplecausing issues with HIVE import. This is because by default the MySQL JDBC connectormaps the TINYINT(1) to java.sql.Types.BIT, which Sqoop by default maps to Boolean.

Solution: A more clean solution is to force MySQL JDBC Connector to stopconverting TINYINT(1) to java.sql.Types.BIT by addingtinyInt1isBit=false into yourJDBC path (to create something likejdbc:mysql://localhost/test?tinyInt1isBit=false).Another solution would be to explicitly override the column mapping for the datatypeTINYINT(1) column. For example, if the column name is foo, then pass the followingoption to Sqoop during import: --map-column-hive foo=tinyint. In the case of non-Hiveimports to HDFS, use --map-column-java foo=integer.

评论
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

当前余额3.43前往充值 >
需支付:10.00
成就一亿技术人!
领取后你会自动成为博主和红包主的粉丝 规则
hope_wisdom
发出的红包
实付
使用余额支付
点击重新获取
扫码支付
钱包余额 0

抵扣说明:

1.余额是钱包充值的虚拟货币,按照1:1的比例进行支付金额的抵扣。
2.余额无法直接购买下载,可以购买VIP、付费专栏及课程。

余额充值