Regard SQOOP-1524, we have removed dependency parquet-hive-bundle-1.4.1.jar. The change is now in trunk. Can you try it out?
-- Xu Qian (Stanely) From: pratik khadloya [mailto:[email protected]] Sent: Saturday, September 13, 2014 8:24 AM To: [email protected] Subject: Re: Hive import is not compatible with importing into AVRO format Oops, looks like The following error states that one should not use the MapredParquetOutputFormat (because Hive 0.13 has native parquet support). java.lang.RuntimeException: Should never be used at org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat.getRecordWriter(MapredParquetOutputFormat.java:76) So, to conclude the --as-parquet support would only work for hive 0.12 Thanks, ~Pratik On Fri, Sep 12, 2014 at 5:06 PM, pratik khadloya <[email protected]<mailto:[email protected]>> wrote: I see that the parquet-hive-bundle which includes parquet-hive-binding-bundle does not have a support for hive 0.13. It only has bindings upto hive parquet-hive-0.12-binding as seen from https://github.com/Parquet/parquet-mr/blob/master/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/pom.xml ~Pratik On Fri, Sep 12, 2014 at 4:34 PM, pratik khadloya <[email protected]<mailto:[email protected]>> wrote: Even when i use the latest parquet-hive-bundle-1.6.0rc1.jar, i see the error "java.lang.NoSuchFieldError: doubleTypeInfo" in the hive query logs. 014-09-12 19:32:49,364 INFO [main]: ql.Driver (SessionState.java:printInfo(543)) - Launching Job 1 out of 0 2014-09-12 19:32:49,398 ERROR [main]: exec.DDLTask (DDLTask.java:execute(478)) - java.lang.NoSuchFieldError: doubleTypeInfo at org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector.getObjectInspector(ArrayWritableObjectInspector.java:66) at org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector.<init>(ArrayWritableObjectInspector.java:59) at org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe.initialize(ParquetHiveSerDe.java:113) at org.apache.hadoop.hive.metastore.MetaStoreUtils.getDeserializer(MetaStoreUtils.java:339) at org.apache.hadoop.hive.ql.metadata.Table.getDeserializerFromMetaStore(Table.java:288) at org.apache.hadoop.hive.ql.metadata.Table.checkValidity(Table.java:194) at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:597) at org.apache.hadoop.hive.ql.exec.DDLTask.createTable(DDLTask.java:4194) at org.apache.hadoop.hive.ql.exec.DDLTask.execute(DDLTask.java:281) at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:153) at org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:85) at org.apache.hadoop.hive.ql.Driver.launchTask(Driver.java:1524) at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:1288) at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1099) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:917) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:907) at org.apache.hive.hcatalog.cli.HCatDriver.run(HCatDriver.java:43) at org.apache.hive.hcatalog.cli.HCatCli.processCmd(HCatCli.java:270) at org.apache.hive.hcatalog.cli.HCatCli.processLine(HCatCli.java:224) at org.apache.hive.hcatalog.cli.HCatCli.processFile(HCatCli.java:243) at org.apache.hive.hcatalog.cli.HCatCli.main(HCatCli.java:188) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.util.RunJar.main(RunJar.java:208) On Fri, Sep 12, 2014 at 4:28 PM, pratik khadloya <[email protected]<mailto:[email protected]>> wrote: I realized that the error "org.apache.hadoop.hive.ql.exec.DDLTask. doubleTypeInfo" was coming due to version mismatch between what sqoop has in its lib vs the hive installation (0.13) on our machine. When i removed the parquet-hive-bundle-1.4.1.jar from sqoop's lib directory, the sqoop proceeded further and could actually create a table in hive. But when it started importing the data from mysql, it understandably failed with the following error since the jar it needs parquet-hive-bundle-1.4.1.jar is missing. java.lang.RuntimeException: Should never be used at org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat.getRecordWriter(MapredParquetOutputFormat.java:76) at org.apache.hive.hcatalog.mapreduce.FileOutputFormatContainer.getRecordWriter(FileOutputFormatContainer.java:103) at org.apache.hive.hcatalog.mapreduce.HCatOutputFormat.getRecordWriter(HCatOutputFormat.java:260) at org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.<init>(MapTask.java:548) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:653) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:330) at org.apache.hadoop.mapred.Child$4.run(Child.java:268) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:396) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408) at org.apache.hadoop.mapred.Child.main(Child.java:262) I will try to figure out how to make sqoop (trunk) as well as hive 0.13 happy at the same time. Will need to check which version of hive is bundled into parquet-hive-bundle-1.4.1.jar Thanks, ~Pratik On Fri, Sep 12, 2014 at 12:02 PM, pratik khadloya <[email protected]<mailto:[email protected]>> wrote: Am using sqoop from trunk. On Fri, Sep 12, 2014 at 12:02 PM, pratik khadloya <[email protected]<mailto:[email protected]>> wrote: Am getting the following error when i am trying to import a table in parquet format into hive using hcatalog. $ bin/sqoop import -jt myjt:xxxx --connect jdbc:mysql://mydbserver.net/mydb<http://mydbserver.net/mydb> --username myuser --password mypwd --query "SELECT.... WHERE \$CONDITIONS" --num-mappers 1 --hcatalog-storage-stanza "STORED AS PARQUET" --create-hcatalog-table --hcatalog-table abc1234 Please set $HBASE_HOME to the root of your HBase installation. Warning: /home/pkhadloya/sqoop-57336d7/bin/../../accumulo does not exist! Accumulo imports will fail. Please set $ACCUMULO_HOME to the root of your Accumulo installation. 14/09/12 14:58:31 INFO sqoop.Sqoop: Running Sqoop version: 1.4.6-SNAPSHOT 14/09/12 14:58:31 WARN tool.BaseSqoopTool: Setting your password on the command-line is insecure. Consider using -P instead. 14/09/12 14:58:31 INFO manager.SqlManager: Using default fetchSize of 1000 14/09/12 14:58:31 INFO tool.CodeGenTool: Beginning code generation 14/09/12 14:58:31 INFO manager.SqlManager: Executing SQL statement: SELECT ... 14/09/12 14:58:31 INFO manager.SqlManager: Executing SQL statement: SELECT ... 14/09/12 14:58:31 INFO manager.SqlManager: Executing SQL statement: SELECT ... 14/09/12 14:58:31 INFO orm.CompilationManager: HADOOP_MAPRED_HOME is /usr/lib/hadoop-0.20-mapreduce Note: /tmp/sqoop-myuser/compile/a8858915f0a296d14457738acc0f6f77/QueryResult.java uses or overrides a deprecated API. Note: Recompile with -Xlint:deprecation for details. 14/09/12 14:58:33 INFO orm.CompilationManager: Writing jar file: /tmp/sqoop-myuser/compile/a8858915f0a296d14457738acc0f6f77/QueryResult.jar 14/09/12 14:58:33 INFO mapreduce.ImportJobBase: Beginning query import. 14/09/12 14:58:33 INFO hcat.SqoopHCatUtilities: Configuring HCatalog for import job 14/09/12 14:58:33 INFO hcat.SqoopHCatUtilities: Configuring HCatalog specific details for job 14/09/12 14:58:33 INFO manager.SqlManager: Executing SQL statement: SELECT... 14/09/12 14:58:33 INFO manager.SqlManager: Executing SQL statement: SELECT... 14/09/12 14:58:33 INFO hcat.SqoopHCatUtilities: Database column names projected : [sid, pid, pna] 14/09/12 14:58:33 INFO hcat.SqoopHCatUtilities: Database column name - info map : sid : [Type : -5,Precision : 20,Scale : 0] pid : [Type : -5,Precision : 20,Scale : 0] pna : [Type : 12,Precision : 255,Scale : 0] 14/09/12 14:58:33 INFO hcat.SqoopHCatUtilities: Creating HCatalog table default.abc1234 for import 14/09/12 14:58:33 INFO hcat.SqoopHCatUtilities: HCatalog Create table statement: create table `default`.`abc1234` ( `sid` bigint, `pid` bigint, `pna` varchar(255)) STORED AS PARQUET 14/09/12 14:58:33 INFO hcat.SqoopHCatUtilities: Executing external HCatalog CLI process with args :-f,/tmp/hcat-script-1410548313797 14/09/12 14:58:39 INFO hcat.SqoopHCatUtilities: Launching Job 1 out of 0 14/09/12 14:58:39 INFO hcat.SqoopHCatUtilities: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. doubleTypeInfo 14/09/12 14:58:39 ERROR tool.ImportTool: Encountered IOException running import job: java.io.IOException: HCat exited with status 1 at org.apache.sqoop.mapreduce.hcat.SqoopHCatUtilities.executeExternalHCatProgram(SqoopHCatUtilities.java:1113) at org.apache.sqoop.mapreduce.hcat.SqoopHCatUtilities.launchHCatCli(SqoopHCatUtilities.java:1062) at org.apache.sqoop.mapreduce.hcat.SqoopHCatUtilities.createHCatTable(SqoopHCatUtilities.java:595) at org.apache.sqoop.mapreduce.hcat.SqoopHCatUtilities.configureHCat(SqoopHCatUtilities.java:318) at org.apache.sqoop.mapreduce.hcat.SqoopHCatUtilities.configureImportOutputFormat(SqoopHCatUtilities.java:753) at org.apache.sqoop.mapreduce.ImportJobBase.configureOutputFormat(ImportJobBase.java:98) at org.apache.sqoop.mapreduce.ImportJobBase.runImport(ImportJobBase.java:252) at org.apache.sqoop.manager.SqlManager.importQuery(SqlManager.java:721) at org.apache.sqoop.tool.ImportTool.importTable(ImportTool.java:499) at org.apache.sqoop.tool.ImportTool.run(ImportTool.java:605) at org.apache.sqoop.Sqoop.run(Sqoop.java:143) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70) at org.apache.sqoop.Sqoop.runSqoop(Sqoop.java:179) at org.apache.sqoop.Sqoop.runTool(Sqoop.java:218) at org.apache.sqoop.Sqoop.runTool(Sqoop.java:227) at org.apache.sqoop.Sqoop.main(Sqoop.java:236) On Fri, Sep 12, 2014 at 11:40 AM, pratik khadloya <[email protected]<mailto:[email protected]>> wrote: Thanks Venkat. Would importing the table as a hcat table instead of a hive table automatically put it in hive? ~Pratik On Fri, Sep 12, 2014 at 10:22 AM, Venkat Ranganathan <[email protected]<mailto:[email protected]>> wrote: Generally, you should be able to use any storage format that hive supports with hcatalog import or export (of course some formats may not work if they don't support the hcatalog used hive serde methods like parquet for example - but you can directly import to parquet with --as-parquetfile Instead of --hive-import and --hive-table, just use --hcatalog-table <hivetablename> Venkat On Fri, Sep 12, 2014 at 10:12 AM, pratik khadloya <[email protected]<mailto:[email protected]>> wrote: Do we need HCAT_HOME if i am only importing to hive? I don't think i have hcatalog installed. ~Pratik On Thu, Sep 11, 2014 at 7:16 PM, Xu, Qian A <[email protected]<mailto:[email protected]>> wrote: Yes. Simply replace `--as-avrodatafile` with `--as-parquetfile`. Please make sure the environment variables HIVE_HOME and HCAT_HOME are set correctly. -- Qian Xu (Stanley) From: pratik khadloya [mailto:[email protected]<mailto:[email protected]>] Sent: Friday, September 12, 2014 10:12 AM To: [email protected]<mailto:[email protected]> Subject: Re: Hive import is not compatible with importing into AVRO format Oh ok, thanks for the information Xu. Can it be invoked using --as-parquetfile with --hive-import ? Regards, Pratik On Thu, Sep 11, 2014 at 6:17 PM, Xu, Qian A <[email protected]<mailto:[email protected]>> wrote: Unfortunately, Avro format is not supported for a Hive import. You can fire a JIRA for that. Note that the trunk version of Sqoop1 supports Hive import as Parquet. -- Qian Xu (Stanley) From: [email protected]<mailto:[email protected]> [mailto:[email protected]<mailto:[email protected]>] Sent: Friday, September 12, 2014 8:56 AM To: [email protected]<mailto:[email protected]> Subject: Re: Hive import is not compatible with importing into AVRO format Hey,there: Does hive support the format of avroFile.As I know it just supoort rcfile,textfile,sequencefile.Hope this helpful to you. From: pratik khadloya<mailto:[email protected]> Date: 2014-09-12 08:26 To: [email protected]<mailto:[email protected]> Subject: Hive import is not compatible with importing into AVRO format I am trying to import data from a free form mysql query into hive. I need the files to be as AVRO data files, but when i pass the --as-avrodatafile option, i get a compatibility error. Is there a way i can tell sqoop to use the avro file format? $ bin/sqoop import -jt <jobtracker> --connect jdbc:mysql://<mydbserver>/<mydb> --username <dbuser> --password <dbpwd> --target-dir /user/pkhadloya/sqoop/mytable --query “<my query> WHERE \$CONDITIONS" --num-mappers 1 --hive-import --hive-table mytable --create-hive-table --as-avrodatafile ~Pratik CONFIDENTIALITY NOTICE NOTICE: This message is intended for the use of the individual or entity to which it is addressed and may contain information that is confidential, privileged and exempt from disclosure under applicable law. If the reader of this message is not the intended recipient, you are hereby notified that any printing, copying, dissemination, distribution, disclosure or forwarding of this communication is strictly prohibited. If you have received this communication in error, please contact the sender immediately and delete it from your system. Thank You.
