mirror of
https://github.com/apache/sqoop.git
synced 2025-05-02 20:09:32 +08:00
SQOOP-1322: Enhance Sqoop HCatalog Integration to cover features introduced in newer Hive versions
(Venkat Ranganathan via Jarek Jarcec Cecho)
This commit is contained in:
parent
1bc8b6bfda
commit
c65b56f6f2
44
build.xml
44
build.xml
@ -43,6 +43,39 @@
|
||||
</else>
|
||||
</if>
|
||||
|
||||
<!--
|
||||
Set default HCatalog profile if not set
|
||||
|
||||
In Hive 0.12, the HCatalog artifcacts moved from org.apache.hcatalog
|
||||
to org.apache.hive.hcatalog. With the advent of hive 0.14, the older
|
||||
hcatalog interfaces used by Sqoop are being removed. So, we are upgrading
|
||||
the hcatalog version to 0.13. This will allow as to support the hcatalog
|
||||
features in terms of datatype parity.
|
||||
|
||||
hcatalog 0.13 will be the default for now.
|
||||
|
||||
-->
|
||||
<if>
|
||||
<isset property="hcatprofile" />
|
||||
<then>
|
||||
<if>
|
||||
<or>
|
||||
<equals arg1="${hcatprofile}" arg2="13" />
|
||||
</or>
|
||||
<then>
|
||||
<echo message="Using HCatalog profile ${hcatprofile}" />
|
||||
</then>
|
||||
<else>
|
||||
<fail message="Invalid value for hcatprofile" />
|
||||
</else>
|
||||
</if>
|
||||
</then>
|
||||
<else>
|
||||
<echo message="Using HCatalog profile 0.13" />
|
||||
<property name="hcatprofile" value="13" />
|
||||
</else>
|
||||
</if>
|
||||
|
||||
<!--
|
||||
Set default Accumulo version
|
||||
|
||||
@ -96,7 +129,7 @@
|
||||
<property name="hbase94.version" value="0.90.3-cdh3u1" />
|
||||
<property name="zookeeper.version" value="3.3.3-cdh3u1" />
|
||||
<property name="hadoop.version.full" value="0.20" />
|
||||
<property name="hcatalog.version" value="0.11.0" />
|
||||
<property name="hcatalog.version" value="0.13.0" />
|
||||
<property name="hbasecompatprofile" value="1" />
|
||||
</then>
|
||||
|
||||
@ -107,7 +140,7 @@
|
||||
<property name="hbase94.version" value="0.92.0" />
|
||||
<property name="zookeeper.version" value="3.4.2" />
|
||||
<property name="hadoop.version.full" value="0.23" />
|
||||
<property name="hcatalog.version" value="0.11.0" />
|
||||
<property name="hcatalog.version" value="0.13.0" />
|
||||
<property name="hbasecompatprofile" value="2" />
|
||||
</then>
|
||||
</elseif>
|
||||
@ -119,7 +152,7 @@
|
||||
<property name="hbase94.version" value="0.92.0" />
|
||||
<property name="zookeeper.version" value="3.4.2" />
|
||||
<property name="hadoop.version.full" value="1.0.0" />
|
||||
<property name="hcatalog.version" value="0.11.0" />
|
||||
<property name="hcatalog.version" value="0.13.0" />
|
||||
<property name="hbasecompatprofile" value="1" />
|
||||
</then>
|
||||
</elseif>
|
||||
@ -131,7 +164,7 @@
|
||||
<property name="hbase94.version" value="0.94.2" />
|
||||
<property name="zookeeper.version" value="3.4.2" />
|
||||
<property name="hadoop.version.full" value="2.0.4-alpha" />
|
||||
<property name="hcatalog.version" value="0.11.0" />
|
||||
<property name="hcatalog.version" value="0.13.0" />
|
||||
<property name="hbasecompatprofile" value="2" />
|
||||
</then>
|
||||
</elseif>
|
||||
@ -143,11 +176,10 @@
|
||||
<property name="hbase94.version" value="0.94.2" />
|
||||
<property name="zookeeper.version" value="3.4.2" />
|
||||
<property name="hadoop.version.full" value="2.1.0-beta" />
|
||||
<property name="hcatalog.version" value="0.11.0" />
|
||||
<property name="hcatalog.version" value="0.13.0" />
|
||||
<property name="hbasecompatprofile" value="2" />
|
||||
</then>
|
||||
</elseif>
|
||||
|
||||
<else>
|
||||
<fail message="Unrecognized hadoopversion. Can only be 20, 23, 100, 200 or 210." />
|
||||
</else>
|
||||
|
18
ivy.xml
18
ivy.xml
@ -41,17 +41,17 @@ under the License.
|
||||
<conf name="hbase95" visibility="private" extends="hbasecompat${hbasecompatprofile}" />
|
||||
<conf name="hbasecompat1" visibility="private" />
|
||||
<conf name="hbasecompat2" visibility="private" />
|
||||
<conf name="hcatalog" visibility="private" />
|
||||
<conf name="hcatalog13" visibility="private" />
|
||||
<conf name="hadoop23" visibility="private"
|
||||
extends="common,runtime,hbase${hbaseprofile},hcatalog,accumulo" />
|
||||
extends="common,runtime,hbase${hbaseprofile},hcatalog${hcatprofile},accumulo" />
|
||||
<conf name="hadoop20" visibility="private"
|
||||
extends="common,runtime,hbase${hbaseprofile},hcatalog,accumulo" />
|
||||
extends="common,runtime,hbase${hbaseprofile},hcatalog${hcatprofile},accumulo" />
|
||||
<conf name="hadoop100" visibility="private"
|
||||
extends="common,runtime,hbase${hbaseprofile},hcatalog,accumulo" />
|
||||
extends="common,runtime,hbase${hbaseprofile},hcatalog${hcatprofile},accumulo" />
|
||||
<conf name="hadoop200" visibility="private"
|
||||
extends="common,runtime,hbase${hbaseprofile},hcatalog,accumulo" />
|
||||
extends="common,runtime,hbase${hbaseprofile},hcatalog${hcatprofile},accumulo" />
|
||||
<conf name="hadoop210" visibility="private"
|
||||
extends="common,runtime,hbase${hbaseprofile},hcatalog,accumulo" />
|
||||
extends="common,runtime,hbase${hbaseprofile},hcatalog${hcatprofile},accumulo" />
|
||||
|
||||
<conf name="test" visibility="private" extends="common,runtime"/>
|
||||
<conf name="hadoop23test" visibility="private" extends="test,hadoop23" />
|
||||
@ -293,9 +293,9 @@ under the License.
|
||||
</dependency>
|
||||
|
||||
|
||||
<dependency org="org.apache.hcatalog" name="hcatalog-core"
|
||||
rev="${hcatalog.version}" conf="hcatalog->default">
|
||||
<artifact name="hcatalog-core" type="jar"/>
|
||||
<dependency org="org.apache.hive.hcatalog" name="hive-hcatalog-core"
|
||||
rev="${hcatalog.version}" conf="hcatalog13->default">
|
||||
<artifact name="hive-hcatalog-core" type="jar"/>
|
||||
</dependency>
|
||||
|
||||
<dependency org="org.postgresql" name="postgresql"
|
||||
|
@ -43,8 +43,8 @@ Exposing HCatalog Tables to Sqoop
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
|
||||
HCatalog integration with Sqoop is patterned on an existing feature set that
|
||||
supports Avro and Hive tables. Five new command line options are introduced,
|
||||
and some command line options defined for Hive are reused.
|
||||
supports Avro and Hive tables. Seven new command line options are introduced,
|
||||
and some command line options defined for Hive have been reused.
|
||||
|
||||
New Command Line Options
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
@ -84,6 +84,32 @@ below.
|
||||
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/value
|
||||
pairs. In the prior releases, +--hive-partition-key+ and
|
||||
+--hive-partition-value+ options were used to specify the static partition
|
||||
key/value pair, but only one level of static partition keys could be provided.
|
||||
The options +--hcatalog-partition-keys+ and +--hcatalog-partition-values+
|
||||
allow 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 are
|
||||
defined with partition key names year, month and date and a specific partition
|
||||
with year=1999, month=12, day=31 is the desired partition, then the values
|
||||
for 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 be
|
||||
provided or neither of the options should be provided.
|
||||
|
||||
Supported Sqoop Hive Options
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
@ -102,9 +128,13 @@ The Hive home location.
|
||||
+--hive-partition-key+::
|
||||
Used for static partitioning filter. The partitioning key should be of
|
||||
type 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.
|
||||
|
||||
Direct Mode support
|
||||
^^^^^^^^^^^^^^^^^^^
|
||||
@ -167,7 +197,7 @@ HCatalog import jobs provide an option that lets a user specifiy the
|
||||
storage format for the created table.
|
||||
|
||||
The option +--create-hcatalog-table+ is used as an indicator that a table
|
||||
has to be created as part of the HCatalog import job. If the option
|
||||
has to be created as part of the HCatalog import job. If the option
|
||||
+--create-hcatalog-table+ is specified and the table exists, then the
|
||||
table creation will fail and the job will be aborted.
|
||||
|
||||
@ -253,9 +283,10 @@ float and double) is assignable to another field of any number type during
|
||||
exports and imports. Depending on the precision and scale of the target type
|
||||
of assignment, truncations can occur.
|
||||
|
||||
Furthermore, date/time/timestamps are mapped to string (the full
|
||||
date/time/timestamp representation) or bigint (the number of milliseconds
|
||||
since epoch) during imports and exports.
|
||||
Furthermore, date/time/timestamps are mapped to date/timestamp hive types.
|
||||
(the full date/time/timestamp representation). Date/time/timstamp columns
|
||||
can also be mapped to bigint Hive type in which case the value will be
|
||||
the number of milliseconds since epoch.
|
||||
|
||||
BLOBs and CLOBs are only supported for imports. The BLOB/CLOB objects when
|
||||
imported are stored in a Sqoop-specific format and knowledge of this format
|
||||
@ -272,15 +303,15 @@ partitioning columns, if any, must be part of the projection when importing
|
||||
data into HCatalog tables.
|
||||
|
||||
Dynamic partitioning fields should be mapped to database columns that are
|
||||
defined with the NOT NULL attribute (although this is not validated). A null
|
||||
value during import for a dynamic partitioning column will abort the Sqoop
|
||||
job.
|
||||
defined with the NOT NULL attribute (although this is not enforced during
|
||||
schema mapping). A null value during import for a dynamic partitioning
|
||||
column will abort the Sqoop job.
|
||||
|
||||
Support for HCatalog Data Types
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
|
||||
All the primitive HCatalog types are supported. Currently all the complex
|
||||
HCatalog types are unsupported.
|
||||
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.
|
||||
|
||||
|
@ -168,7 +168,10 @@ public String toString() {
|
||||
private String hCatStorageStanza;
|
||||
private String hCatHome; // not serialized to metastore.
|
||||
private boolean skipDistCache;
|
||||
|
||||
@StoredAsProperty("hcatalog.partition.keys")
|
||||
private String hCatalogPartitionKeys;
|
||||
@StoredAsProperty("hcatalog.partition.values")
|
||||
private String hCatalogPartitionValues;
|
||||
// User explicit mapping of types
|
||||
private Properties mapColumnJava; // stored as map.colum.java
|
||||
private Properties mapColumnHive; // stored as map.column.hive
|
||||
@ -2465,4 +2468,20 @@ public void setRelaxedIsolation(boolean b) {
|
||||
public boolean getRelaxedIsolation() {
|
||||
return this.relaxedIsolation;
|
||||
}
|
||||
|
||||
public String getHCatalogPartitionKeys() {
|
||||
return hCatalogPartitionKeys;
|
||||
}
|
||||
|
||||
public void setHCatalogPartitionKeys(String hpks) {
|
||||
this.hCatalogPartitionKeys = hpks;
|
||||
}
|
||||
|
||||
public String getHCatalogPartitionValues() {
|
||||
return hCatalogPartitionValues;
|
||||
}
|
||||
|
||||
public void setHCatalogPartitionValues(String hpvs) {
|
||||
this.hCatalogPartitionValues = hpvs;
|
||||
}
|
||||
}
|
||||
|
@ -36,6 +36,7 @@
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.sqoop.mapreduce.hcat.SqoopHCatUtilities;
|
||||
|
||||
import com.cloudera.sqoop.SqoopOptions;
|
||||
import com.cloudera.sqoop.hive.HiveTypes;
|
||||
@ -172,60 +173,7 @@ public String toHiveType(int sqlType) {
|
||||
* @return hcat type
|
||||
*/
|
||||
public String toHCatType(int sqlType) {
|
||||
switch (sqlType) {
|
||||
|
||||
// Ideally TINYINT and SMALLINT should be mapped to their
|
||||
// HCat equivalents tinyint and smallint respectively
|
||||
// But the Sqoop Java type conversion has them mapped to Integer
|
||||
// Even though the referenced Java doc clearly recommends otherwise.
|
||||
// Chaning this now can cause many of the sequence file usages to
|
||||
// break as value class implementations will change. So, we
|
||||
// just use the same behavior here.
|
||||
case Types.SMALLINT:
|
||||
case Types.TINYINT:
|
||||
case Types.INTEGER:
|
||||
return "int";
|
||||
|
||||
case Types.VARCHAR:
|
||||
case Types.CHAR:
|
||||
case Types.LONGVARCHAR:
|
||||
case Types.NVARCHAR:
|
||||
case Types.NCHAR:
|
||||
case Types.LONGNVARCHAR:
|
||||
case Types.DATE:
|
||||
case Types.TIME:
|
||||
case Types.TIMESTAMP:
|
||||
case Types.CLOB:
|
||||
return "string";
|
||||
|
||||
case Types.FLOAT:
|
||||
case Types.REAL:
|
||||
return "float";
|
||||
|
||||
case Types.NUMERIC:
|
||||
case Types.DECIMAL:
|
||||
return "string";
|
||||
|
||||
case Types.DOUBLE:
|
||||
return "double";
|
||||
|
||||
case Types.BIT:
|
||||
case Types.BOOLEAN:
|
||||
return "boolean";
|
||||
|
||||
case Types.BIGINT:
|
||||
return "bigint";
|
||||
|
||||
case Types.BINARY:
|
||||
case Types.VARBINARY:
|
||||
case Types.BLOB:
|
||||
case Types.LONGVARBINARY:
|
||||
return "binary";
|
||||
|
||||
default:
|
||||
throw new IllegalArgumentException(
|
||||
"Cannot convert SQL type to HCatalog type " + sqlType);
|
||||
}
|
||||
return SqoopHCatUtilities.toHCatType(sqlType);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -358,6 +306,71 @@ public Map<String, Integer> getColumnTypes(String tableName,
|
||||
return columnTypes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return an unordered mapping from colname to sqltype, precision and scale
|
||||
* for all columns in a table.
|
||||
*
|
||||
* Precision and scale are as defined in the resultset metadata,
|
||||
*
|
||||
* The Integer type id is a constant from java.sql.Types
|
||||
*/
|
||||
public Map<String, List<Integer>> getColumnInfo(String tableName) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Get column information is not supported by this manager");
|
||||
}
|
||||
/**
|
||||
* Return an unordered mapping from colname to sqltype, precision and scale
|
||||
* all the input arguments for a stored procedure.
|
||||
*
|
||||
* Precision and scale are as defined in the resultset metadata,
|
||||
*
|
||||
* The Integer type id is a constant from java.sql.Types
|
||||
*/
|
||||
public Map<String, List<Integer>> getColumnInfoForProcedure(
|
||||
String procedureName) {
|
||||
throw new UnsupportedOperationException(
|
||||
"No stored procedure support for this database");
|
||||
}
|
||||
/**
|
||||
* Return an unordered mapping from colname to sqltype, precision and scale
|
||||
* for all columns in a query.
|
||||
*
|
||||
* Precision and scale are as defined in the resultset metadata,
|
||||
*
|
||||
* The Integer type id is a constant from java.sql.Types
|
||||
*/
|
||||
public Map<String, List<Integer>> getColumnInfoForQuery(String query) {
|
||||
LOG.error("This database does not support free-form query column info.");
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return an unordered mapping from colname to sqltype, precision and scale
|
||||
* for all columns in a table or query.
|
||||
*
|
||||
* The Integer type id is a constant from java.sql.Types
|
||||
* Precision and scale are as defined in the resultset metadata,
|
||||
* @param tableName the name of the table
|
||||
* @param sqlQuery the SQL query to use if tableName is null
|
||||
*/
|
||||
public Map<String, List<Integer>> getColumnInfo(String tableName,
|
||||
String sqlQuery) throws IOException {
|
||||
Map<String, List<Integer>> colInfo;
|
||||
if (null != tableName) {
|
||||
// We're generating a class based on a table import.
|
||||
colInfo = getColumnInfo(tableName);
|
||||
} else {
|
||||
// This is based on an arbitrary query.
|
||||
String query = sqlQuery;
|
||||
if (query.indexOf(SqlManager.SUBSTITUTE_TOKEN) == -1) {
|
||||
throw new IOException("Query [" + query + "] must contain '"
|
||||
+ SqlManager.SUBSTITUTE_TOKEN + "' in WHERE clause.");
|
||||
}
|
||||
colInfo = getColumnInfoForQuery(query);
|
||||
}
|
||||
return colInfo;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return an unordered mapping from colname to sql type name for
|
||||
* all columns in a table.
|
||||
|
@ -42,9 +42,9 @@
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.sqoop.accumulo.AccumuloUtil;
|
||||
import org.apache.sqoop.mapreduce.AccumuloImportJob;
|
||||
import org.apache.sqoop.mapreduce.HBaseBulkImportJob;
|
||||
import org.apache.sqoop.mapreduce.JdbcCallExportJob;
|
||||
import org.apache.sqoop.util.LoggingUtils;
|
||||
import org.apache.sqoop.mapreduce.HBaseBulkImportJob;
|
||||
import org.apache.sqoop.util.SqlTypeMap;
|
||||
|
||||
import com.cloudera.sqoop.SqoopOptions;
|
||||
@ -237,8 +237,34 @@ public Map<String, Integer> getColumnTypesForQuery(String query) {
|
||||
* Get column types for a query statement that we do not modify further.
|
||||
*/
|
||||
protected Map<String, Integer> getColumnTypesForRawQuery(String stmt) {
|
||||
Map<String, List<Integer>> colInfo = getColumnInfoForRawQuery(stmt);
|
||||
if (colInfo == null) {
|
||||
return null;
|
||||
}
|
||||
Map<String, Integer> colTypes = new SqlTypeMap<String, Integer>();
|
||||
for (String s : colInfo.keySet()) {
|
||||
List<Integer> info = colInfo.get(s);
|
||||
colTypes.put(s, info.get(0));
|
||||
}
|
||||
return colTypes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, List<Integer>> getColumnInfo(String tableName) {
|
||||
String stmt = getColNamesQuery(tableName);
|
||||
return getColumnInfoForRawQuery(stmt);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, List<Integer>> getColumnInfoForQuery(String query) {
|
||||
// Manipulate the query to return immediately, with zero rows.
|
||||
String rawQuery = query.replace(SUBSTITUTE_TOKEN, " (1 = 0) ");
|
||||
return getColumnInfoForRawQuery(rawQuery);
|
||||
}
|
||||
|
||||
protected Map<String, List<Integer>> getColumnInfoForRawQuery(String stmt) {
|
||||
ResultSet results;
|
||||
LOG.debug("Execute getColumnTypesRawQuery : " + stmt);
|
||||
LOG.debug("Execute getColumnInfoRawQuery : " + stmt);
|
||||
try {
|
||||
results = execute(stmt);
|
||||
} catch (SQLException sqlE) {
|
||||
@ -249,12 +275,16 @@ protected Map<String, Integer> getColumnTypesForRawQuery(String stmt) {
|
||||
}
|
||||
|
||||
try {
|
||||
Map<String, Integer> colTypes = new SqlTypeMap<String, Integer>();
|
||||
Map<String, List<Integer>> colInfo =
|
||||
new SqlTypeMap<String, List<Integer>>();
|
||||
|
||||
int cols = results.getMetaData().getColumnCount();
|
||||
ResultSetMetaData metadata = results.getMetaData();
|
||||
for (int i = 1; i < cols + 1; i++) {
|
||||
int typeId = metadata.getColumnType(i);
|
||||
int precision = metadata.getPrecision(i);
|
||||
int scale = metadata.getScale(i);
|
||||
|
||||
// If we have an unsigned int we need to make extra room by
|
||||
// plopping it into a bigint
|
||||
if (typeId == Types.INTEGER && !metadata.isSigned(i)){
|
||||
@ -265,11 +295,14 @@ protected Map<String, Integer> getColumnTypesForRawQuery(String stmt) {
|
||||
if (colName == null || colName.equals("")) {
|
||||
colName = metadata.getColumnName(i);
|
||||
}
|
||||
|
||||
colTypes.put(colName, Integer.valueOf(typeId));
|
||||
List<Integer> info = new ArrayList<Integer>(3);
|
||||
info.add(Integer.valueOf(typeId));
|
||||
info.add(precision);
|
||||
info.add(scale);
|
||||
colInfo.put(colName, info);
|
||||
}
|
||||
|
||||
return colTypes;
|
||||
return colInfo;
|
||||
} catch (SQLException sqlException) {
|
||||
LoggingUtils.logAll(LOG, "Error reading from database: "
|
||||
+ sqlException.toString(), sqlException);
|
||||
@ -381,7 +414,23 @@ public String[] listDatabases() {
|
||||
|
||||
@Override
|
||||
public Map<String, Integer> getColumnTypesForProcedure(String procedureName) {
|
||||
Map<String, Integer> ret = new TreeMap<String, Integer>();
|
||||
Map<String, List<Integer>> colInfo =
|
||||
getColumnInfoForProcedure(procedureName);
|
||||
if (colInfo == null) {
|
||||
return null;
|
||||
}
|
||||
Map<String, Integer> colTypes = new SqlTypeMap<String, Integer>();
|
||||
for (String s : colInfo.keySet()) {
|
||||
List<Integer> info = colInfo.get(s);
|
||||
colTypes.put(s, info.get(0));
|
||||
}
|
||||
return colTypes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, List<Integer>>
|
||||
getColumnInfoForProcedure(String procedureName) {
|
||||
Map<String, List<Integer>> ret = new TreeMap<String, List<Integer>>();
|
||||
try {
|
||||
DatabaseMetaData metaData = this.getConnection().getMetaData();
|
||||
ResultSet results = metaData.getProcedureColumns(null, null,
|
||||
@ -398,9 +447,11 @@ public Map<String, Integer> getColumnTypesForProcedure(String procedureName) {
|
||||
// we don't care if we get several rows for the
|
||||
// same ORDINAL_POSITION (e.g. like H2 gives us)
|
||||
// as we'll just overwrite the entry in the map:
|
||||
ret.put(
|
||||
results.getString("COLUMN_NAME"),
|
||||
results.getInt("DATA_TYPE"));
|
||||
List<Integer> info = new ArrayList<Integer>(3);
|
||||
info.add(results.getInt("DATA_TYPE"));
|
||||
info.add(results.getInt("PRECISION"));
|
||||
info.add(results.getInt("SCALE"));
|
||||
ret.put(results.getString("COLUMN_NAME"), info);
|
||||
}
|
||||
}
|
||||
LOG.debug("Columns returned = " + StringUtils.join(ret.keySet(), ","));
|
||||
|
@ -23,7 +23,7 @@
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hcatalog.data.HCatRecord;
|
||||
import org.apache.hive.hcatalog.data.HCatRecord;
|
||||
import org.apache.sqoop.lib.DelimiterSet;
|
||||
import org.apache.sqoop.lib.SqoopRecord;
|
||||
import org.apache.sqoop.mapreduce.hcat.SqoopHCatExportHelper;
|
||||
|
@ -23,7 +23,7 @@
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.apache.hcatalog.data.HCatRecord;
|
||||
import org.apache.hive.hcatalog.data.HCatRecord;
|
||||
import org.apache.sqoop.config.ConfigurationHelper;
|
||||
import org.apache.sqoop.lib.RecordParser;
|
||||
import org.apache.sqoop.lib.SqoopRecord;
|
||||
|
@ -21,8 +21,6 @@
|
||||
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.sqoop.lib.SqoopRecord;
|
||||
import org.apache.sqoop.mapreduce.hcat.SqoopHCatImportHelper;
|
||||
|
||||
/**
|
||||
* Netezza import mapper using external tables for text formats.
|
||||
|
@ -31,8 +31,8 @@
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.hadoop.mapreduce.RecordReader;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hcatalog.data.HCatRecord;
|
||||
import org.apache.hcatalog.mapreduce.HCatInputFormat;
|
||||
import org.apache.hive.hcatalog.data.HCatRecord;
|
||||
import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
|
||||
import org.apache.sqoop.mapreduce.ExportInputFormat;
|
||||
|
||||
/**
|
||||
|
@ -27,20 +27,22 @@
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hive.common.type.HiveDecimal;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.DefaultStringifier;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.MapWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.apache.hcatalog.common.HCatConstants;
|
||||
import org.apache.hcatalog.common.HCatUtil;
|
||||
import org.apache.hcatalog.data.HCatRecord;
|
||||
import org.apache.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hcatalog.data.schema.HCatSchema;
|
||||
import org.apache.hcatalog.mapreduce.InputJobInfo;
|
||||
import org.apache.hive.hcatalog.common.HCatConstants;
|
||||
import org.apache.hive.hcatalog.common.HCatUtil;
|
||||
import org.apache.hive.hcatalog.data.HCatRecord;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatSchema;
|
||||
import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
|
||||
import org.apache.sqoop.lib.SqoopRecord;
|
||||
import org.apache.sqoop.mapreduce.ExportJobBase;
|
||||
import org.apache.sqoop.mapreduce.ImportJobBase;
|
||||
|
||||
/**
|
||||
* Helper class for Sqoop HCat Integration export jobs.
|
||||
@ -51,6 +53,7 @@ public class SqoopHCatExportHelper {
|
||||
public static final Log LOG = LogFactory
|
||||
.getLog(SqoopHCatExportHelper.class.getName());
|
||||
private SqoopRecord sqoopRecord;
|
||||
private boolean bigDecimalFormatString;
|
||||
private static final String TIMESTAMP_TYPE = "java.sql.Timestamp";
|
||||
private static final String TIME_TYPE = "java.sql.Time";
|
||||
private static final String DATE_TYPE = "java.sql.Date";
|
||||
@ -85,6 +88,11 @@ public SqoopHCatExportHelper(Configuration conf)
|
||||
+ ExportJobBase.SQOOP_EXPORT_TABLE_CLASS_KEY
|
||||
+ ") is not set!");
|
||||
}
|
||||
|
||||
bigDecimalFormatString = conf.getBoolean(
|
||||
ImportJobBase.PROPERTY_BIGDECIMAL_FORMAT,
|
||||
ImportJobBase.PROPERTY_BIGDECIMAL_FORMAT_DEFAULT);
|
||||
|
||||
debugHCatExportMapper = conf.getBoolean(
|
||||
SqoopHCatUtilities.DEBUG_HCAT_EXPORT_MAPPER_PROP, false);
|
||||
try {
|
||||
@ -181,7 +189,29 @@ private Object convertToSqoop(Object val,
|
||||
}
|
||||
}
|
||||
break;
|
||||
case DATE:
|
||||
Date date = (Date) val;
|
||||
if (javaColType.equals(DATE_TYPE)) {
|
||||
return date;
|
||||
} else if (javaColType.equals(TIME_TYPE)) {
|
||||
return new Time(date.getTime());
|
||||
} else if (javaColType.equals(TIMESTAMP_TYPE)) {
|
||||
return new Timestamp(date.getTime());
|
||||
}
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
Timestamp ts = (Timestamp) val;
|
||||
if (javaColType.equals(DATE_TYPE)) {
|
||||
return new Date(ts.getTime());
|
||||
} else if (javaColType.equals(TIME_TYPE)) {
|
||||
return new Time(ts.getTime());
|
||||
} else if (javaColType.equals(TIMESTAMP_TYPE)) {
|
||||
return ts;
|
||||
}
|
||||
break;
|
||||
case STRING:
|
||||
case VARCHAR:
|
||||
case CHAR:
|
||||
val = convertStringTypes(val, javaColType);
|
||||
if (val != null) {
|
||||
return val;
|
||||
@ -193,6 +223,12 @@ private Object convertToSqoop(Object val,
|
||||
return val;
|
||||
}
|
||||
break;
|
||||
case DECIMAL:
|
||||
val = convertDecimalTypes(val, javaColType);
|
||||
if (val != null) {
|
||||
return val;
|
||||
}
|
||||
break;
|
||||
case ARRAY:
|
||||
case MAP:
|
||||
case STRUCT:
|
||||
@ -206,6 +242,23 @@ private Object convertToSqoop(Object val,
|
||||
return null;
|
||||
}
|
||||
|
||||
private Object convertDecimalTypes(Object val, String javaColType) {
|
||||
HiveDecimal hd = (HiveDecimal) val;
|
||||
BigDecimal bd = hd.bigDecimalValue();
|
||||
|
||||
if (javaColType.equals(BIG_DECIMAL_TYPE)) {
|
||||
return bd;
|
||||
} else if (javaColType.equals(STRING_TYPE)) {
|
||||
String bdStr = null;
|
||||
if (bigDecimalFormatString) {
|
||||
bdStr = bd.toPlainString();
|
||||
} else {
|
||||
bdStr = bd.toString();
|
||||
}
|
||||
return bdStr;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
private Object convertBinaryTypes(Object val, String javaColType) {
|
||||
byte[] bb = (byte[]) val;
|
||||
if (javaColType.equals(BYTESWRITABLE)) {
|
||||
@ -225,7 +278,9 @@ private Object convertStringTypes(Object val, String javaColType) {
|
||||
|| javaColType.equals(TIMESTAMP_TYPE)) {
|
||||
// Oracle expects timestamps for Date also by default based on version
|
||||
// Just allow all date types to be assignment compatible
|
||||
if (valStr.length() == 10) { // Date in yyyy-mm-dd format
|
||||
if (valStr.length() == 10
|
||||
&& valStr.matches("^\\d{4}-\\d{2}-\\d{2}$")) {
|
||||
// Date in yyyy-mm-dd format
|
||||
Date d = Date.valueOf(valStr);
|
||||
if (javaColType.equals(DATE_TYPE)) {
|
||||
return d;
|
||||
@ -234,7 +289,9 @@ private Object convertStringTypes(Object val, String javaColType) {
|
||||
} else if (javaColType.equals(TIMESTAMP_TYPE)) {
|
||||
return new Timestamp(d.getTime());
|
||||
}
|
||||
} else if (valStr.length() == 8) { // time in hh:mm:ss
|
||||
} else if (valStr.length() == 8
|
||||
&& valStr.matches("^\\d{2}:\\d{2}:\\d{2}$")) {
|
||||
// time in hh:mm:ss
|
||||
Time t = Time.valueOf(valStr);
|
||||
if (javaColType.equals(DATE_TYPE)) {
|
||||
return new Date(t.getTime());
|
||||
@ -243,7 +300,11 @@ private Object convertStringTypes(Object val, String javaColType) {
|
||||
} else if (javaColType.equals(TIMESTAMP_TYPE)) {
|
||||
return new Timestamp(t.getTime());
|
||||
}
|
||||
} else if (valStr.length() == 19) { // timestamp in yyyy-mm-dd hh:ss:mm
|
||||
} else if (valStr.length() >= 19
|
||||
&& valStr.length() <= 26
|
||||
&& valStr.
|
||||
matches("^\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}(.\\d+)?$")) {
|
||||
// timestamp in yyyy-mm-dd hh:mm:ss
|
||||
Timestamp ts = Timestamp.valueOf(valStr);
|
||||
if (javaColType.equals(DATE_TYPE)) {
|
||||
return new Date(ts.getTime());
|
||||
|
@ -25,7 +25,7 @@
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
import org.apache.hcatalog.data.HCatRecord;
|
||||
import org.apache.hive.hcatalog.data.HCatRecord;
|
||||
import org.apache.sqoop.lib.SqoopRecord;
|
||||
import org.apache.sqoop.mapreduce.AutoProgressMapper;
|
||||
|
||||
|
@ -20,10 +20,12 @@
|
||||
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.math.MathContext;
|
||||
import java.sql.Date;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Time;
|
||||
import java.sql.Timestamp;
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
@ -31,17 +33,22 @@
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.common.type.HiveChar;
|
||||
import org.apache.hadoop.hive.common.type.HiveDecimal;
|
||||
import org.apache.hadoop.hive.common.type.HiveVarchar;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.DefaultStringifier;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hcatalog.common.HCatConstants;
|
||||
import org.apache.hcatalog.common.HCatUtil;
|
||||
import org.apache.hcatalog.data.DefaultHCatRecord;
|
||||
import org.apache.hcatalog.data.HCatRecord;
|
||||
import org.apache.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hcatalog.data.schema.HCatSchema;
|
||||
import org.apache.hcatalog.mapreduce.InputJobInfo;
|
||||
import org.apache.hcatalog.mapreduce.StorerInfo;
|
||||
import org.apache.hive.hcatalog.common.HCatConstants;
|
||||
import org.apache.hive.hcatalog.common.HCatUtil;
|
||||
import org.apache.hive.hcatalog.data.DefaultHCatRecord;
|
||||
import org.apache.hive.hcatalog.data.HCatRecord;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatSchema;
|
||||
import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
|
||||
import org.apache.hive.hcatalog.mapreduce.StorerInfo;
|
||||
import org.apache.sqoop.lib.SqoopRecord;
|
||||
import org.apache.sqoop.mapreduce.ImportJobBase;
|
||||
|
||||
@ -70,7 +77,7 @@ public class SqoopHCatImportHelper {
|
||||
private String hiveDelimsReplacement;
|
||||
private boolean doHiveDelimsReplacement = false;
|
||||
private DelimiterSet hiveDelimiters;
|
||||
private String staticPartitionKey;
|
||||
private String[] staticPartitionKeys;
|
||||
private int[] hCatFieldPositions;
|
||||
private int colCount;
|
||||
|
||||
@ -135,9 +142,11 @@ public SqoopHCatImportHelper(Configuration conf) throws IOException,
|
||||
LOG.debug("Hive delims replacement enabled : " + doHiveDelimsReplacement);
|
||||
LOG.debug("Hive Delimiters : " + hiveDelimiters.toString());
|
||||
LOG.debug("Hive delimiters replacement : " + hiveDelimsReplacement);
|
||||
staticPartitionKey = conf
|
||||
.get(SqoopHCatUtilities.HCAT_STATIC_PARTITION_KEY_PROP);
|
||||
LOG.debug("Static partition key used : " + staticPartitionKey);
|
||||
staticPartitionKeys = conf
|
||||
.getStrings(SqoopHCatUtilities.HCAT_STATIC_PARTITION_KEY_PROP);
|
||||
String partKeysString = staticPartitionKeys == null ? ""
|
||||
: Arrays.toString(staticPartitionKeys);
|
||||
LOG.debug("Static partition key used : " + partKeysString);
|
||||
}
|
||||
|
||||
public HCatRecord convertToHCatRecord(SqoopRecord sqr) throws IOException,
|
||||
@ -159,7 +168,16 @@ public HCatRecord convertToHCatRecord(SqoopRecord sqr) throws IOException,
|
||||
String key = entry.getKey();
|
||||
Object val = entry.getValue();
|
||||
String hfn = key.toLowerCase();
|
||||
if (staticPartitionKey != null && staticPartitionKey.equals(hfn)) {
|
||||
boolean skip = false;
|
||||
if (staticPartitionKeys != null && staticPartitionKeys.length > 0) {
|
||||
for (int i = 0; i < staticPartitionKeys.length; ++i) {
|
||||
if (staticPartitionKeys[i].equals(hfn)) {
|
||||
skip = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (skip) {
|
||||
continue;
|
||||
}
|
||||
HCatFieldSchema hfs = hCatFullTableSchema.get(hfn);
|
||||
@ -168,7 +186,7 @@ public HCatRecord convertToHCatRecord(SqoopRecord sqr) throws IOException,
|
||||
+ " of type " + (val == null ? null : val.getClass().getName())
|
||||
+ ", hcattype " + hfs.getTypeString());
|
||||
}
|
||||
Object hCatVal = toHCat(val, hfs.getType(), hfs.getTypeString());
|
||||
Object hCatVal = toHCat(val, hfs);
|
||||
|
||||
result.set(hfn, hCatFullTableSchema, hCatVal);
|
||||
}
|
||||
@ -176,9 +194,8 @@ public HCatRecord convertToHCatRecord(SqoopRecord sqr) throws IOException,
|
||||
return result;
|
||||
}
|
||||
|
||||
private Object toHCat(Object val, HCatFieldSchema.Type hfsType,
|
||||
String hCatTypeString) {
|
||||
|
||||
private Object toHCat(Object val, HCatFieldSchema hfs) {
|
||||
HCatFieldSchema.Type hfsType = hfs.getType();
|
||||
if (val == null) {
|
||||
return null;
|
||||
}
|
||||
@ -186,21 +203,13 @@ private Object toHCat(Object val, HCatFieldSchema.Type hfsType,
|
||||
Object retVal = null;
|
||||
|
||||
if (val instanceof Number) {
|
||||
retVal = convertNumberTypes(val, hfsType);
|
||||
retVal = convertNumberTypes(val, hfs);
|
||||
} else if (val instanceof Boolean) {
|
||||
retVal = convertBooleanTypes(val, hfsType);
|
||||
retVal = convertBooleanTypes(val, hfs);
|
||||
} else if (val instanceof String) {
|
||||
if (hfsType == HCatFieldSchema.Type.STRING) {
|
||||
String str = (String) val;
|
||||
if (doHiveDelimsReplacement) {
|
||||
retVal = FieldFormatter.hiveStringReplaceDelims(str,
|
||||
hiveDelimsReplacement, hiveDelimiters);
|
||||
} else {
|
||||
retVal = str;
|
||||
}
|
||||
}
|
||||
retVal = convertStringTypes(val, hfs);
|
||||
} else if (val instanceof java.util.Date) {
|
||||
retVal = converDateTypes(val, hfsType);
|
||||
retVal = converDateTypes(val, hfs);
|
||||
} else if (val instanceof BytesWritable) {
|
||||
if (hfsType == HCatFieldSchema.Type.BINARY) {
|
||||
BytesWritable bw = (BytesWritable) val;
|
||||
@ -214,46 +223,135 @@ private Object toHCat(Object val, HCatFieldSchema.Type hfsType,
|
||||
retVal = bytes;
|
||||
}
|
||||
} else if (val instanceof ClobRef) {
|
||||
if (hfsType == HCatFieldSchema.Type.STRING) {
|
||||
ClobRef cr = (ClobRef) val;
|
||||
String s = cr.isExternal() ? cr.toString() : cr.getData();
|
||||
retVal = s;
|
||||
}
|
||||
retVal = convertClobType(val, hfs);
|
||||
} else {
|
||||
throw new UnsupportedOperationException("Objects of type "
|
||||
+ val.getClass().getName() + " are not suported");
|
||||
}
|
||||
if (retVal == null) {
|
||||
LOG.error("Objects of type " + val.getClass().getName()
|
||||
+ " can not be mapped to HCatalog type " + hCatTypeString);
|
||||
LOG.error("Unable to convert [" + val
|
||||
+ "] of type " + val.getClass().getName()
|
||||
+ " to HCatalog type " + hfs.getTypeString());
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
private Object converDateTypes(Object val, HCatFieldSchema.Type hfsType) {
|
||||
private Object convertClobType(Object val, HCatFieldSchema hfs) {
|
||||
HCatFieldSchema.Type hfsType = hfs.getType();
|
||||
ClobRef cr = (ClobRef) val;
|
||||
String s = cr.isExternal() ? cr.toString() : cr.getData();
|
||||
|
||||
if (hfsType == HCatFieldSchema.Type.STRING) {
|
||||
return s;
|
||||
} else if (hfsType == HCatFieldSchema.Type.VARCHAR) {
|
||||
VarcharTypeInfo vti = (VarcharTypeInfo) hfs.getTypeInfo();
|
||||
HiveVarchar hvc = new HiveVarchar(s, vti.getLength());
|
||||
return hvc;
|
||||
} else if (hfsType == HCatFieldSchema.Type.CHAR) {
|
||||
CharTypeInfo cti = (CharTypeInfo) hfs.getTypeInfo();
|
||||
HiveChar hc = new HiveChar(s, cti.getLength());
|
||||
return hc;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private Object converDateTypes(Object val, HCatFieldSchema hfs) {
|
||||
HCatFieldSchema.Type hfsType = hfs.getType();
|
||||
Date d;
|
||||
Time t;
|
||||
Timestamp ts;
|
||||
if (val instanceof java.sql.Date) {
|
||||
if (hfsType == HCatFieldSchema.Type.BIGINT) {
|
||||
return ((Date) val).getTime();
|
||||
d = (Date) val;
|
||||
if (hfsType == HCatFieldSchema.Type.DATE) {
|
||||
return d;
|
||||
} else if (hfsType == HCatFieldSchema.Type.TIMESTAMP) {
|
||||
return new Timestamp(d.getTime());
|
||||
} else if (hfsType == HCatFieldSchema.Type.BIGINT) {
|
||||
return (d.getTime());
|
||||
} else if (hfsType == HCatFieldSchema.Type.STRING) {
|
||||
return val.toString();
|
||||
} else if (hfsType == HCatFieldSchema.Type.VARCHAR) {
|
||||
VarcharTypeInfo vti = (VarcharTypeInfo) hfs.getTypeInfo();
|
||||
HiveVarchar hvc = new HiveVarchar(val.toString(), vti.getLength());
|
||||
return hvc;
|
||||
} else if (hfsType == HCatFieldSchema.Type.CHAR) {
|
||||
CharTypeInfo cti = (CharTypeInfo) hfs.getTypeInfo();
|
||||
HiveChar hChar = new HiveChar(val.toString(), cti.getLength());
|
||||
return hChar;
|
||||
}
|
||||
} else if (val instanceof java.sql.Time) {
|
||||
if (hfsType == HCatFieldSchema.Type.BIGINT) {
|
||||
t = (Time) val;
|
||||
if (hfsType == HCatFieldSchema.Type.DATE) {
|
||||
return new Date(t.getTime());
|
||||
} else if (hfsType == HCatFieldSchema.Type.TIMESTAMP) {
|
||||
return new Timestamp(t.getTime());
|
||||
} else if (hfsType == HCatFieldSchema.Type.BIGINT) {
|
||||
return ((Time) val).getTime();
|
||||
} else if (hfsType == HCatFieldSchema.Type.STRING) {
|
||||
return val.toString();
|
||||
} else if (hfsType == HCatFieldSchema.Type.VARCHAR) {
|
||||
VarcharTypeInfo vti = (VarcharTypeInfo) hfs.getTypeInfo();
|
||||
HiveVarchar hvc = new HiveVarchar(val.toString(), vti.getLength());
|
||||
return hvc;
|
||||
} else if (hfsType == HCatFieldSchema.Type.CHAR) {
|
||||
CharTypeInfo cti = (CharTypeInfo) hfs.getTypeInfo();
|
||||
HiveChar hChar = new HiveChar(val.toString(), cti.getLength());
|
||||
return hChar;
|
||||
}
|
||||
} else if (val instanceof java.sql.Timestamp) {
|
||||
if (hfsType == HCatFieldSchema.Type.BIGINT) {
|
||||
return ((Timestamp) val).getTime();
|
||||
ts = (Timestamp) val;
|
||||
if (hfsType == HCatFieldSchema.Type.DATE) {
|
||||
return new Date(ts.getTime());
|
||||
} else if (hfsType == HCatFieldSchema.Type.TIMESTAMP) {
|
||||
return ts;
|
||||
} else if (hfsType == HCatFieldSchema.Type.BIGINT) {
|
||||
return ts.getTime();
|
||||
} else if (hfsType == HCatFieldSchema.Type.STRING) {
|
||||
return val.toString();
|
||||
} else if (hfsType == HCatFieldSchema.Type.VARCHAR) {
|
||||
VarcharTypeInfo vti = (VarcharTypeInfo) hfs.getTypeInfo();
|
||||
HiveVarchar hvc = new HiveVarchar(val.toString(), vti.getLength());
|
||||
return hvc;
|
||||
} else if (hfsType == HCatFieldSchema.Type.CHAR) {
|
||||
CharTypeInfo cti = (CharTypeInfo) hfs.getTypeInfo();
|
||||
HiveChar hc = new HiveChar(val.toString(), cti.getLength());
|
||||
return hc;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private Object convertBooleanTypes(Object val, HCatFieldSchema.Type hfsType) {
|
||||
private Object convertStringTypes(Object val, HCatFieldSchema hfs) {
|
||||
HCatFieldSchema.Type hfsType = hfs.getType();
|
||||
if (hfsType == HCatFieldSchema.Type.STRING
|
||||
|| hfsType == HCatFieldSchema.Type.VARCHAR
|
||||
|| hfsType == HCatFieldSchema.Type.CHAR) {
|
||||
String str = val.toString();
|
||||
if (doHiveDelimsReplacement) {
|
||||
str = FieldFormatter.hiveStringReplaceDelims(str,
|
||||
hiveDelimsReplacement, hiveDelimiters);
|
||||
}
|
||||
if (hfsType == HCatFieldSchema.Type.STRING) {
|
||||
return str;
|
||||
} else if (hfsType == HCatFieldSchema.Type.VARCHAR) {
|
||||
VarcharTypeInfo vti = (VarcharTypeInfo) hfs.getTypeInfo();
|
||||
HiveVarchar hvc = new HiveVarchar(str, vti.getLength());
|
||||
return hvc;
|
||||
} else if (hfsType == HCatFieldSchema.Type.CHAR) {
|
||||
CharTypeInfo cti = (CharTypeInfo) hfs.getTypeInfo();
|
||||
HiveChar hc = new HiveChar(val.toString(), cti.getLength());
|
||||
return hc;
|
||||
}
|
||||
} else if (hfsType == HCatFieldSchema.Type.DECIMAL) {
|
||||
BigDecimal bd = new BigDecimal(val.toString(), MathContext.DECIMAL128);
|
||||
HiveDecimal hd = HiveDecimal.create(bd);
|
||||
return hd;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private Object convertBooleanTypes(Object val, HCatFieldSchema hfs) {
|
||||
HCatFieldSchema.Type hfsType = hfs.getType();
|
||||
Boolean b = (Boolean) val;
|
||||
if (hfsType == HCatFieldSchema.Type.BOOLEAN) {
|
||||
return b;
|
||||
@ -271,20 +369,45 @@ private Object convertBooleanTypes(Object val, HCatFieldSchema.Type hfsType) {
|
||||
return (double) (b ? 1 : 0);
|
||||
} else if (hfsType == HCatFieldSchema.Type.STRING) {
|
||||
return val.toString();
|
||||
} else if (hfsType == HCatFieldSchema.Type.VARCHAR) {
|
||||
VarcharTypeInfo vti = (VarcharTypeInfo) hfs.getTypeInfo();
|
||||
HiveVarchar hvc = new HiveVarchar(val.toString(), vti.getLength());
|
||||
return hvc;
|
||||
} else if (hfsType == HCatFieldSchema.Type.CHAR) {
|
||||
CharTypeInfo cti = (CharTypeInfo) hfs.getTypeInfo();
|
||||
HiveChar hChar = new HiveChar(val.toString(), cti.getLength());
|
||||
return hChar;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private Object convertNumberTypes(Object val, HCatFieldSchema.Type hfsType) {
|
||||
private Object convertNumberTypes(Object val, HCatFieldSchema hfs) {
|
||||
HCatFieldSchema.Type hfsType = hfs.getType();
|
||||
|
||||
if (!(val instanceof Number)) {
|
||||
return null;
|
||||
}
|
||||
if (val instanceof BigDecimal && hfsType == HCatFieldSchema.Type.STRING) {
|
||||
if (val instanceof BigDecimal
|
||||
&& hfsType == HCatFieldSchema.Type.STRING
|
||||
|| hfsType == HCatFieldSchema.Type.VARCHAR
|
||||
|| hfsType == HCatFieldSchema.Type.CHAR) {
|
||||
BigDecimal bd = (BigDecimal) val;
|
||||
String bdStr = null;
|
||||
if (bigDecimalFormatString) {
|
||||
return bd.toPlainString();
|
||||
bdStr = bd.toPlainString();
|
||||
} else {
|
||||
return bd.toString();
|
||||
bdStr = bd.toString();
|
||||
}
|
||||
if (hfsType == HCatFieldSchema.Type.VARCHAR) {
|
||||
VarcharTypeInfo vti = (VarcharTypeInfo) hfs.getTypeInfo();
|
||||
HiveVarchar hvc = new HiveVarchar(bdStr, vti.getLength());
|
||||
return hvc;
|
||||
} else if (hfsType == HCatFieldSchema.Type.VARCHAR) {
|
||||
CharTypeInfo cti = (CharTypeInfo) hfs.getTypeInfo();
|
||||
HiveChar hChar = new HiveChar(bdStr, cti.getLength());
|
||||
return hChar;
|
||||
} else {
|
||||
return bdStr;
|
||||
}
|
||||
}
|
||||
Number n = (Number) val;
|
||||
@ -304,6 +427,18 @@ private Object convertNumberTypes(Object val, HCatFieldSchema.Type hfsType) {
|
||||
return n.byteValue() == 0 ? Boolean.FALSE : Boolean.TRUE;
|
||||
} else if (hfsType == HCatFieldSchema.Type.STRING) {
|
||||
return n.toString();
|
||||
} else if (hfsType == HCatFieldSchema.Type.VARCHAR) {
|
||||
VarcharTypeInfo vti = (VarcharTypeInfo) hfs.getTypeInfo();
|
||||
HiveVarchar hvc = new HiveVarchar(val.toString(), vti.getLength());
|
||||
return hvc;
|
||||
} else if (hfsType == HCatFieldSchema.Type.CHAR) {
|
||||
CharTypeInfo cti = (CharTypeInfo) hfs.getTypeInfo();
|
||||
HiveChar hChar = new HiveChar(val.toString(), cti.getLength());
|
||||
return hChar;
|
||||
} else if (hfsType == HCatFieldSchema.Type.DECIMAL) {
|
||||
BigDecimal bd = new BigDecimal(n.doubleValue(),
|
||||
MathContext.DECIMAL128);
|
||||
return HiveDecimal.create(bd);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -24,7 +24,7 @@
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
import org.apache.hcatalog.data.HCatRecord;
|
||||
import org.apache.hive.hcatalog.data.HCatRecord;
|
||||
import org.apache.sqoop.lib.SqoopRecord;
|
||||
import org.apache.sqoop.mapreduce.SqoopMapper;
|
||||
|
||||
|
@ -29,7 +29,7 @@
|
||||
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapreduce.InputSplit;
|
||||
import org.apache.hcatalog.mapreduce.HCatSplit;
|
||||
import org.apache.hive.hcatalog.mapreduce.HCatSplit;
|
||||
|
||||
/**
|
||||
* An abstraction of a combined HCatSplits.
|
||||
|
@ -26,7 +26,7 @@
|
||||
import org.apache.hadoop.mapreduce.InputSplit;
|
||||
import org.apache.hadoop.mapreduce.RecordReader;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hcatalog.data.HCatRecord;
|
||||
import org.apache.hive.hcatalog.data.HCatRecord;
|
||||
|
||||
/**
|
||||
* A Record Reader that can combine underlying splits.
|
||||
|
@ -40,6 +40,9 @@
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.shims.HadoopShims;
|
||||
import org.apache.hadoop.hive.shims.HadoopShims.HCatHadoopShims;
|
||||
import org.apache.hadoop.hive.shims.ShimLoader;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.DefaultStringifier;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
@ -52,13 +55,13 @@
|
||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||
import org.apache.hadoop.util.Shell;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hcatalog.common.HCatConstants;
|
||||
import org.apache.hcatalog.data.DefaultHCatRecord;
|
||||
import org.apache.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hcatalog.data.schema.HCatSchema;
|
||||
import org.apache.hcatalog.mapreduce.HCatInputFormat;
|
||||
import org.apache.hcatalog.mapreduce.HCatOutputFormat;
|
||||
import org.apache.hcatalog.mapreduce.OutputJobInfo;
|
||||
import org.apache.hive.hcatalog.common.HCatConstants;
|
||||
import org.apache.hive.hcatalog.data.DefaultHCatRecord;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatSchema;
|
||||
import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
|
||||
import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
|
||||
import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
|
||||
import org.apache.sqoop.config.ConfigurationConstants;
|
||||
import org.apache.sqoop.hive.HiveTypes;
|
||||
import org.apache.sqoop.manager.ConnManager;
|
||||
@ -89,7 +92,7 @@ public final class SqoopHCatUtilities {
|
||||
public static final String HCAT_DB_OUTPUT_COLTYPES_SQL =
|
||||
"sqoop.hcat.db.output.coltypes.sql";
|
||||
public static final String HCAT_CLI_MAIN_CLASS =
|
||||
"org.apache.hcatalog.cli.HCatCli";
|
||||
"org.apache.hive.hcatalog.cli.HCatCli";
|
||||
public static final String HCAT_DEF_STORAGE_STANZA = "stored as rcfile";
|
||||
public static final String HIVE_DELIMITERS_TO_REPLACE_PROP =
|
||||
"sqoop.hive.delims.to.replace";
|
||||
@ -106,6 +109,9 @@ public final class SqoopHCatUtilities {
|
||||
public static final String DEBUG_HCAT_EXPORT_MAPPER_PROP =
|
||||
"sqoop.hcat.debug.export.mapper";
|
||||
private static final String HCATCMD = Shell.WINDOWS ? "hcat.py" : "hcat";
|
||||
private static final int MAX_HIVE_CHAR_PREC = 65535;
|
||||
private static final int MAX_HIVE_DECIMAL_PREC = 65;
|
||||
private static final int MAX_HIVE_DECIMAL_SCALE = 30;
|
||||
private SqoopOptions options;
|
||||
private ConnManager connManager;
|
||||
private String hCatTableName;
|
||||
@ -118,14 +124,13 @@ public final class SqoopHCatUtilities {
|
||||
private boolean configured;
|
||||
|
||||
private String hCatQualifiedTableName;
|
||||
private String hCatStaticPartitionKey;
|
||||
private List<String> hCatStaticPartitionKeys;
|
||||
private List<String> hCatStaticPartitionValues;
|
||||
private List<String> hCatDynamicPartitionKeys;
|
||||
// DB stuff
|
||||
private String[] dbColumnNames;
|
||||
private String dbTableName;
|
||||
private LCKeyMap<Integer> dbColumnTypes;
|
||||
|
||||
private Map<String, Integer> externalColTypes;
|
||||
private LCKeyMap<List<Integer>> dbColumnInfo;
|
||||
|
||||
private int[] hCatFieldPositions; // For each DB column, HCat position
|
||||
|
||||
@ -222,14 +227,7 @@ public boolean isConfigured() {
|
||||
return configured;
|
||||
}
|
||||
|
||||
public void configureHCat(final SqoopOptions opts, final Job job,
|
||||
final ConnManager connMgr, final String dbTable,
|
||||
final Configuration config) throws IOException {
|
||||
if (configured) {
|
||||
LOG.info("Ignoring configuration request for HCatalog info");
|
||||
return;
|
||||
}
|
||||
options = opts;
|
||||
private void checkHomeDirs(SqoopOptions opts) {
|
||||
|
||||
LOG.info("Configuring HCatalog specific details for job");
|
||||
|
||||
@ -238,7 +236,7 @@ public void configureHCat(final SqoopOptions opts, final Job job,
|
||||
if (home == null || home.length() == 0) {
|
||||
LOG.warn("Hive home is not set. job may fail if needed jar files "
|
||||
+ "are not found correctly. Please set HIVE_HOME in"
|
||||
+ " sqoop-env.sh or provide --hive-home option. Setting HIVE_HOME "
|
||||
+ " sqoop-env.sh or provide --hive-home option. HIVE_HOME defaulted "
|
||||
+ " to " + SqoopOptions.getHiveHomeDefault());
|
||||
}
|
||||
|
||||
@ -247,8 +245,19 @@ public void configureHCat(final SqoopOptions opts, final Job job,
|
||||
LOG.warn("HCatalog home is not set. job may fail if needed jar "
|
||||
+ "files are not found correctly. Please set HCAT_HOME in"
|
||||
+ " sqoop-env.sh or provide --hcatalog-home option. "
|
||||
+ " Setting HCAT_HOME to " + SqoopOptions.getHCatHomeDefault());
|
||||
+ "HCAT_HOME defaulted to " + SqoopOptions.getHCatHomeDefault());
|
||||
}
|
||||
}
|
||||
|
||||
public void configureHCat(final SqoopOptions opts, final Job job,
|
||||
final ConnManager connMgr, final String dbTable,
|
||||
final Configuration config) throws IOException {
|
||||
if (configured) {
|
||||
LOG.info("Ignoring configuration request for HCatalog info");
|
||||
return;
|
||||
}
|
||||
options = opts;
|
||||
checkHomeDirs(opts);
|
||||
connManager = connMgr;
|
||||
dbTableName = dbTable;
|
||||
configuration = config;
|
||||
@ -275,8 +284,24 @@ public void configureHCat(final SqoopOptions opts, final Job job,
|
||||
if (principalID != null) {
|
||||
configuration.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
|
||||
}
|
||||
hCatStaticPartitionKey = options.getHivePartitionKey();
|
||||
hCatStaticPartitionKeys = new ArrayList<String>();
|
||||
hCatStaticPartitionValues = new ArrayList<String>();
|
||||
String partKeysString = options.getHCatalogPartitionKeys();
|
||||
String partKeysVals = options.getHCatalogPartitionValues();
|
||||
if (partKeysString != null) {
|
||||
String[] keys = partKeysString.split(",");
|
||||
hCatStaticPartitionKeys.addAll(Arrays.asList(keys));
|
||||
String[] vals = partKeysVals.split(",");
|
||||
hCatStaticPartitionValues.addAll(Arrays.asList(vals));
|
||||
} else {
|
||||
partKeysString = options.getHivePartitionKey();
|
||||
if (partKeysString != null) {
|
||||
hCatStaticPartitionKeys.add(partKeysString);
|
||||
}
|
||||
partKeysVals = options.getHivePartitionValue();
|
||||
hCatStaticPartitionValues.add(partKeysVals);
|
||||
|
||||
}
|
||||
Properties userMapping = options.getMapColumnHive();
|
||||
userHiveMapping = new LCKeyMap<String>();
|
||||
for (Object o : userMapping.keySet()) {
|
||||
@ -286,7 +311,7 @@ public void configureHCat(final SqoopOptions opts, final Job job,
|
||||
// Get the partition key filter if needed
|
||||
Map<String, String> filterMap = getHCatSPFilterMap();
|
||||
String filterStr = getHCatSPFilterStr();
|
||||
initDBColumnNamesAndTypes();
|
||||
initDBColumnInfo();
|
||||
if (options.doCreateHCatalogTable()) {
|
||||
LOG.info("Creating HCatalog table " + hCatQualifiedTableName
|
||||
+ " for import");
|
||||
@ -345,8 +370,8 @@ public void configureHCat(final SqoopOptions opts, final Job job,
|
||||
throw new IOException("Database column " + col + " not found in "
|
||||
+ " hcatalog table.");
|
||||
}
|
||||
if (hCatStaticPartitionKey != null
|
||||
&& hCatStaticPartitionKey.equals(col)) {
|
||||
if (hCatStaticPartitionKeys != null
|
||||
&& hCatStaticPartitionKeys.equals(col)) {
|
||||
continue;
|
||||
}
|
||||
outputFieldList.add(hCatFullTableSchema.get(col));
|
||||
@ -415,17 +440,10 @@ public void validateHCatTableFieldTypes() throws IOException {
|
||||
/**
|
||||
* Get the column names to import.
|
||||
*/
|
||||
private void initDBColumnNamesAndTypes() throws IOException {
|
||||
private void initDBColumnInfo() throws IOException {
|
||||
String[] colNames = options.getColumns();
|
||||
if (null == colNames) {
|
||||
if (null != externalColTypes) {
|
||||
// Test-injection column mapping. Extract the col names from
|
||||
ArrayList<String> keyList = new ArrayList<String>();
|
||||
for (String key : externalColTypes.keySet()) {
|
||||
keyList.add(key);
|
||||
}
|
||||
colNames = keyList.toArray(new String[keyList.size()]);
|
||||
} else if (null != dbTableName) {
|
||||
if (null != dbTableName) {
|
||||
colNames = connManager.getColumnNames(dbTableName);
|
||||
} else if (options.getCall() != null) {
|
||||
// Read procedure arguments from metadata
|
||||
@ -442,56 +460,97 @@ private void initDBColumnNamesAndTypes() throws IOException {
|
||||
dbColumnNames[i] = colNames[i].toLowerCase();
|
||||
}
|
||||
|
||||
LCKeyMap<Integer> colTypes = new LCKeyMap<Integer>();
|
||||
if (externalColTypes != null) { // Use pre-defined column types.
|
||||
colTypes.putAll(externalColTypes);
|
||||
} else { // Get these from the database.
|
||||
if (dbTableName != null) {
|
||||
colTypes.putAll(connManager.getColumnTypes(dbTableName));
|
||||
} else if (options.getCall() != null) {
|
||||
LCKeyMap<List<Integer>> colInfo = new LCKeyMap<List<Integer>>();
|
||||
if (dbTableName != null) {
|
||||
colInfo.putAll(connManager.getColumnInfo(dbTableName));
|
||||
} else if (options.getCall() != null) {
|
||||
// Read procedure arguments from metadata
|
||||
colTypes.putAll(connManager.getColumnTypesForProcedure(this.options
|
||||
colInfo.putAll(connManager.getColumnInfoForProcedure(this.options
|
||||
.getCall()));
|
||||
} else {
|
||||
colTypes.putAll(connManager.getColumnTypesForQuery(options
|
||||
} else {
|
||||
colInfo.putAll(connManager.getColumnInfoForQuery(options
|
||||
.getSqlQuery()));
|
||||
}
|
||||
}
|
||||
|
||||
if (options.getColumns() == null) {
|
||||
dbColumnTypes = colTypes;
|
||||
dbColumnInfo = colInfo;
|
||||
} else {
|
||||
dbColumnTypes = new LCKeyMap<Integer>();
|
||||
dbColumnInfo = new LCKeyMap<List<Integer>>();
|
||||
// prune column types based on projection
|
||||
for (String col : dbColumnNames) {
|
||||
Integer type = colTypes.get(col);
|
||||
if (type == null) {
|
||||
List<Integer> info = colInfo.get(col);
|
||||
if (info == null) {
|
||||
throw new IOException("Projected column " + col
|
||||
+ " not in list of columns from database");
|
||||
}
|
||||
dbColumnTypes.put(col, type);
|
||||
dbColumnInfo.put(col, info);
|
||||
}
|
||||
}
|
||||
LOG.info("Database column names projected : "
|
||||
+ Arrays.toString(dbColumnNames));
|
||||
LOG.info("Database column name - type map :\n\tNames: "
|
||||
+ Arrays.toString(dbColumnTypes.keySet().toArray()) + "\n\tTypes : "
|
||||
+ Arrays.toString(dbColumnTypes.values().toArray()));
|
||||
StringBuilder sb = new StringBuilder(1024);
|
||||
for (String col : dbColumnInfo.keySet()) {
|
||||
sb.append('\t').append(col).append(" : [");
|
||||
List<Integer> info = dbColumnInfo.get(col);
|
||||
sb.append("Type : ").append(info.get(0)).append(',');
|
||||
sb.append("Precision : ").append(info.get(1)).append(',');
|
||||
sb.append("Scale : ").append(info.get(2)).append(']');
|
||||
sb.append('\n');
|
||||
}
|
||||
LOG.info("Database column name - info map :\n" + sb.toString());
|
||||
}
|
||||
|
||||
public static StringBuilder escHCatObj(String objectName) {
|
||||
StringBuilder sb = new StringBuilder(64);
|
||||
sb.append('`').append(objectName).append('`');
|
||||
return sb;
|
||||
}
|
||||
|
||||
private void createHCatTable() throws IOException {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("create table ").
|
||||
append(hCatDatabaseName).append('.');
|
||||
sb.append(hCatTableName).append(" (\n\t");
|
||||
append(escHCatObj(hCatDatabaseName)).append('.');
|
||||
sb.append(escHCatObj(hCatTableName)).append(" (\n\t");
|
||||
boolean first = true;
|
||||
for (String col : dbColumnNames) {
|
||||
String type = userHiveMapping.get(col);
|
||||
int prec = -1;
|
||||
int scale = -1;
|
||||
if (type == null) {
|
||||
type = connManager.toHCatType(dbColumnTypes.get(col));
|
||||
type = connManager.toHCatType(dbColumnInfo.get(col).get(0));
|
||||
}
|
||||
if (hCatStaticPartitionKey != null
|
||||
&& col.equals(hCatStaticPartitionKey)) {
|
||||
if (type.equals("char") || type.equals("varchar")) {
|
||||
prec = dbColumnInfo.get(col).get(1);
|
||||
if (prec > MAX_HIVE_CHAR_PREC) {
|
||||
LOG.warn("Truncating precison of column " + col + " from " + prec
|
||||
+ " to " + MAX_HIVE_CHAR_PREC);
|
||||
prec = MAX_HIVE_CHAR_PREC;
|
||||
}
|
||||
} else if (type.equals("decimal")) {
|
||||
prec = dbColumnInfo.get(col).get(1);
|
||||
if (prec > MAX_HIVE_DECIMAL_PREC) {
|
||||
LOG.warn("Truncating precison of column " + col + " from " + prec
|
||||
+ " to " + MAX_HIVE_DECIMAL_PREC);
|
||||
prec = MAX_HIVE_DECIMAL_PREC;
|
||||
}
|
||||
|
||||
scale = dbColumnInfo.get(col).get(2);
|
||||
if (scale > MAX_HIVE_DECIMAL_SCALE) {
|
||||
LOG.warn("Truncating precison of column " + col + " from " + scale
|
||||
+ " to " + MAX_HIVE_DECIMAL_SCALE);
|
||||
scale = MAX_HIVE_DECIMAL_SCALE;
|
||||
}
|
||||
}
|
||||
boolean skip=false;
|
||||
if (hCatStaticPartitionKeys != null) {
|
||||
for (String key : hCatStaticPartitionKeys) {
|
||||
if (col.equals(key)) {
|
||||
skip=true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (skip) {
|
||||
continue;
|
||||
}
|
||||
if (first) {
|
||||
@ -499,12 +558,28 @@ private void createHCatTable() throws IOException {
|
||||
} else {
|
||||
sb.append(",\n\t");
|
||||
}
|
||||
sb.append(col).append(' ').append(type);
|
||||
sb.append(escHCatObj(col)).append(' ').append(type);
|
||||
if (prec > 0) {
|
||||
sb.append('(').append(prec);
|
||||
if (scale > 0) {
|
||||
sb.append(',').append(scale);
|
||||
}
|
||||
sb.append(')');
|
||||
}
|
||||
}
|
||||
sb.append(")\n");
|
||||
if (hCatStaticPartitionKey != null) {
|
||||
first = true;
|
||||
if (hCatStaticPartitionKeys != null && hCatStaticPartitionKeys.size() > 0) {
|
||||
sb.append("partitioned by (\n\t");
|
||||
sb.append(hCatStaticPartitionKey).append(" string)\n");
|
||||
for (String key : hCatStaticPartitionKeys) {
|
||||
if (first) {
|
||||
first = false;
|
||||
} else {
|
||||
sb.append(", ");
|
||||
}
|
||||
sb.append(escHCatObj(key)).append(" string");
|
||||
}
|
||||
sb.append(")\n");
|
||||
}
|
||||
String storageStanza = options.getHCatStorageStanza();
|
||||
if (storageStanza == null) {
|
||||
@ -545,11 +620,11 @@ private void validateFieldAndColumnMappings() throws IOException {
|
||||
for (int indx = 0; indx < dbColumnNames.length; ++indx) {
|
||||
boolean userMapped = false;
|
||||
String col = dbColumnNames[indx];
|
||||
Integer colType = dbColumnTypes.get(col);
|
||||
List<Integer> colInfo = dbColumnInfo.get(col);
|
||||
String hCatColType = userHiveMapping.get(col);
|
||||
if (hCatColType == null) {
|
||||
LOG.debug("No user defined type mapping for HCatalog field " + col);
|
||||
hCatColType = connManager.toHCatType(colType);
|
||||
hCatColType = connManager.toHCatType(colInfo.get(0));
|
||||
} else {
|
||||
LOG.debug("Found type mapping for HCatalog filed " + col);
|
||||
userMapped = true;
|
||||
@ -576,13 +651,14 @@ private void validateFieldAndColumnMappings() throws IOException {
|
||||
if (!hCatFS.getTypeString().equals(hCatColType)) {
|
||||
LOG.warn("The HCatalog field " + col + " has type "
|
||||
+ hCatFS.getTypeString() + ". Expected = " + hCatColType
|
||||
+ " based on database column type : " + sqlTypeString(colType));
|
||||
+ " based on database column type : "
|
||||
+ sqlTypeString(colInfo.get(0)));
|
||||
LOG.warn("The Sqoop job can fail if types are not "
|
||||
+ " assignment compatible");
|
||||
}
|
||||
}
|
||||
|
||||
if (HiveTypes.isHiveTypeImprovised(colType)) {
|
||||
if (HiveTypes.isHiveTypeImprovised(colInfo.get(0))) {
|
||||
LOG.warn("Column " + col + " had to be cast to a less precise type "
|
||||
+ hCatColType + " in hcatalog");
|
||||
}
|
||||
@ -603,20 +679,33 @@ private void validateFieldAndColumnMappings() throws IOException {
|
||||
}
|
||||
|
||||
private String getHCatSPFilterStr() {
|
||||
if (hCatStaticPartitionKey != null) {
|
||||
if (hCatStaticPartitionKeys != null && hCatStaticPartitionKeys.size() > 0) {
|
||||
StringBuilder filter = new StringBuilder();
|
||||
filter.append(options.getHivePartitionKey()).append('=').append('\'')
|
||||
.append(options.getHivePartitionValue()).append('\'');
|
||||
boolean first = true;
|
||||
for (int i = 0; i < hCatStaticPartitionKeys.size(); ++i) {
|
||||
String key = hCatStaticPartitionKeys.get(i);
|
||||
String val = hCatStaticPartitionValues.get(i);
|
||||
if (first) {
|
||||
first = false;
|
||||
} else {
|
||||
filter.append(" AND ");
|
||||
}
|
||||
filter.append(key).append('=').append('\'')
|
||||
.append(val).append('\'');
|
||||
}
|
||||
return filter.toString();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private Map<String, String> getHCatSPFilterMap() {
|
||||
if (hCatStaticPartitionKey != null) {
|
||||
if (hCatStaticPartitionKeys != null && hCatStaticPartitionKeys.size() > 0) {
|
||||
Map<String, String> filter = new HashMap<String, String>();
|
||||
filter
|
||||
.put(options.getHivePartitionKey(), options.getHivePartitionValue());
|
||||
for (int i = 0; i < hCatStaticPartitionKeys.size(); ++i) {
|
||||
String key = hCatStaticPartitionKeys.get(i);
|
||||
String val = hCatStaticPartitionValues.get(i);
|
||||
filter.put(key, val);
|
||||
}
|
||||
return filter;
|
||||
}
|
||||
return null;
|
||||
@ -626,27 +715,34 @@ private void validateStaticPartitionKey() throws IOException {
|
||||
// check the static partition key from command line
|
||||
List<HCatFieldSchema> partFields = hCatPartitionSchema.getFields();
|
||||
|
||||
if (hCatStaticPartitionKey != null) {
|
||||
boolean found = false;
|
||||
for (HCatFieldSchema hfs : partFields) {
|
||||
if (hfs.getName().equals(hCatStaticPartitionKey)) {
|
||||
found = true;
|
||||
break;
|
||||
if (hCatStaticPartitionKeys != null
|
||||
&& hCatStaticPartitionKeys.size() > 0) {
|
||||
for (String key : hCatStaticPartitionKeys) {
|
||||
boolean found = false;
|
||||
for (HCatFieldSchema hfs : partFields) {
|
||||
if (hfs.getName().equals(key)) {
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!found) {
|
||||
throw new IOException("The provided hive partition key "
|
||||
+ hCatStaticPartitionKey + " is not part of the partition "
|
||||
if (!found) {
|
||||
throw new IOException("The provided hive partition key "
|
||||
+ key + " is not part of the partition "
|
||||
+ " keys for table " + getQualifiedHCatTableName());
|
||||
}
|
||||
}
|
||||
}
|
||||
hCatDynamicPartitionKeys = new ArrayList<String>();
|
||||
hCatDynamicPartitionKeys.addAll(hCatPartitionSchema.getFieldNames());
|
||||
if (hCatStaticPartitionKey != null) {
|
||||
hCatDynamicPartitionKeys.remove(hCatStaticPartitionKey);
|
||||
if (hCatStaticPartitionKeys != null
|
||||
&& hCatStaticPartitionKeys.size() > 0) {
|
||||
for (String key : hCatStaticPartitionKeys) {
|
||||
hCatDynamicPartitionKeys.remove(key);
|
||||
}
|
||||
}
|
||||
configuration.set(HCAT_STATIC_PARTITION_KEY_PROP,
|
||||
hCatStaticPartitionKey == null ? "" : hCatStaticPartitionKey);
|
||||
configuration.setStrings(HCAT_STATIC_PARTITION_KEY_PROP,
|
||||
hCatStaticPartitionKeys.toArray(
|
||||
new String[hCatStaticPartitionKeys.size()]));
|
||||
}
|
||||
|
||||
public static void configureImportOutputFormat(SqoopOptions opts, Job job,
|
||||
@ -700,18 +796,18 @@ public static void configureExportInputFormat(SqoopOptions opts, Job job,
|
||||
hCatUtils
|
||||
.configureHCat(opts, job, connMgr, dbTable, job.getConfiguration());
|
||||
job.setInputFormatClass(getInputFormatClass());
|
||||
Map<String, Integer> dbColTypes = hCatUtils.getDbColumnTypes();
|
||||
Map<String, List<Integer>> dbColInfo = hCatUtils.getDbColumnInfo();
|
||||
MapWritable columnTypesJava = new MapWritable();
|
||||
for (Map.Entry<String, Integer> e : dbColTypes.entrySet()) {
|
||||
for (Map.Entry<String, List<Integer>> e : dbColInfo.entrySet()) {
|
||||
Text columnName = new Text(e.getKey());
|
||||
Text columnText = new Text(connMgr.toJavaType(dbTable, e.getKey(),
|
||||
e.getValue()));
|
||||
e.getValue().get(0)));
|
||||
columnTypesJava.put(columnName, columnText);
|
||||
}
|
||||
MapWritable columnTypesSql = new MapWritable();
|
||||
for (Map.Entry<String, Integer> e : dbColTypes.entrySet()) {
|
||||
for (Map.Entry<String, List<Integer>> e : dbColInfo.entrySet()) {
|
||||
Text columnName = new Text(e.getKey());
|
||||
IntWritable sqlType = new IntWritable(e.getValue());
|
||||
IntWritable sqlType = new IntWritable(e.getValue().get(0));
|
||||
columnTypesSql.put(columnName, sqlType);
|
||||
}
|
||||
DefaultStringifier.store(config, columnTypesJava,
|
||||
@ -836,6 +932,71 @@ public static boolean isHadoop1() {
|
||||
}
|
||||
return false;
|
||||
}
|
||||
/**
|
||||
* Resolve a database-specific type to HCat data type. Largely follows Sqoop's
|
||||
* hive translation.
|
||||
* @param sqlType
|
||||
* sql type
|
||||
* @return hcat type
|
||||
*/
|
||||
public static String toHCatType(int sqlType) {
|
||||
switch (sqlType) {
|
||||
|
||||
// Ideally TINYINT and SMALLINT should be mapped to their
|
||||
// HCat equivalents tinyint and smallint respectively
|
||||
// But the Sqoop Java type conversion has them mapped to Integer
|
||||
// Even though the referenced Java doc clearly recommends otherwise.
|
||||
// Changing this now can cause many of the sequence file usages to
|
||||
// break as value class implementations will change. So, we
|
||||
// just use the same behavior here.
|
||||
case Types.SMALLINT:
|
||||
case Types.TINYINT:
|
||||
case Types.INTEGER:
|
||||
return "int";
|
||||
|
||||
case Types.VARCHAR:
|
||||
return "varchar";
|
||||
case Types.CHAR:
|
||||
return "char";
|
||||
case Types.LONGVARCHAR:
|
||||
case Types.NVARCHAR:
|
||||
case Types.NCHAR:
|
||||
case Types.LONGNVARCHAR:
|
||||
case Types.DATE:
|
||||
case Types.TIME:
|
||||
case Types.TIMESTAMP:
|
||||
case Types.CLOB:
|
||||
return "string";
|
||||
|
||||
case Types.FLOAT:
|
||||
case Types.REAL:
|
||||
return "float";
|
||||
|
||||
case Types.NUMERIC:
|
||||
case Types.DECIMAL:
|
||||
return "decimal";
|
||||
|
||||
case Types.DOUBLE:
|
||||
return "double";
|
||||
|
||||
case Types.BIT:
|
||||
case Types.BOOLEAN:
|
||||
return "boolean";
|
||||
|
||||
case Types.BIGINT:
|
||||
return "bigint";
|
||||
|
||||
case Types.BINARY:
|
||||
case Types.VARBINARY:
|
||||
case Types.BLOB:
|
||||
case Types.LONGVARBINARY:
|
||||
return "binary";
|
||||
|
||||
default:
|
||||
throw new IllegalArgumentException(
|
||||
"Cannot convert SQL type to HCatalog type " + sqlType);
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean isLocalJobTracker(Job job) {
|
||||
Configuration conf = job.getConfiguration();
|
||||
@ -857,51 +1018,10 @@ public void invokeOutputCommitterForLocalMode(Job job) throws IOException {
|
||||
// HCatalog 0.11- do have special class HCatHadoopShims, however this
|
||||
// class got merged into Hive Shim layer in 0.12+. Following method will
|
||||
// try to find correct implementation via reflection.
|
||||
|
||||
// Final Shim layer
|
||||
Object shimLayer = null;
|
||||
Class shimClass = null;
|
||||
|
||||
// Let's try Hive 0.11-
|
||||
HadoopShims shims = ShimLoader.getHadoopShims();
|
||||
HCatHadoopShims hcatShims = shims.getHCatShim();
|
||||
try {
|
||||
shimClass = Class.forName("org.apache.hcatalog.shims.HCatHadoopShims");
|
||||
|
||||
Class shimInstanceClass = Class.forName("org.apache.hcatalog.shims.HCatHadoopShims$Instance");
|
||||
Method getMethod = shimInstanceClass.getMethod("get");
|
||||
|
||||
shimLayer = getMethod.invoke(null);
|
||||
} catch (Exception e) {
|
||||
LOG.debug("Not found HCatalog 0.11- implementation of the Shim layer", e);
|
||||
}
|
||||
|
||||
// For Hive 0.12+
|
||||
if (shimClass == null || shimLayer == null) {
|
||||
try {
|
||||
shimClass = Class.forName("org.apache.hadoop.hive.shims.HadoopShims$HCatHadoopShims");
|
||||
|
||||
Class shimLoader = Class.forName("org.apache.hadoop.hive.shims.ShimLoader");
|
||||
Method getHadoopShims = shimLoader.getMethod("getHadoopShims");
|
||||
|
||||
Object hadoopShims = getHadoopShims.invoke(null);
|
||||
|
||||
Class hadoopShimClass = Class.forName("org.apache.hadoop.hive.shims.HadoopShims");
|
||||
Method getHCatShim = hadoopShimClass.getMethod("getHCatShim");
|
||||
|
||||
shimLayer = getHCatShim.invoke(hadoopShims);
|
||||
} catch (Exception e) {
|
||||
LOG.debug("Not found HCatalog 0.12+ implementation of the Shim layer", e);
|
||||
}
|
||||
}
|
||||
|
||||
if (shimClass == null || shimLayer == null) {
|
||||
throw new IOException("Did not found HCatalog shim layer to commit the job");
|
||||
}
|
||||
|
||||
// Part that is the same for both shim layer implementations
|
||||
try {
|
||||
Method commitJobMethod = shimClass.getMethod("commitJob", OutputFormat.class, Job.class);
|
||||
LOG.info("Explicitly committing job in local mode");
|
||||
commitJobMethod.invoke(shimLayer, new HCatOutputFormat(), job);
|
||||
hcatShims.commitJob(new HCatOutputFormat(), job);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException("Can't explicitly commit job", e);
|
||||
}
|
||||
@ -1097,14 +1217,6 @@ public static void setImportValueClass(Class<? extends Writable> clz) {
|
||||
importValueClass = clz;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the column type map to be used. (dependency injection for testing; not
|
||||
* used in production.)
|
||||
*/
|
||||
public void setColumnTypes(Map<String, Integer> colTypes) {
|
||||
externalColTypes = colTypes;
|
||||
LOG.debug("Using test-controlled type map");
|
||||
}
|
||||
|
||||
public String getDatabaseTable() {
|
||||
return dbTableName;
|
||||
@ -1126,8 +1238,8 @@ public List<String> getHCatDynamicPartitionKeys() {
|
||||
return hCatDynamicPartitionKeys;
|
||||
}
|
||||
|
||||
public String getHCatStaticPartitionKey() {
|
||||
return hCatStaticPartitionKey;
|
||||
public List<String> getHCatStaticPartitionKey() {
|
||||
return hCatStaticPartitionKeys;
|
||||
}
|
||||
|
||||
public String[] getDBColumnNames() {
|
||||
@ -1150,8 +1262,8 @@ public void setHCatPartitionSchema(HCatSchema schema) {
|
||||
hCatPartitionSchema = schema;
|
||||
}
|
||||
|
||||
public void setHCatStaticPartitionKey(String key) {
|
||||
hCatStaticPartitionKey = key;
|
||||
public void setHCatStaticPartitionKey(List<String> keys) {
|
||||
hCatStaticPartitionKeys = keys;
|
||||
}
|
||||
|
||||
public void setHCatDynamicPartitionKeys(List<String> keys) {
|
||||
@ -1166,12 +1278,8 @@ public void setDbColumnNames(String[] names) {
|
||||
dbColumnNames = names;
|
||||
}
|
||||
|
||||
public Map<String, Integer> getDbColumnTypes() {
|
||||
return dbColumnTypes;
|
||||
}
|
||||
|
||||
public void setDbColumnTypes(Map<String, Integer> types) {
|
||||
dbColumnTypes.putAll(types);
|
||||
public Map<String, List<Integer>> getDbColumnInfo() {
|
||||
return dbColumnInfo;
|
||||
}
|
||||
|
||||
public String gethCatTableName() {
|
||||
|
@ -33,6 +33,8 @@
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.sqoop.util.CredentialsUtil;
|
||||
import org.apache.sqoop.util.LoggingUtils;
|
||||
|
||||
import com.cloudera.sqoop.ConnFactory;
|
||||
import com.cloudera.sqoop.Sqoop;
|
||||
@ -43,8 +45,6 @@
|
||||
import com.cloudera.sqoop.lib.DelimiterSet;
|
||||
import com.cloudera.sqoop.manager.ConnManager;
|
||||
import com.cloudera.sqoop.metastore.JobData;
|
||||
import org.apache.sqoop.util.CredentialsUtil;
|
||||
import org.apache.sqoop.util.LoggingUtils;
|
||||
|
||||
/**
|
||||
* Layer on top of SqoopTool that provides some basic common code
|
||||
@ -107,6 +107,10 @@ public abstract class BaseSqoopTool extends com.cloudera.sqoop.tool.SqoopTool {
|
||||
"hive-delims-replacement";
|
||||
public static final String HIVE_PARTITION_KEY_ARG = "hive-partition-key";
|
||||
public static final String HIVE_PARTITION_VALUE_ARG = "hive-partition-value";
|
||||
public static final String HCATCALOG_PARTITION_KEYS_ARG =
|
||||
"hcatalog-partition-keys";
|
||||
public static final String HCATALOG_PARTITION_VALUES_ARG =
|
||||
"hcatalog-partition-values";
|
||||
public static final String CREATE_HIVE_TABLE_ARG =
|
||||
"create-hive-table";
|
||||
public static final String HCATALOG_TABLE_ARG = "hcatalog-table";
|
||||
@ -564,7 +568,17 @@ protected RelatedOptions getHCatalogOptions() {
|
||||
+ " types.")
|
||||
.withLongOpt(MAP_COLUMN_HIVE)
|
||||
.create());
|
||||
|
||||
hCatOptions.addOption(OptionBuilder.withArgName("partition-key")
|
||||
.hasArg()
|
||||
.withDescription("Sets the partition keys to use when importing to hive")
|
||||
.withLongOpt(HCATCALOG_PARTITION_KEYS_ARG)
|
||||
.create());
|
||||
hCatOptions.addOption(OptionBuilder.withArgName("partition-value")
|
||||
.hasArg()
|
||||
.withDescription("Sets the partition values to use when importing "
|
||||
+ "to hive")
|
||||
.withLongOpt(HCATALOG_PARTITION_VALUES_ARG)
|
||||
.create());
|
||||
return hCatOptions;
|
||||
}
|
||||
|
||||
@ -1071,7 +1085,7 @@ protected void applyHiveOptions(CommandLine in, SqoopOptions out)
|
||||
}
|
||||
}
|
||||
|
||||
protected void applyHCatOptions(CommandLine in, SqoopOptions out) {
|
||||
protected void applyHCatalogOptions(CommandLine in, SqoopOptions out) {
|
||||
if (in.hasOption(HCATALOG_TABLE_ARG)) {
|
||||
out.setHCatTableName(in.getOptionValue(HCATALOG_TABLE_ARG));
|
||||
}
|
||||
@ -1098,6 +1112,16 @@ protected void applyHCatOptions(CommandLine in, SqoopOptions out) {
|
||||
out.setHiveHome(in.getOptionValue(HIVE_HOME_ARG));
|
||||
}
|
||||
|
||||
if (in.hasOption(HCATCALOG_PARTITION_KEYS_ARG)) {
|
||||
out.setHCatalogPartitionKeys(
|
||||
in.getOptionValue(HCATCALOG_PARTITION_KEYS_ARG));
|
||||
}
|
||||
|
||||
if (in.hasOption(HCATALOG_PARTITION_VALUES_ARG)) {
|
||||
out.setHCatalogPartitionValues(
|
||||
in.getOptionValue(HCATALOG_PARTITION_VALUES_ARG));
|
||||
}
|
||||
|
||||
if (in.hasOption(HIVE_PARTITION_KEY_ARG)) {
|
||||
out.setHivePartitionKey(in.getOptionValue(HIVE_PARTITION_KEY_ARG));
|
||||
}
|
||||
@ -1109,6 +1133,7 @@ protected void applyHCatOptions(CommandLine in, SqoopOptions out) {
|
||||
if (in.hasOption(MAP_COLUMN_HIVE)) {
|
||||
out.setMapColumnHive(in.getOptionValue(MAP_COLUMN_HIVE));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
@ -1325,13 +1350,13 @@ protected void validateHiveOptions(SqoopOptions options)
|
||||
+ " option." + HELP_STR);
|
||||
}
|
||||
|
||||
if(options.doHiveImport()
|
||||
if (options.doHiveImport()
|
||||
&& options.getFileLayout() == SqoopOptions.FileLayout.AvroDataFile) {
|
||||
throw new InvalidOptionsException("Hive import is not compatible with "
|
||||
+ "importing into AVRO format.");
|
||||
}
|
||||
|
||||
if(options.doHiveImport()
|
||||
if (options.doHiveImport()
|
||||
&& options.getFileLayout() == SqoopOptions.FileLayout.SequenceFile) {
|
||||
throw new InvalidOptionsException("Hive import is not compatible with "
|
||||
+ "importing into SequenceFile format.");
|
||||
@ -1508,7 +1533,36 @@ protected void validateHCatalogOptions(SqoopOptions options)
|
||||
throw new InvalidOptionsException("HCatalog job is not compatible with "
|
||||
+ "SequenceFile format option " + FMT_SEQUENCEFILE_ARG
|
||||
+ " option." + HELP_STR);
|
||||
}
|
||||
|
||||
if (options.getHCatalogPartitionKeys() != null
|
||||
&& options.getHCatalogPartitionValues() == null) {
|
||||
throw new InvalidOptionsException("Either both --hcatalog-partition-keys"
|
||||
+ " and --hcatalog-partition-values should be provided or both of these"
|
||||
+ " options should be omitted.");
|
||||
}
|
||||
|
||||
if (options.getHCatalogPartitionKeys() != null) {
|
||||
if (options.getHivePartitionKey() != null) {
|
||||
LOG.warn("Both --hcatalog-partition-keys and --hive-partition-key"
|
||||
+ "options are provided. --hive-partition-key option will be"
|
||||
+ "ignored");
|
||||
}
|
||||
|
||||
String[] keys = options.getHCatalogPartitionKeys().split(",");
|
||||
String[] vals = options.getHCatalogPartitionValues().split(",");
|
||||
|
||||
if (keys.length != vals.length) {
|
||||
throw new InvalidOptionsException("Number of static partition keys "
|
||||
+ "provided dpes match the number of partition values");
|
||||
}
|
||||
} else {
|
||||
if (options.getHivePartitionKey() != null
|
||||
&& options.getHivePartitionValue() == null) {
|
||||
throw new InvalidOptionsException("Either both --hive-partition-key and"
|
||||
+ " --hive-partition-value options should be provided or both of "
|
||||
+ "these options should be omitted");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -1527,8 +1581,8 @@ protected void validateHBaseOptions(SqoopOptions options)
|
||||
}
|
||||
|
||||
if (options.isBulkLoadEnabled() && options.getHBaseTable() == null) {
|
||||
String validationMessage = String.format("Can't run import with %s " +
|
||||
"without %s",
|
||||
String validationMessage = String.format("Can't run import with %s "
|
||||
+ "without %s",
|
||||
BaseSqoopTool.HBASE_BULK_LOAD_ENABLED_ARG,
|
||||
BaseSqoopTool.HBASE_TABLE_ARG);
|
||||
throw new InvalidOptionsException(validationMessage);
|
||||
|
@ -189,7 +189,7 @@ public void applyOptions(CommandLine in, SqoopOptions out)
|
||||
applyInputFormatOptions(in, out);
|
||||
applyCodeGenOptions(in, out, false);
|
||||
applyHiveOptions(in, out);
|
||||
applyHCatOptions(in, out);
|
||||
applyHCatalogOptions(in, out);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -292,7 +292,7 @@ public void applyOptions(CommandLine in, SqoopOptions out)
|
||||
applyInputFormatOptions(in, out);
|
||||
applyOutputFormatOptions(in, out);
|
||||
applyCodeGenOptions(in, out, false);
|
||||
applyHCatOptions(in, out);
|
||||
applyHCatalogOptions(in, out);
|
||||
} catch (NumberFormatException nfe) {
|
||||
throw new InvalidOptionsException("Error: expected numeric argument.\n"
|
||||
+ "Try --help for usage.");
|
||||
|
@ -19,7 +19,6 @@
|
||||
package org.apache.sqoop.tool;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import java.sql.Connection;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.ResultSetMetaData;
|
||||
@ -33,7 +32,8 @@
|
||||
import org.apache.commons.cli.OptionBuilder;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.cloudera.sqoop.Sqoop;
|
||||
@ -43,14 +43,11 @@
|
||||
import com.cloudera.sqoop.cli.ToolOptions;
|
||||
import com.cloudera.sqoop.hive.HiveImport;
|
||||
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||
|
||||
import com.cloudera.sqoop.metastore.JobData;
|
||||
import com.cloudera.sqoop.metastore.JobStorage;
|
||||
import com.cloudera.sqoop.metastore.JobStorageFactory;
|
||||
import com.cloudera.sqoop.util.AppendUtils;
|
||||
import com.cloudera.sqoop.util.ImportException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
/**
|
||||
* Tool that performs database imports to HDFS.
|
||||
@ -860,7 +857,7 @@ public void applyOptions(CommandLine in, SqoopOptions out)
|
||||
applyInputFormatOptions(in, out);
|
||||
applyCodeGenOptions(in, out, allTables);
|
||||
applyHBaseOptions(in, out);
|
||||
applyHCatOptions(in, out);
|
||||
applyHCatalogOptions(in, out);
|
||||
applyAccumuloOptions(in, out);
|
||||
|
||||
} catch (NumberFormatException nfe) {
|
||||
|
@ -18,20 +18,22 @@
|
||||
|
||||
package com.cloudera.sqoop;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.sql.Connection;
|
||||
import java.sql.ResultSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import com.cloudera.sqoop.manager.ConnManager;
|
||||
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||
import com.cloudera.sqoop.manager.ManagerFactory;
|
||||
import com.cloudera.sqoop.metastore.JobData;
|
||||
import com.cloudera.sqoop.tool.ImportTool;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.sql.Connection;
|
||||
import java.sql.ResultSet;
|
||||
|
||||
/**
|
||||
* Test the ConnFactory implementation and its ability to delegate to multiple
|
||||
* different ManagerFactory implementations using reflection.
|
||||
@ -130,6 +132,10 @@ public String getPrimaryKey(String tableName) {
|
||||
return null;
|
||||
}
|
||||
|
||||
public Map<String,List<Integer>> getColumnInfo(String tableName) {
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Default implementation.
|
||||
* @param sqlType sql data type
|
||||
|
@ -34,7 +34,10 @@
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hadoop.hive.common.type.HiveChar;
|
||||
import org.apache.hadoop.hive.common.type.HiveDecimal;
|
||||
import org.apache.hadoop.hive.common.type.HiveVarchar;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.sqoop.hcat.HCatalogTestUtils.ColumnGenerator;
|
||||
import org.apache.sqoop.hcat.HCatalogTestUtils.CreateMode;
|
||||
import org.apache.sqoop.hcat.HCatalogTestUtils.KeyType;
|
||||
@ -143,19 +146,19 @@ public void testIntTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"boolean", Types.BOOLEAN, HCatFieldSchema.Type.BOOLEAN,
|
||||
"boolean", Types.BOOLEAN, HCatFieldSchema.Type.BOOLEAN, 0, 0,
|
||||
Boolean.TRUE, Boolean.TRUE, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"tinyint", Types.INTEGER, HCatFieldSchema.Type.INT, 10,
|
||||
"tinyint", Types.INTEGER, HCatFieldSchema.Type.INT, 0, 0, 10,
|
||||
10, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"smallint", Types.INTEGER, HCatFieldSchema.Type.INT, 100,
|
||||
"smallint", Types.INTEGER, HCatFieldSchema.Type.INT, 0, 0, 100,
|
||||
100, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(3),
|
||||
"int", Types.INTEGER, HCatFieldSchema.Type.INT, 1000,
|
||||
"int", Types.INTEGER, HCatFieldSchema.Type.INT, 0, 0, 1000,
|
||||
1000, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(4),
|
||||
"bigint", Types.BIGINT, HCatFieldSchema.Type.BIGINT, 10000L,
|
||||
"bigint", Types.BIGINT, HCatFieldSchema.Type.BIGINT, 0, 0, 10000L,
|
||||
10000L, KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -167,13 +170,13 @@ public void testFloatTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"float", Types.FLOAT, HCatFieldSchema.Type.FLOAT, 10.0F,
|
||||
"float", Types.FLOAT, HCatFieldSchema.Type.FLOAT, 0, 0, 10.0F,
|
||||
10.F, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"real", Types.FLOAT, HCatFieldSchema.Type.FLOAT, 20.0F,
|
||||
"real", Types.FLOAT, HCatFieldSchema.Type.FLOAT, 0, 0, 20.0F,
|
||||
20.0F, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"double", Types.DOUBLE, HCatFieldSchema.Type.DOUBLE, 30.0D,
|
||||
"double", Types.DOUBLE, HCatFieldSchema.Type.DOUBLE, 0, 0, 30.0D,
|
||||
30.0D, KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -185,11 +188,15 @@ public void testNumberTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"numeric(18,2)", Types.NUMERIC, HCatFieldSchema.Type.STRING, "1000",
|
||||
new BigDecimal("1000"), KeyType.NOT_A_KEY),
|
||||
"numeric(18,2)", Types.NUMERIC, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1000", new BigDecimal("1000"), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"decimal(18,2)", Types.DECIMAL, HCatFieldSchema.Type.STRING, "2000",
|
||||
new BigDecimal("2000"), KeyType.NOT_A_KEY),
|
||||
"decimal(18,2)", Types.DECIMAL, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2000", new BigDecimal("2000"), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"decimal(18,2)", Types.DECIMAL, HCatFieldSchema.Type.DECIMAL, 18, 2,
|
||||
HiveDecimal.create(new BigDecimal("2000")),
|
||||
new BigDecimal("2000"), KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
runHCatExport(addlArgsArray, TOTAL_RECORDS, table, cols);
|
||||
@ -200,15 +207,24 @@ public void testDateTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"date", Types.DATE, HCatFieldSchema.Type.STRING, "2013-12-31",
|
||||
new Date(113, 11, 31), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"time", Types.TIME, HCatFieldSchema.Type.STRING, "10:11:12",
|
||||
new Time(10, 11, 12), KeyType.NOT_A_KEY),
|
||||
"date", Types.DATE, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2013-12-31", new Date(113, 11, 31), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"date", Types.DATE, HCatFieldSchema.Type.DATE, 0, 0,
|
||||
new Date(113, 11, 31),
|
||||
new Date(113, 11, 31), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"timestamp", Types.TIMESTAMP, HCatFieldSchema.Type.STRING,
|
||||
"time", Types.TIME, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"10:11:12", new Time(10, 11, 12), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(3),
|
||||
"timestamp", Types.TIMESTAMP, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2013-12-31 10:11:12", new Timestamp(113, 11, 31, 10, 11, 12, 0),
|
||||
KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(4),
|
||||
"timestamp", Types.TIMESTAMP, HCatFieldSchema.Type.TIMESTAMP, 0, 0,
|
||||
new Timestamp(113, 11, 31, 10, 11, 12, 0),
|
||||
new Timestamp(113, 11, 31, 10, 11, 12, 0),
|
||||
KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
runHCatExport(addlArgsArray, TOTAL_RECORDS, table, cols);
|
||||
@ -220,13 +236,13 @@ public void testDateTypesToBigInt() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"date", Types.DATE, HCatFieldSchema.Type.BIGINT, 0 - offset,
|
||||
"date", Types.DATE, HCatFieldSchema.Type.BIGINT, 0, 0, 0 - offset,
|
||||
new Date(70, 0, 1), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"time", Types.TIME, HCatFieldSchema.Type.BIGINT, 36672000L - offset,
|
||||
new Time(10, 11, 12), KeyType.NOT_A_KEY),
|
||||
"time", Types.TIME, HCatFieldSchema.Type.BIGINT, 0, 0,
|
||||
36672000L - offset, new Time(10, 11, 12), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"timestamp", Types.TIMESTAMP, HCatFieldSchema.Type.BIGINT,
|
||||
"timestamp", Types.TIMESTAMP, HCatFieldSchema.Type.BIGINT, 0, 0,
|
||||
36672000L - offset, new Timestamp(70, 0, 1, 10, 11, 12, 0),
|
||||
KeyType.NOT_A_KEY),
|
||||
};
|
||||
@ -241,10 +257,18 @@ public void testStringTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.STRING, "string to test",
|
||||
"string to test", KeyType.NOT_A_KEY),
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"string to test", "string to test", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"longvarchar", Types.LONGVARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.CHAR, 14, 0,
|
||||
new HiveChar("string to test", 14), "string to test",
|
||||
KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.VARCHAR, 14, 0,
|
||||
new HiveVarchar("string to test", 14), "string to test",
|
||||
KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(3),
|
||||
"longvarchar", Types.LONGVARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"string to test", "string to test", KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -258,10 +282,10 @@ public void testBinaryTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"binary(10)", Types.BINARY, HCatFieldSchema.Type.BINARY,
|
||||
"binary(10)", Types.BINARY, HCatFieldSchema.Type.BINARY, 0, 0,
|
||||
bb.array(), bb.array(), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varbinary(10)", Types.BINARY, HCatFieldSchema.Type.BINARY,
|
||||
"varbinary(10)", Types.BINARY, HCatFieldSchema.Type.BINARY, 0, 0,
|
||||
bb.array(), bb.array(), KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -273,7 +297,7 @@ public void testColumnProjection() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", null, KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -287,7 +311,7 @@ public void testStaticPartitioning() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -299,12 +323,32 @@ public void testStaticPartitioning() throws Exception {
|
||||
runHCatExport(addlArgsArray, TOTAL_RECORDS, table, cols);
|
||||
}
|
||||
|
||||
public void testStaticPartitioningWithMultipleKeys() throws Exception {
|
||||
final int TOTAL_RECORDS = 1 * 10;
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2", "2", KeyType.STATIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--hcatalog-partition-keys");
|
||||
addlArgsArray.add("col0,col1");
|
||||
addlArgsArray.add("--hcatalog-partition-values");
|
||||
addlArgsArray.add("1,2");
|
||||
|
||||
runHCatExport(addlArgsArray, TOTAL_RECORDS, table, cols);
|
||||
}
|
||||
|
||||
public void testDynamicPartitioning() throws Exception {
|
||||
final int TOTAL_RECORDS = 1 * 10;
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.DYNAMIC_KEY),
|
||||
};
|
||||
|
||||
@ -312,15 +356,15 @@ public void testDynamicPartitioning() throws Exception {
|
||||
runHCatExport(addlArgsArray, TOTAL_RECORDS, table, cols);
|
||||
}
|
||||
|
||||
public void testStaicAndDynamicPartitioning() throws Exception {
|
||||
public void testStaticAndDynamicPartitioning() throws Exception {
|
||||
final int TOTAL_RECORDS = 1 * 10;
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2", "2", KeyType.DYNAMIC_KEY),
|
||||
};
|
||||
|
||||
@ -332,6 +376,29 @@ public void testStaicAndDynamicPartitioning() throws Exception {
|
||||
runHCatExport(addlArgsArray, TOTAL_RECORDS, table, cols);
|
||||
}
|
||||
|
||||
public void testMultipleStaticKeysAndDynamicPartitioning() throws Exception {
|
||||
final int TOTAL_RECORDS = 1 * 10;
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2", "2", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"3", "3", KeyType.DYNAMIC_KEY),
|
||||
};
|
||||
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--hcatalog-partition-keys");
|
||||
addlArgsArray.add("col0,col1");
|
||||
addlArgsArray.add("--hcatalog-partition-values");
|
||||
addlArgsArray.add("1,2");
|
||||
runHCatExport(addlArgsArray, TOTAL_RECORDS, table, cols);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test other file formats.
|
||||
*/
|
||||
@ -340,11 +407,11 @@ public void testSequenceFile() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "1",
|
||||
"1", KeyType.STATIC_KEY),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "2",
|
||||
"2", KeyType.DYNAMIC_KEY), };
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2", "2", KeyType.DYNAMIC_KEY), };
|
||||
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--hive-partition-key");
|
||||
@ -360,11 +427,11 @@ public void testTextFile() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "1",
|
||||
"1", KeyType.STATIC_KEY),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "2",
|
||||
"2", KeyType.DYNAMIC_KEY), };
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2", "2", KeyType.DYNAMIC_KEY), };
|
||||
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--hive-partition-key");
|
||||
|
@ -38,9 +38,12 @@
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hcatalog.data.HCatRecord;
|
||||
import org.apache.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hcatalog.data.schema.HCatSchema;
|
||||
import org.apache.hadoop.hive.common.type.HiveChar;
|
||||
import org.apache.hadoop.hive.common.type.HiveDecimal;
|
||||
import org.apache.hadoop.hive.common.type.HiveVarchar;
|
||||
import org.apache.hive.hcatalog.data.HCatRecord;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatSchema;
|
||||
import org.apache.sqoop.hcat.HCatalogTestUtils.ColumnGenerator;
|
||||
import org.apache.sqoop.hcat.HCatalogTestUtils.CreateMode;
|
||||
import org.apache.sqoop.hcat.HCatalogTestUtils.KeyType;
|
||||
@ -340,19 +343,19 @@ public void testIntTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"boolean", Types.BOOLEAN, HCatFieldSchema.Type.BOOLEAN,
|
||||
"boolean", Types.BOOLEAN, HCatFieldSchema.Type.BOOLEAN, 0, 0,
|
||||
Boolean.TRUE, Boolean.TRUE, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"tinyint", Types.INTEGER, HCatFieldSchema.Type.INT, 10,
|
||||
"tinyint", Types.INTEGER, HCatFieldSchema.Type.INT, 0, 0, 10,
|
||||
10, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"smallint", Types.INTEGER, HCatFieldSchema.Type.INT, 100,
|
||||
"smallint", Types.INTEGER, HCatFieldSchema.Type.INT, 0, 0, 100,
|
||||
100, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(3),
|
||||
"int", Types.INTEGER, HCatFieldSchema.Type.INT, 1000,
|
||||
"int", Types.INTEGER, HCatFieldSchema.Type.INT, 0, 0, 1000,
|
||||
1000, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(4),
|
||||
"bigint", Types.BIGINT, HCatFieldSchema.Type.BIGINT, 10000L,
|
||||
"bigint", Types.BIGINT, HCatFieldSchema.Type.BIGINT, 0, 0, 10000L,
|
||||
10000L, KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -365,13 +368,13 @@ public void testFloatTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"float", Types.FLOAT, HCatFieldSchema.Type.FLOAT, 10.0F,
|
||||
"float", Types.FLOAT, HCatFieldSchema.Type.FLOAT, 0, 0, 10.0F,
|
||||
10.F, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"real", Types.FLOAT, HCatFieldSchema.Type.FLOAT, 20.0F,
|
||||
"real", Types.FLOAT, HCatFieldSchema.Type.FLOAT, 0, 0, 20.0F,
|
||||
20.0F, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"double", Types.DOUBLE, HCatFieldSchema.Type.DOUBLE, 30.0D,
|
||||
"double", Types.DOUBLE, HCatFieldSchema.Type.DOUBLE, 0, 0, 30.0D,
|
||||
30.0D, KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -384,11 +387,15 @@ public void testNumberTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"numeric(18,2)", Types.NUMERIC, HCatFieldSchema.Type.STRING, "1000",
|
||||
new BigDecimal("1000"), KeyType.NOT_A_KEY),
|
||||
"numeric(18,2)", Types.NUMERIC, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1000", new BigDecimal("1000"), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"decimal(18,2)", Types.DECIMAL, HCatFieldSchema.Type.STRING, "2000",
|
||||
new BigDecimal("2000"), KeyType.NOT_A_KEY),
|
||||
"decimal(18,2)", Types.DECIMAL, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2000", new BigDecimal("2000"), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"decimal(18,2)", Types.DECIMAL, HCatFieldSchema.Type.DECIMAL, 18, 2,
|
||||
HiveDecimal.create(new BigDecimal("2000")),
|
||||
new BigDecimal("2000"), KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
setExtraArgs(addlArgsArray);
|
||||
@ -400,15 +407,24 @@ public void testDateTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"date", Types.DATE, HCatFieldSchema.Type.STRING, "2013-12-31",
|
||||
"date", Types.DATE, HCatFieldSchema.Type.STRING, 0, 0, "2013-12-31",
|
||||
new Date(113, 11, 31), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"time", Types.TIME, HCatFieldSchema.Type.STRING, "10:11:12",
|
||||
new Time(10, 11, 12), KeyType.NOT_A_KEY),
|
||||
"date", Types.DATE, HCatFieldSchema.Type.DATE, 0, 0,
|
||||
new Date(113, 11, 31),
|
||||
new Date(113, 11, 31), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"timestamp", Types.TIMESTAMP, HCatFieldSchema.Type.STRING,
|
||||
"time", Types.TIME, HCatFieldSchema.Type.STRING, 0, 0, "10:11:12",
|
||||
new Time(10, 11, 12), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(3),
|
||||
"timestamp", Types.TIMESTAMP, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2013-12-31 10:11:12.0", new Timestamp(113, 11, 31, 10, 11, 12, 0),
|
||||
KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(4),
|
||||
"timestamp", Types.TIMESTAMP, HCatFieldSchema.Type.TIMESTAMP, 0, 0,
|
||||
new Timestamp(113, 11, 31, 10, 11, 12, 0),
|
||||
new Timestamp(113, 11, 31, 10, 11, 12, 0),
|
||||
KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
setExtraArgs(addlArgsArray);
|
||||
@ -421,13 +437,13 @@ public void testDateTypesToBigInt() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"date", Types.DATE, HCatFieldSchema.Type.BIGINT, 0 - offset,
|
||||
"date", Types.DATE, HCatFieldSchema.Type.BIGINT, 0, 0, 0 - offset,
|
||||
new Date(70, 0, 1), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"time", Types.TIME, HCatFieldSchema.Type.BIGINT, 36672000L - offset,
|
||||
new Time(10, 11, 12), KeyType.NOT_A_KEY),
|
||||
"time", Types.TIME, HCatFieldSchema.Type.BIGINT, 0, 0,
|
||||
36672000L - offset, new Time(10, 11, 12), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"timestamp", Types.TIMESTAMP, HCatFieldSchema.Type.BIGINT,
|
||||
"timestamp", Types.TIMESTAMP, HCatFieldSchema.Type.BIGINT, 0, 0,
|
||||
36672000L - offset, new Timestamp(70, 0, 1, 10, 11, 12, 0),
|
||||
KeyType.NOT_A_KEY),
|
||||
};
|
||||
@ -443,10 +459,18 @@ public void testStringTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.STRING, "string to test",
|
||||
"string to test", KeyType.NOT_A_KEY),
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"string to test", "string to test", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"longvarchar", Types.LONGVARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.CHAR, 14, 0,
|
||||
new HiveChar("string to test", 14), "string to test",
|
||||
KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.VARCHAR, 14, 0,
|
||||
new HiveVarchar("string to test", 14), "string to test",
|
||||
KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(3),
|
||||
"longvarchar", Types.LONGVARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"string to test", "string to test", KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -460,10 +484,10 @@ public void testBinaryTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"binary(10)", Types.BINARY, HCatFieldSchema.Type.BINARY,
|
||||
"binary(10)", Types.BINARY, HCatFieldSchema.Type.BINARY, 0, 0,
|
||||
bb.array(), bb.array(), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"longvarbinary", Types.BINARY, HCatFieldSchema.Type.BINARY,
|
||||
"longvarbinary", Types.BINARY, HCatFieldSchema.Type.BINARY, 0, 0,
|
||||
bb.array(), bb.array(), KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -476,7 +500,7 @@ public void testColumnProjection() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
null, null, KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -494,7 +518,7 @@ public void testColumnProjectionMissingPartKeys() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
null, null, KeyType.DYNAMIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -519,7 +543,7 @@ public void testStaticPartitioning() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -531,12 +555,32 @@ public void testStaticPartitioning() throws Exception {
|
||||
runHCatImport(addlArgsArray, TOTAL_RECORDS, table, cols, null);
|
||||
}
|
||||
|
||||
public void testStaticPartitioningWithMultipleKeys() throws Exception {
|
||||
final int TOTAL_RECORDS = 1 * 10;
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2", "2", KeyType.STATIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--hcatalog-partition-keys");
|
||||
addlArgsArray.add("col0,col1");
|
||||
addlArgsArray.add("--hcatalog-partition-values");
|
||||
addlArgsArray.add("1,2");
|
||||
setExtraArgs(addlArgsArray);
|
||||
runHCatImport(addlArgsArray, TOTAL_RECORDS, table, cols, null);
|
||||
}
|
||||
|
||||
public void testDynamicPartitioning() throws Exception {
|
||||
final int TOTAL_RECORDS = 1 * 10;
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.DYNAMIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -544,15 +588,15 @@ public void testDynamicPartitioning() throws Exception {
|
||||
runHCatImport(addlArgsArray, TOTAL_RECORDS, table, cols, null);
|
||||
}
|
||||
|
||||
public void testStaicAndDynamicPartitioning() throws Exception {
|
||||
public void testStaticAndDynamicPartitioning() throws Exception {
|
||||
final int TOTAL_RECORDS = 1 * 10;
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2", "2", KeyType.DYNAMIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -564,6 +608,29 @@ public void testStaicAndDynamicPartitioning() throws Exception {
|
||||
runHCatImport(addlArgsArray, TOTAL_RECORDS, table, cols, null);
|
||||
}
|
||||
|
||||
public void testMultipleStaticKeysAndDynamicPartitioning() throws Exception {
|
||||
final int TOTAL_RECORDS = 1 * 10;
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2", "2", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"3", "3", KeyType.DYNAMIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--hcatalog-partition-keys");
|
||||
addlArgsArray.add("col0,col1");
|
||||
addlArgsArray.add("--hcatalog-partition-values");
|
||||
addlArgsArray.add("1,2");
|
||||
setExtraArgs(addlArgsArray);
|
||||
runHCatImport(addlArgsArray, TOTAL_RECORDS, table, cols, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test other file formats.
|
||||
*/
|
||||
@ -572,11 +639,11 @@ public void testSequenceFile() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "1",
|
||||
"1", KeyType.STATIC_KEY),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "2",
|
||||
"2", KeyType.DYNAMIC_KEY), };
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2", "2", KeyType.DYNAMIC_KEY), };
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--hive-partition-key");
|
||||
addlArgsArray.add("col0");
|
||||
@ -592,10 +659,10 @@ public void testTextFile() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2", "2", KeyType.DYNAMIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -613,11 +680,11 @@ public void testTableCreation() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"1", "1", KeyType.STATIC_KEY),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
new HiveVarchar("1", 20), "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"2", "2", KeyType.DYNAMIC_KEY),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
new HiveVarchar("2", 20), "2", KeyType.DYNAMIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--create-hcatalog-table");
|
||||
@ -631,11 +698,11 @@ public void testTableCreationWithPartition() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"1", "1", KeyType.NOT_A_KEY),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.VARCHAR, 20, 0,
|
||||
new HiveVarchar("1", 20), "1", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"2", "2", KeyType.STATIC_KEY),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.VARCHAR, 20, 0,
|
||||
new HiveVarchar("2", 20), "2", KeyType.STATIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--hive-partition-key");
|
||||
@ -647,16 +714,37 @@ public void testTableCreationWithPartition() throws Exception {
|
||||
runHCatImport(addlArgsArray, TOTAL_RECORDS, table, cols, null, true, false);
|
||||
}
|
||||
|
||||
public void testTableCreationWithMultipleStaticPartKeys() throws Exception {
|
||||
final int TOTAL_RECORDS = 1 * 10;
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.VARCHAR, 20, 0,
|
||||
new HiveVarchar("1", 20), "1", KeyType.STATIC_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.VARCHAR, 20, 0,
|
||||
new HiveVarchar("2", 20), "2", KeyType.STATIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--hcatalog-partition-keys");
|
||||
addlArgsArray.add("col0,col1");
|
||||
addlArgsArray.add("--hcatalog-partition-values");
|
||||
addlArgsArray.add("1,2");
|
||||
addlArgsArray.add("--create-hcatalog-table");
|
||||
setExtraArgs(addlArgsArray);
|
||||
runHCatImport(addlArgsArray, TOTAL_RECORDS, table, cols, null, true, false);
|
||||
}
|
||||
|
||||
public void testTableCreationWithStorageStanza() throws Exception {
|
||||
final int TOTAL_RECORDS = 1 * 10;
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"1", "1", KeyType.NOT_A_KEY),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.VARCHAR, 20, 0,
|
||||
new HiveVarchar("1", 20), "1", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"2", "2", KeyType.STATIC_KEY),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.VARCHAR, 20, 0,
|
||||
new HiveVarchar("2", 20), "2", KeyType.STATIC_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--hive-partition-key");
|
||||
@ -675,10 +763,10 @@ public void testHiveDropDelims() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"Test", "\u0001\n\rTest", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"Test2", "\u0001\r\nTest2", KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -692,10 +780,10 @@ public void testHiveDelimsReplacement() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"^^^Test", "\u0001\n\rTest", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"^^^Test2", "\u0001\r\nTest2", KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -710,10 +798,10 @@ public void testDynamicKeyInMiddle() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "1",
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0, "1",
|
||||
"1", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "2",
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0, "2",
|
||||
"2", KeyType.DYNAMIC_KEY), };
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
setExtraArgs(addlArgsArray);
|
||||
@ -726,10 +814,10 @@ public void testQueryImport() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "1",
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0, "1",
|
||||
"1", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "2",
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0, "2",
|
||||
"2", KeyType.DYNAMIC_KEY), };
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
setExtraArgs(addlArgsArray);
|
||||
@ -742,11 +830,11 @@ public void testCreateTableWithPreExistingTable() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "1",
|
||||
"1", KeyType.NOT_A_KEY),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.VARCHAR, 20, 0,
|
||||
new HiveVarchar("1", 20), "1", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, "2",
|
||||
"2", KeyType.DYNAMIC_KEY), };
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.VARCHAR, 20, 0,
|
||||
new HiveVarchar("2", 20), "2", KeyType.DYNAMIC_KEY), };
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
addlArgsArray.add("--create-hcatalog-table");
|
||||
setExtraArgs(addlArgsArray);
|
||||
|
@ -35,7 +35,13 @@
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.common.type.HiveChar;
|
||||
import org.apache.hadoop.hive.common.type.HiveVarchar;
|
||||
import org.apache.hadoop.hive.metastore.api.MetaException;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
@ -46,13 +52,13 @@
|
||||
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
|
||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
|
||||
import org.apache.hcatalog.data.DefaultHCatRecord;
|
||||
import org.apache.hcatalog.data.HCatRecord;
|
||||
import org.apache.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hcatalog.data.schema.HCatSchema;
|
||||
import org.apache.hcatalog.mapreduce.HCatInputFormat;
|
||||
import org.apache.hcatalog.mapreduce.HCatOutputFormat;
|
||||
import org.apache.hcatalog.mapreduce.OutputJobInfo;
|
||||
import org.apache.hive.hcatalog.data.DefaultHCatRecord;
|
||||
import org.apache.hive.hcatalog.data.HCatRecord;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatSchema;
|
||||
import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
|
||||
import org.apache.hive.hcatalog.mapreduce.HCatOutputFormat;
|
||||
import org.apache.hive.hcatalog.mapreduce.OutputJobInfo;
|
||||
import org.apache.sqoop.config.ConfigurationConstants;
|
||||
import org.apache.sqoop.mapreduce.hcat.SqoopHCatUtilities;
|
||||
import org.junit.Assert;
|
||||
@ -97,6 +103,10 @@ public static HCatalogTestUtils instance() {
|
||||
return Holder.INSTANCE;
|
||||
}
|
||||
|
||||
public static StringBuilder escHCatObj(String objectName) {
|
||||
return SqoopHCatUtilities.escHCatObj(objectName);
|
||||
}
|
||||
|
||||
public void initUtils() throws IOException, MetaException {
|
||||
if (initialized) {
|
||||
return;
|
||||
@ -123,24 +133,25 @@ public void setStorageInfo(String info) {
|
||||
storageInfo = info;
|
||||
}
|
||||
|
||||
private static String getDropTableCmd(final String dbName,
|
||||
private static String getHCatDropTableCmd(final String dbName,
|
||||
final String tableName) {
|
||||
return "DROP TABLE IF EXISTS " + dbName.toLowerCase() + "."
|
||||
+ tableName.toLowerCase();
|
||||
return "DROP TABLE IF EXISTS " + escHCatObj(dbName.toLowerCase()) + "."
|
||||
+ escHCatObj(tableName.toLowerCase());
|
||||
}
|
||||
|
||||
private static String getHCatCreateTableCmd(String dbName,
|
||||
String tableName, List<HCatFieldSchema> tableCols,
|
||||
List<HCatFieldSchema> partKeys) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("create table ").append(dbName.toLowerCase()).append('.');
|
||||
sb.append(tableName.toLowerCase()).append(" (\n\t");
|
||||
sb.append("create table ")
|
||||
.append(escHCatObj(dbName.toLowerCase()).append('.'));
|
||||
sb.append(escHCatObj(tableName.toLowerCase()).append(" (\n\t"));
|
||||
for (int i = 0; i < tableCols.size(); ++i) {
|
||||
HCatFieldSchema hfs = tableCols.get(i);
|
||||
if (i > 0) {
|
||||
sb.append(",\n\t");
|
||||
}
|
||||
sb.append(hfs.getName().toLowerCase());
|
||||
sb.append(escHCatObj(hfs.getName().toLowerCase()));
|
||||
sb.append(' ').append(hfs.getTypeString());
|
||||
}
|
||||
sb.append(")\n");
|
||||
@ -151,7 +162,7 @@ private static String getHCatCreateTableCmd(String dbName,
|
||||
if (i > 0) {
|
||||
sb.append("\n\t,");
|
||||
}
|
||||
sb.append(hfs.getName().toLowerCase());
|
||||
sb.append(escHCatObj(hfs.getName().toLowerCase()));
|
||||
sb.append(' ').append(hfs.getTypeString());
|
||||
}
|
||||
sb.append(")\n");
|
||||
@ -174,7 +185,7 @@ public void createHCatTableUsingSchema(String dbName,
|
||||
? SqoopHCatUtilities.DEFHCATDB : dbName;
|
||||
LOG.info("Dropping HCatalog table if it exists " + databaseName
|
||||
+ '.' + tableName);
|
||||
String dropCmd = getDropTableCmd(databaseName, tableName);
|
||||
String dropCmd = getHCatDropTableCmd(databaseName, tableName);
|
||||
|
||||
try {
|
||||
utils.launchHCatCli(dropCmd);
|
||||
@ -411,6 +422,11 @@ public interface ColumnGenerator {
|
||||
/** Return the HCat type for this column. */
|
||||
HCatFieldSchema.Type getHCatType();
|
||||
|
||||
/** Return the precision/length of the field if any. */
|
||||
int getHCatPrecision();
|
||||
|
||||
/** Return the scale of the field if any. */
|
||||
int getHCatScale();
|
||||
|
||||
/**
|
||||
* If the field is a partition key, then whether is part of the static
|
||||
@ -437,7 +453,8 @@ public static String forIdx(int idx) {
|
||||
|
||||
public static ColumnGenerator colGenerator(final String name,
|
||||
final String dbType, final int sqlType,
|
||||
final HCatFieldSchema.Type hCatType, final Object hCatValue,
|
||||
final HCatFieldSchema.Type hCatType, final int hCatPrecision,
|
||||
final int hCatScale, final Object hCatValue,
|
||||
final Object dbValue, final KeyType keyType) {
|
||||
return new ColumnGenerator() {
|
||||
|
||||
@ -471,6 +488,16 @@ public HCatFieldSchema.Type getHCatType() {
|
||||
return hCatType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getHCatPrecision() {
|
||||
return hCatPrecision;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getHCatScale() {
|
||||
return hCatScale;
|
||||
}
|
||||
|
||||
public KeyType getKeyType() {
|
||||
return keyType;
|
||||
}
|
||||
@ -503,6 +530,28 @@ public static void assertEquals(Object expectedVal,
|
||||
.assertEquals("Got unexpected column value", expectedVal,
|
||||
actualVal);
|
||||
}
|
||||
} else if (expectedVal instanceof HiveVarchar) {
|
||||
HiveVarchar vc1 = (HiveVarchar) expectedVal;
|
||||
if (actualVal instanceof HiveVarchar) {
|
||||
HiveVarchar vc2 = (HiveVarchar)actualVal;
|
||||
assertEquals(vc1.getCharacterLength(), vc2.getCharacterLength());
|
||||
assertEquals(vc1.getValue(), vc2.getValue());
|
||||
} else {
|
||||
String vc2 = (String)actualVal;
|
||||
assertEquals(vc1.getCharacterLength(), vc2.length());
|
||||
assertEquals(vc1.getValue(), vc2);
|
||||
}
|
||||
} else if (expectedVal instanceof HiveChar) {
|
||||
HiveChar c1 = (HiveChar) expectedVal;
|
||||
if (actualVal instanceof HiveChar) {
|
||||
HiveChar c2 = (HiveChar)actualVal;
|
||||
assertEquals(c1.getCharacterLength(), c2.getCharacterLength());
|
||||
assertEquals(c1.getValue(), c2.getValue());
|
||||
} else {
|
||||
String c2 = (String) actualVal;
|
||||
assertEquals(c1.getCharacterLength(), c2.length());
|
||||
assertEquals(c1.getValue(), c2);
|
||||
}
|
||||
} else {
|
||||
Assert
|
||||
.assertEquals("Got unexpected column value", expectedVal,
|
||||
@ -626,7 +675,7 @@ public static String getSqlInsertTableStatement(String tableName,
|
||||
sb.append(tableName);
|
||||
sb.append(" (id, msg");
|
||||
int colNum = 0;
|
||||
for (ColumnGenerator gen : extraCols) {
|
||||
for (int i = 0; i < extraCols.length; ++i) {
|
||||
sb.append(", " + forIdx(colNum++));
|
||||
}
|
||||
sb.append(") VALUES ( ?, ?");
|
||||
@ -733,13 +782,34 @@ private HCatSchema generateHCatTableSchema(ColumnGenerator... extraCols)
|
||||
throws Exception {
|
||||
List<HCatFieldSchema> hCatTblCols = new ArrayList<HCatFieldSchema>();
|
||||
hCatTblCols.clear();
|
||||
hCatTblCols.add(new HCatFieldSchema("id", HCatFieldSchema.Type.INT, ""));
|
||||
PrimitiveTypeInfo tInfo;
|
||||
tInfo = new PrimitiveTypeInfo();
|
||||
tInfo.setTypeName(HCatFieldSchema.Type.INT.name().toLowerCase());
|
||||
hCatTblCols.add(new HCatFieldSchema("id", tInfo, ""));
|
||||
tInfo = new PrimitiveTypeInfo();
|
||||
tInfo.setTypeName(HCatFieldSchema.Type.STRING.name().toLowerCase());
|
||||
hCatTblCols
|
||||
.add(new HCatFieldSchema("msg", HCatFieldSchema.Type.STRING, ""));
|
||||
.add(new HCatFieldSchema("msg", tInfo, ""));
|
||||
for (ColumnGenerator gen : extraCols) {
|
||||
if (gen.getKeyType() == KeyType.NOT_A_KEY) {
|
||||
switch(gen.getHCatType()) {
|
||||
case CHAR:
|
||||
tInfo = new CharTypeInfo(gen.getHCatPrecision());
|
||||
break;
|
||||
case VARCHAR:
|
||||
tInfo = new VarcharTypeInfo(gen.getHCatPrecision());
|
||||
break;
|
||||
case DECIMAL:
|
||||
tInfo = new DecimalTypeInfo(gen.getHCatPrecision(),
|
||||
gen.getHCatScale());
|
||||
break;
|
||||
default:
|
||||
tInfo = new PrimitiveTypeInfo();
|
||||
tInfo.setTypeName(gen.getHCatType().name().toLowerCase());
|
||||
break;
|
||||
}
|
||||
hCatTblCols
|
||||
.add(new HCatFieldSchema(gen.getName(), gen.getHCatType(), ""));
|
||||
.add(new HCatFieldSchema(gen.getName(), tInfo, ""));
|
||||
}
|
||||
}
|
||||
HCatSchema hCatTblSchema = new HCatSchema(hCatTblCols);
|
||||
@ -749,11 +819,28 @@ private HCatSchema generateHCatTableSchema(ColumnGenerator... extraCols)
|
||||
private HCatSchema generateHCatPartitionSchema(ColumnGenerator... extraCols)
|
||||
throws Exception {
|
||||
List<HCatFieldSchema> hCatPartCols = new ArrayList<HCatFieldSchema>();
|
||||
PrimitiveTypeInfo tInfo;
|
||||
|
||||
for (ColumnGenerator gen : extraCols) {
|
||||
if (gen.getKeyType() != KeyType.NOT_A_KEY) {
|
||||
switch(gen.getHCatType()) {
|
||||
case CHAR:
|
||||
tInfo = new CharTypeInfo(gen.getHCatPrecision());
|
||||
break;
|
||||
case VARCHAR:
|
||||
tInfo = new VarcharTypeInfo(gen.getHCatPrecision());
|
||||
break;
|
||||
case DECIMAL:
|
||||
tInfo = new DecimalTypeInfo(gen.getHCatPrecision(),
|
||||
gen.getHCatScale());
|
||||
break;
|
||||
default:
|
||||
tInfo = new PrimitiveTypeInfo();
|
||||
tInfo.setTypeName(gen.getHCatType().name().toLowerCase());
|
||||
break;
|
||||
}
|
||||
hCatPartCols
|
||||
.add(new HCatFieldSchema(gen.getName(), gen.getHCatType(), ""));
|
||||
.add(new HCatFieldSchema(gen.getName(), tInfo, ""));
|
||||
}
|
||||
}
|
||||
HCatSchema hCatPartSchema = new HCatSchema(hCatPartCols);
|
||||
@ -763,16 +850,32 @@ private HCatSchema generateHCatPartitionSchema(ColumnGenerator... extraCols)
|
||||
private HCatSchema generateHCatDynamicPartitionSchema(
|
||||
ColumnGenerator... extraCols) throws Exception {
|
||||
List<HCatFieldSchema> hCatPartCols = new ArrayList<HCatFieldSchema>();
|
||||
PrimitiveTypeInfo tInfo;
|
||||
|
||||
hCatPartCols.clear();
|
||||
boolean staticFound = false;
|
||||
for (ColumnGenerator gen : extraCols) {
|
||||
if (gen.getKeyType() != KeyType.NOT_A_KEY) {
|
||||
if (gen.getKeyType() == KeyType.STATIC_KEY && !staticFound) {
|
||||
staticFound = true;
|
||||
if (gen.getKeyType() == KeyType.STATIC_KEY) {
|
||||
continue;
|
||||
}
|
||||
switch(gen.getHCatType()) {
|
||||
case CHAR:
|
||||
tInfo = new CharTypeInfo(gen.getHCatPrecision());
|
||||
break;
|
||||
case VARCHAR:
|
||||
tInfo = new VarcharTypeInfo(gen.getHCatPrecision());
|
||||
break;
|
||||
case DECIMAL:
|
||||
tInfo = new DecimalTypeInfo(gen.getHCatPrecision(),
|
||||
gen.getHCatScale());
|
||||
break;
|
||||
default:
|
||||
tInfo = new PrimitiveTypeInfo();
|
||||
tInfo.setTypeName(gen.getHCatType().name().toLowerCase());
|
||||
break;
|
||||
}
|
||||
hCatPartCols
|
||||
.add(new HCatFieldSchema(gen.getName(), gen.getHCatType(), ""));
|
||||
.add(new HCatFieldSchema(gen.getName(), tInfo, ""));
|
||||
}
|
||||
}
|
||||
HCatSchema hCatPartSchema = new HCatSchema(hCatPartCols);
|
||||
@ -783,11 +886,29 @@ private HCatSchema generateHCatDynamicPartitionSchema(
|
||||
private HCatSchema generateHCatStaticPartitionSchema(
|
||||
ColumnGenerator... extraCols) throws Exception {
|
||||
List<HCatFieldSchema> hCatPartCols = new ArrayList<HCatFieldSchema>();
|
||||
PrimitiveTypeInfo tInfo;
|
||||
|
||||
hCatPartCols.clear();
|
||||
for (ColumnGenerator gen : extraCols) {
|
||||
if (gen.getKeyType() == KeyType.STATIC_KEY) {
|
||||
switch(gen.getHCatType()) {
|
||||
case CHAR:
|
||||
tInfo = new CharTypeInfo(gen.getHCatPrecision());
|
||||
break;
|
||||
case VARCHAR:
|
||||
tInfo = new VarcharTypeInfo(gen.getHCatPrecision());
|
||||
break;
|
||||
case DECIMAL:
|
||||
tInfo = new DecimalTypeInfo(gen.getHCatPrecision(),
|
||||
gen.getHCatScale());
|
||||
break;
|
||||
default:
|
||||
tInfo = new PrimitiveTypeInfo();
|
||||
tInfo.setTypeName(gen.getHCatType().name().toLowerCase());
|
||||
break;
|
||||
}
|
||||
hCatPartCols
|
||||
.add(new HCatFieldSchema(gen.getName(), gen.getHCatType(), ""));
|
||||
.add(new HCatFieldSchema(gen.getName(), tInfo, ""));
|
||||
break;
|
||||
}
|
||||
}
|
||||
@ -804,12 +925,9 @@ private List<HCatRecord> generateHCatRecords(int numRecords,
|
||||
DefaultHCatRecord record = new DefaultHCatRecord(size);
|
||||
record.set(hCatTblCols.get(0).getName(), hCatTblSchema, i);
|
||||
record.set(hCatTblCols.get(1).getName(), hCatTblSchema, "textfield" + i);
|
||||
boolean staticFound = false;
|
||||
int idx = 0;
|
||||
for (int j = 0; j < extraCols.length; ++j) {
|
||||
if (extraCols[j].getKeyType() == KeyType.STATIC_KEY
|
||||
&& !staticFound) {
|
||||
staticFound = true;
|
||||
if (extraCols[j].getKeyType() == KeyType.STATIC_KEY) {
|
||||
continue;
|
||||
}
|
||||
record.set(hCatTblCols.get(idx + 2).getName(), hCatTblSchema,
|
||||
|
@ -230,4 +230,79 @@ public void testHCatImportWithDatabase() throws Exception {
|
||||
};
|
||||
SqoopOptions opts = parseImportArgs(args);
|
||||
}
|
||||
|
||||
public void testHCatImportWithPartKeys() throws Exception {
|
||||
String[] args = {
|
||||
"--hcatalog-table",
|
||||
"table",
|
||||
"--hcatalog-partition-keys",
|
||||
"k1,k2",
|
||||
"--hcatalog-partition-values",
|
||||
"v1,v2",
|
||||
};
|
||||
SqoopOptions opts = parseImportArgs(args);
|
||||
}
|
||||
|
||||
public void testHCatImportWithOnlyHCatKeys() throws Exception {
|
||||
String[] args = {
|
||||
"--connect",
|
||||
"jdbc:db:url",
|
||||
"--table",
|
||||
"dbtable",
|
||||
"--hcatalog-table",
|
||||
"table",
|
||||
"--hcatalog-partition-keys",
|
||||
"k1,k2",
|
||||
};
|
||||
try {
|
||||
SqoopOptions opts = parseImportArgs(args);
|
||||
importTool.validateOptions(opts);
|
||||
fail("Expected InvalidOptionsException");
|
||||
} catch (SqoopOptions.InvalidOptionsException ioe) {
|
||||
// expected.
|
||||
}
|
||||
}
|
||||
|
||||
public void testHCatImportWithMismatchedKeysAndVals() throws Exception {
|
||||
String[] args = {
|
||||
"--connect",
|
||||
"jdbc:db:url",
|
||||
"--table",
|
||||
"dbtable",
|
||||
"--hcatalog-table",
|
||||
"table",
|
||||
"--hcatalog-partition-keys",
|
||||
"k1,k2",
|
||||
"--hcatalog-partition-values",
|
||||
"v1",
|
||||
};
|
||||
try {
|
||||
SqoopOptions opts = parseImportArgs(args);
|
||||
importTool.validateOptions(opts);
|
||||
fail("Expected InvalidOptionsException");
|
||||
} catch (SqoopOptions.InvalidOptionsException ioe) {
|
||||
// expected.
|
||||
}
|
||||
}
|
||||
|
||||
public void testHCatImportWithBothHCatAndHivePartOptions() throws Exception {
|
||||
String[] args = {
|
||||
"--connect",
|
||||
"jdbc:db:url",
|
||||
"--table",
|
||||
"dbtable",
|
||||
"--hcatalog-table",
|
||||
"table",
|
||||
"--hcatalog-partition-keys",
|
||||
"k1,k2",
|
||||
"--hcatalog-partition-values",
|
||||
"v1,v2",
|
||||
"--hive-partition-key",
|
||||
"k1",
|
||||
"--hive-partition-value",
|
||||
"v1",
|
||||
};
|
||||
SqoopOptions opts = parseImportArgs(args);
|
||||
importTool.validateOptions(opts);
|
||||
}
|
||||
}
|
||||
|
@ -25,7 +25,7 @@
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.sqoop.hcat.HCatalogExportTest;
|
||||
import org.apache.sqoop.hcat.HCatalogTestUtils;
|
||||
import org.apache.sqoop.hcat.HCatalogTestUtils.ColumnGenerator;
|
||||
@ -93,17 +93,17 @@ public void testIntTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"boolean", Types.BOOLEAN, HCatFieldSchema.Type.BOOLEAN,
|
||||
"boolean", Types.BOOLEAN, HCatFieldSchema.Type.BOOLEAN, 0, 0,
|
||||
Boolean.TRUE, Boolean.TRUE, KeyType.NOT_A_KEY),
|
||||
// Netezza does not have tinyint
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"smallint", Types.SMALLINT, HCatFieldSchema.Type.INT, 100,
|
||||
"smallint", Types.SMALLINT, HCatFieldSchema.Type.INT, 0, 0, 100,
|
||||
(short) 100, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"int", Types.INTEGER, HCatFieldSchema.Type.INT, 1000,
|
||||
"int", Types.INTEGER, HCatFieldSchema.Type.INT, 0, 0, 1000,
|
||||
1000, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(3),
|
||||
"bigint", Types.BIGINT, HCatFieldSchema.Type.BIGINT, 10000L,
|
||||
"bigint", Types.BIGINT, HCatFieldSchema.Type.BIGINT, 0, 0, 10000L,
|
||||
10000L, KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -115,10 +115,10 @@ public void testStringTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.STRING, "string to test",
|
||||
"string to test", KeyType.NOT_A_KEY),
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"string to test", "string to test", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"string to test", "string to test", KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -130,11 +130,11 @@ public void testNumberTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"numeric(18,2)", Types.NUMERIC, HCatFieldSchema.Type.STRING, "1000.00",
|
||||
new BigDecimal("1000.00"), KeyType.NOT_A_KEY),
|
||||
"numeric(18,2)", Types.NUMERIC, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1000.00", new BigDecimal("1000.00"), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"decimal(18,2)", Types.DECIMAL, HCatFieldSchema.Type.STRING, "2000.00",
|
||||
new BigDecimal("2000.00"), KeyType.NOT_A_KEY),
|
||||
"decimal(18,2)", Types.DECIMAL, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2000.00", new BigDecimal("2000.00"), KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
runHCatExport(addlArgsArray, TOTAL_RECORDS, table, cols);
|
||||
|
@ -25,7 +25,7 @@
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
|
||||
import org.apache.sqoop.hcat.HCatalogImportTest;
|
||||
import org.apache.sqoop.hcat.HCatalogTestUtils;
|
||||
import org.apache.sqoop.hcat.HCatalogTestUtils.ColumnGenerator;
|
||||
@ -93,17 +93,17 @@ public void testIntTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"boolean", Types.BOOLEAN, HCatFieldSchema.Type.BOOLEAN,
|
||||
"boolean", Types.BOOLEAN, HCatFieldSchema.Type.BOOLEAN, 0, 0,
|
||||
Boolean.TRUE, Boolean.TRUE, KeyType.NOT_A_KEY),
|
||||
// Netezza does not have tinyint
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"smallint", Types.INTEGER, HCatFieldSchema.Type.INT, 100,
|
||||
"smallint", Types.INTEGER, HCatFieldSchema.Type.INT, 0, 0, 100,
|
||||
100, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(2),
|
||||
"int", Types.INTEGER, HCatFieldSchema.Type.INT, 1000,
|
||||
"int", Types.INTEGER, HCatFieldSchema.Type.INT, 0, 0, 1000,
|
||||
1000, KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(3),
|
||||
"bigint", Types.BIGINT, HCatFieldSchema.Type.BIGINT, 10000L,
|
||||
"bigint", Types.BIGINT, HCatFieldSchema.Type.BIGINT, 0, 0, 10000L,
|
||||
10000L, KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -116,10 +116,10 @@ public void testStringTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.STRING, "string to test",
|
||||
"string to test", KeyType.NOT_A_KEY),
|
||||
"char(14)", Types.CHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"string to test", "string to test", KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING,
|
||||
"varchar(20)", Types.VARCHAR, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"string to test", "string to test", KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
@ -132,11 +132,11 @@ public void testNumberTypes() throws Exception {
|
||||
String table = getTableName().toUpperCase();
|
||||
ColumnGenerator[] cols = new ColumnGenerator[] {
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(0),
|
||||
"numeric(18,2)", Types.NUMERIC, HCatFieldSchema.Type.STRING, "1000.00",
|
||||
new BigDecimal("1000"), KeyType.NOT_A_KEY),
|
||||
"numeric(18,2)", Types.NUMERIC, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"1000.00", new BigDecimal("1000"), KeyType.NOT_A_KEY),
|
||||
HCatalogTestUtils.colGenerator(HCatalogTestUtils.forIdx(1),
|
||||
"decimal(18,2)", Types.DECIMAL, HCatFieldSchema.Type.STRING, "2000.00",
|
||||
new BigDecimal("2000"), KeyType.NOT_A_KEY),
|
||||
"decimal(18,2)", Types.DECIMAL, HCatFieldSchema.Type.STRING, 0, 0,
|
||||
"2000.00", new BigDecimal("2000"), KeyType.NOT_A_KEY),
|
||||
};
|
||||
List<String> addlArgsArray = new ArrayList<String>();
|
||||
setExtraArgs(addlArgsArray);
|
||||
|
Loading…
Reference in New Issue
Block a user