Details
-
Improvement
-
Status: Open
-
Major
-
Resolution: Unresolved
-
4.0.0-alpha-2
-
None
Description
Step to reproduce:
- Create hive_catalog iceberg table by Trino/Presto/Flink (Spark with iceberg.engine.hive.enabled disabled)
- show table info with hive beeline:
+----------------------------------------------------+ | createtab_stmt | +----------------------------------------------------+ | CREATE EXTERNAL TABLE `iceberg_hive`.`testtrinoice`( | | `id` int) | | ROW FORMAT SERDE | | 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' | | STORED AS INPUTFORMAT | | 'org.apache.hadoop.mapred.FileInputFormat' | | OUTPUTFORMAT | | 'org.apache.hadoop.mapred.FileOutputFormat' | | LOCATION | | 'hdfs://localhost:8020/iceberg_hive.db/testtrinoice-08642c05e622415ab3e2da4b4c35224d' | | TBLPROPERTIES ( | | 'metadata_location'='hdfs://localhost:8020/iceberg_hive.db/testtrinoice-08642c05e622415ab3e2da4b4c35224d/metadata/00000-3303dd99-e4d1-4cb0-9d12-9744cbe0a1c9.metadata.json', | | 'table_type'='iceberg', | | 'transient_lastDdlTime'='1667292082') | +----------------------------------------------------+
You can see that the iceberg table created by trino has no iceberg inputformat/outputformat which is used to read/write iceberg data for HS2.
- Query this iceberg table with HS2:
select * from iceberg_hive.testtrinoice;
ERROR : Failed with exception java.io.IOException:java.io.IOException: Cannot create an instance of InputFormat class org.apache.hadoop.mapred.FileInputFormat as specified in mapredWork! java.io.IOException: java.io.IOException: Cannot create an instance of InputFormat class org.apache.hadoop.mapred.FileInputFormat as specified in mapredWork! at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextRow(FetchOperator.java:624) at org.apache.hadoop.hive.ql.exec.FetchOperator.pushRow(FetchOperator.java:531) at org.apache.hadoop.hive.ql.exec.FetchTask.executeInner(FetchTask.java:197) at org.apache.hadoop.hive.ql.exec.FetchTask.execute(FetchTask.java:98) at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:212) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:154) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:149) at org.apache.hadoop.hive.ql.reexec.ReExecDriver.run(ReExecDriver.java:185) at org.apache.hive.service.cli.operation.SQLOperation.runQuery(SQLOperation.java:234) at org.apache.hive.service.cli.operation.SQLOperation.access$500(SQLOperation.java:88) at org.apache.hive.service.cli.operation.SQLOperation$BackgroundWork$1.run(SQLOperation.java:337) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1878) at org.apache.hive.service.cli.operation.SQLOperation$BackgroundWork.run(SQLOperation.java:357) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.io.IOException: Cannot create an instance of InputFormat class org.apache.hadoop.mapred.FileInputFormat as specified in mapredWork! at org.apache.hadoop.hive.ql.exec.FetchOperator.getInputFormatFromCache(FetchOperator.java:233) at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextSplits(FetchOperator.java:374) at org.apache.hadoop.hive.ql.exec.FetchOperator.getRecordReader(FetchOperator.java:306) at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextRow(FetchOperator.java:562) ... 19 more Caused by: java.lang.RuntimeException: java.lang.InstantiationException at org.apache.hive.common.util.ReflectionUtil.newInstance(ReflectionUtil.java:85) at org.apache.hadoop.hive.ql.exec.FetchOperator.getInputFormatFromCache(FetchOperator.java:229) ... 22 more Caused by: java.lang.InstantiationException at sun.reflect.InstantiationExceptionConstructorAccessorImpl.newInstance(InstantiationExceptionConstructorAccessorImpl.java:48) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.hive.common.util.ReflectionUtil.newInstance(ReflectionUtil.java:83) ... 23 more
Attachments
Issue Links
- is related to
-
HIVE-25226 Hive changes 'storage_handler' for existing Iceberg table when hive.engine.enabled is false
- Resolved
- links to