mirror of
https://github.com/apache/sqoop.git
synced 2025-05-03 10:41:57 +08:00
SQOOP-1393: Import data from database to Hive as Parquet files
(Richard via Jarek Jarcec Cecho)
This commit is contained in:
parent
2e1e09422c
commit
e2544a9a92
4
ivy.xml
4
ivy.xml
@ -183,7 +183,9 @@ under the License.
|
|||||||
conf="common->default;redist->default"/>
|
conf="common->default;redist->default"/>
|
||||||
<dependency org="commons-io" name="commons-io" rev="${commons-io.version}"
|
<dependency org="commons-io" name="commons-io" rev="${commons-io.version}"
|
||||||
conf="common->default;redist->default"/>
|
conf="common->default;redist->default"/>
|
||||||
<dependency org="org.kitesdk" name="kite-data-mapreduce" rev="${kite-data-mapreduce.version}"
|
<dependency org="org.kitesdk" name="kite-data-mapreduce" rev="${kite-data.version}"
|
||||||
|
conf="avro->default;redist->default"/>
|
||||||
|
<dependency org="org.kitesdk" name="kite-data-hcatalog" rev="${kite-data.version}"
|
||||||
conf="avro->default;redist->default"/>
|
conf="avro->default;redist->default"/>
|
||||||
|
|
||||||
<!-- dependencies for static analysis -->
|
<!-- dependencies for static analysis -->
|
||||||
|
@ -20,7 +20,7 @@
|
|||||||
|
|
||||||
avro.version=1.7.5
|
avro.version=1.7.5
|
||||||
|
|
||||||
kite-data-mapreduce.version=0.15.0
|
kite-data.version=0.16.0
|
||||||
|
|
||||||
checkstyle.version=5.0
|
checkstyle.version=5.0
|
||||||
|
|
||||||
|
@ -106,7 +106,7 @@ protected void configureMapper(Job job, String tableName,
|
|||||||
Schema schema = new Schema.Parser().parse(conf.get("avro.schema"));
|
Schema schema = new Schema.Parser().parse(conf.get("avro.schema"));
|
||||||
String uri = "";
|
String uri = "";
|
||||||
if (options.doHiveImport()) {
|
if (options.doHiveImport()) {
|
||||||
// TODO: SQOOP-1393
|
uri = "dataset:hive?dataset=" + options.getTableName();
|
||||||
} else {
|
} else {
|
||||||
FileSystem fs = FileSystem.get(conf);
|
FileSystem fs = FileSystem.get(conf);
|
||||||
uri = "dataset:" + fs.makeQualified(getContext().getDestination());
|
uri = "dataset:" + fs.makeQualified(getContext().getDestination());
|
||||||
|
@ -180,6 +180,21 @@ protected void cacheJars(Job job, ConnManager mgr)
|
|||||||
+ "all job dependencies.");
|
+ "all job dependencies.");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// If the user run import into hive as Parquet file,
|
||||||
|
// Add anything in $HIVE_HOME/lib.
|
||||||
|
if (options.doHiveImport() && (options.getFileLayout() == SqoopOptions.FileLayout.ParquetFile)) {
|
||||||
|
String hiveHome = options.getHiveHome();
|
||||||
|
if (null != hiveHome) {
|
||||||
|
File hiveHomeFile = new File(hiveHome);
|
||||||
|
File hiveLibFile = new File(hiveHomeFile, "lib");
|
||||||
|
if (hiveLibFile.exists()) {
|
||||||
|
addDirToCache(hiveLibFile, fs, localUrls);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
LOG.warn("HIVE_HOME is unset. Cannot add hive libs as dependencies.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// If we didn't put anything in our set, then there's nothing to cache.
|
// If we didn't put anything in our set, then there's nothing to cache.
|
||||||
if (localUrls.isEmpty()) {
|
if (localUrls.isEmpty()) {
|
||||||
return;
|
return;
|
||||||
|
@ -117,12 +117,15 @@ public int run(SqoopOptions options) {
|
|||||||
// them to files (but don't actually perform the import -- thus
|
// them to files (but don't actually perform the import -- thus
|
||||||
// the generateOnly=true in the constructor).
|
// the generateOnly=true in the constructor).
|
||||||
if (options.doHiveImport()) {
|
if (options.doHiveImport()) {
|
||||||
HiveImport hiveImport = new HiveImport(options, manager,
|
// For Parquet file, the import action will create hive table directly
|
||||||
options.getConf(), true);
|
// via kite. So there is no need to create hive table again.
|
||||||
hiveImport.importTable(options.getTableName(),
|
if (options.getFileLayout() != SqoopOptions.FileLayout.ParquetFile) {
|
||||||
options.getHiveTableName(), true);
|
HiveImport hiveImport = new HiveImport(options, manager,
|
||||||
|
options.getConf(), true);
|
||||||
|
hiveImport.importTable(options.getTableName(),
|
||||||
|
options.getHiveTableName(), true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
LOG.error("Encountered IOException running codegen job: "
|
LOG.error("Encountered IOException running codegen job: "
|
||||||
+ StringUtils.stringifyException(ioe));
|
+ StringUtils.stringifyException(ioe));
|
||||||
|
@ -508,7 +508,11 @@ protected boolean importTable(SqoopOptions options, String tableName,
|
|||||||
|
|
||||||
// If the user wants this table to be in Hive, perform that post-load.
|
// If the user wants this table to be in Hive, perform that post-load.
|
||||||
if (options.doHiveImport()) {
|
if (options.doHiveImport()) {
|
||||||
hiveImport.importTable(tableName, options.getHiveTableName(), false);
|
// For Parquet file, the import action will create hive table directly via
|
||||||
|
// kite. So there is no need to do hive import as a post step again.
|
||||||
|
if (options.getFileLayout() != SqoopOptions.FileLayout.ParquetFile) {
|
||||||
|
hiveImport.importTable(tableName, options.getHiveTableName(), false);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
saveIncrementalState(options);
|
saveIncrementalState(options);
|
||||||
|
@ -24,6 +24,7 @@
|
|||||||
import java.io.FileReader;
|
import java.io.FileReader;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
@ -262,6 +263,22 @@ public void testNormalHiveImport() throws IOException {
|
|||||||
getArgv(false, null), new ImportTool());
|
getArgv(false, null), new ImportTool());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Test that strings and ints are handled in the normal fashion as parquet
|
||||||
|
* file. */
|
||||||
|
@Test
|
||||||
|
public void testNormalHiveImportAsParquet() throws IOException {
|
||||||
|
final String TABLE_NAME = "NORMAL_HIVE_IMPORT_AS_PARQUET";
|
||||||
|
setCurTableName(TABLE_NAME);
|
||||||
|
setNumCols(3);
|
||||||
|
String [] types = { "VARCHAR(32)", "INTEGER", "CHAR(64)" };
|
||||||
|
String [] vals = { "'test'", "42", "'somestring'" };
|
||||||
|
String [] args_array = getArgv(false, null);
|
||||||
|
ArrayList<String> args = new ArrayList<String>(Arrays.asList(args_array));
|
||||||
|
args.add("--as-parquetfile");
|
||||||
|
runImportTest(TABLE_NAME, types, vals, "normalImportAsParquet.q", args.toArray(new String[0]),
|
||||||
|
new ImportTool());
|
||||||
|
}
|
||||||
|
|
||||||
/** Test that table is created in hive with no data import. */
|
/** Test that table is created in hive with no data import. */
|
||||||
@Test
|
@Test
|
||||||
public void testCreateOnlyHiveImport() throws IOException {
|
public void testCreateOnlyHiveImport() throws IOException {
|
||||||
|
17
testdata/hive/scripts/normalImportAsParquet.q
vendored
Normal file
17
testdata/hive/scripts/normalImportAsParquet.q
vendored
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
-- Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
-- or more contributor license agreements. See the NOTICE file
|
||||||
|
-- distributed with this work for additional information
|
||||||
|
-- regarding copyright ownership. The ASF licenses this file
|
||||||
|
-- to you under the Apache License, Version 2.0 (the
|
||||||
|
-- "License"); you may not use this file except in compliance
|
||||||
|
-- with the License. You may obtain a copy of the License at
|
||||||
|
--
|
||||||
|
-- http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
--
|
||||||
|
-- Unless required by applicable law or agreed to in writing, software
|
||||||
|
-- distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
-- See the License for the specific language governing permissions and
|
||||||
|
-- limitations under the License.
|
||||||
|
CREATE TABLE IF NOT EXISTS `NORMAL_HIVE_IMPORT_AS_PARQUET` ( `DATA_COL0` STRING, `DATA_COL1` INT, `DATA_COL2` STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\001' LINES TERMINATED BY '\012' STORED AS PARQUET;
|
||||||
|
LOAD DATA INPATH 'file:BASEPATH/sqoop/warehouse/NORMAL_HIVE_IMPORT_AS_PARQUET' INTO TABLE `NORMAL_HIVE_IMPORT_AS_PARQUET`;
|
Loading…
Reference in New Issue
Block a user