You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hive.apache.org by "t oo (JIRA)" <ji...@apache.org> on 2018/10/25 01:35:00 UTC

[jira] [Created] (HIVE-20803) Hive can't read S3 parquet file with timestamp partition

t oo created HIVE-20803:
---------------------------

             Summary: Hive can't read S3 parquet file with timestamp partition
                 Key: HIVE-20803
                 URL: https://issues.apache.org/jira/browse/HIVE-20803
             Project: Hive
          Issue Type: Bug
          Components: SQL
    Affects Versions: 2.3.2
            Reporter: t oo


SparkThriftServer can select * from the table fine and get data. But HiveServer2 throws below error on select *:

 

hive.msck.path.validation = ignore in hive-site.xml
then ran msck repair my_sch.h_l

aws s3 ls s3://priv1/priv2/H_L/ --recursive
2018-10-18 03:00:56 2474983 priv1/priv2/H_L/part_dt=20180309/part_src=xyz/part_src_file=MY_LOC/part_ldts=2018-10-18 02:59:46/part-00000-2536ca01-243c-4220-8e55-6869a045fba2.snappy.parquet


show create table my_sch.h_l;
+----------------------------------------------------+
| createtab_stmt |
+----------------------------------------------------+
| CREATE EXTERNAL TABLE `my_sch.h_l`( |
| `xy_hkey_h_l` binary, |
| `xy_rtts` timestamp, |
| `xy_rsrc` string, |
| `xy_bkcc` string, |
| `xy_mltid` string, |
| `location_id` bigint) |
| PARTITIONED BY ( |
| `part_dt` string, |
| `part_src` string, |
| `part_src_file` string, |
| `part_ldts` timestamp) |
| ROW FORMAT SERDE |
| 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' |
| STORED AS INPUTFORMAT |
| 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' |
| OUTPUTFORMAT |
| 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' |
| LOCATION |
| 's3a://priv1/priv2/H_L' |
| TBLPROPERTIES ( |
| 'spark.sql.partitionProvider'='catalog', |
| 'spark.sql.sources.schema.numPartCols'='4', |
| 'spark.sql.sources.schema.numParts'='1', |
| 'spark.sql.sources.schema.part.0'='\{"type":"struct","fields":[{"name":"xy_hkey_h_l","type":"binary","nullable":true,"metadata":{}},\{"name":"xy_rtts","type":"timestamp","nullable":true,"metadata":{}},\{"name":"xy_rsrc","type":"string","nullable":true,"metadata":{}},\{"name":"xy_bkcc","type":"string","nullable":true,"metadata":{}},\{"name":"xy_mltid","type":"string","nullable":true,"metadata":{}},\{"name":"location_id","type":"long","nullable":true,"metadata":{}},\{"name":"part_dt","type":"string","nullable":true,"metadata":{}},\{"name":"part_src","type":"string","nullable":true,"metadata":{}},\{"name":"part_src_file","type":"string","nullable":true,"metadata":{}},\{"name":"part_ldts","type":"timestamp","nullable":true,"metadata":{}}]}', |
| 'spark.sql.sources.schema.partCol.0'='part_dt', |
| 'spark.sql.sources.schema.partCol.1'='part_src', |
| 'spark.sql.sources.schema.partCol.2'='part_src_file', |
| 'spark.sql.sources.schema.partCol.3'='part_ldts', |
| 'transient_lastDdlTime'='1540421484') |
+----------------------------------------------------+


 select * from my_sch.h_l limit 5;
Error: java.io.IOException: java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: part_ldts=2018-10-18 02:59:46 (state=,code=0)
org.apache.hive.service.cli.HiveSQLException: java.io.IOException: java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: part_ldts=2018-10-18 02:59:46
 at org.apache.hive.jdbc.Utils.verifySuccess(Utils.java:267)
 at org.apache.hive.jdbc.Utils.verifySuccessWithInfo(Utils.java:253)
 at org.apache.hive.jdbc.HiveQueryResultSet.next(HiveQueryResultSet.java:374)
 at org.apache.hive.beeline.BufferedRows.<init>(BufferedRows.java:53)
 at org.apache.hive.beeline.IncrementalRowsWithNormalization.<init>(IncrementalRowsWithNormalization.java:50)
 at org.apache.hive.beeline.BeeLine.print(BeeLine.java:2192)
 at org.apache.hive.beeline.Commands.executeInternal(Commands.java:1009)
 at org.apache.hive.beeline.Commands.execute(Commands.java:1205)
 at org.apache.hive.beeline.Commands.sql(Commands.java:1134)
 at org.apache.hive.beeline.BeeLine.dispatch(BeeLine.java:1314)
 at org.apache.hive.beeline.BeeLine.execute(BeeLine.java:1178)
 at org.apache.hive.beeline.BeeLine.begin(BeeLine.java:1033)
 at org.apache.hive.beeline.BeeLine.mainWithInputRedirection(BeeLine.java:519)
 at org.apache.hive.beeline.BeeLine.main(BeeLine.java:501)
 at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
 at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
 at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 at java.lang.reflect.Method.invoke(Method.java:498)
 at org.apache.hadoop.util.RunJar.run(RunJar.java:239)
 at org.apache.hadoop.util.RunJar.main(RunJar.java:153)
Caused by: org.apache.hive.service.cli.HiveSQLException: java.io.IOException: java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: part_ldts=2018-10-18 02:59:46
 at org.apache.hive.service.cli.operation.SQLOperation.getNextRowSet(SQLOperation.java:499)
 at org.apache.hive.service.cli.operation.OperationManager.getOperationNextRowSet(OperationManager.java:307)
 at org.apache.hive.service.cli.session.HiveSessionImpl.fetchResults(HiveSessionImpl.java:878)
 at sun.reflect.GeneratedMethodAccessor51.invoke(Unknown Source)
 at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 at java.lang.reflect.Method.invoke(Method.java:498)
 at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:78)
 at org.apache.hive.service.cli.session.HiveSessionProxy.access$000(HiveSessionProxy.java:36)
 at org.apache.hive.service.cli.session.HiveSessionProxy$1.run(HiveSessionProxy.java:63)
 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:1844)
 at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:59)
 at com.sun.proxy.$Proxy42.fetchResults(Unknown Source)
 at org.apache.hive.service.cli.CLIService.fetchResults(CLIService.java:559)
 at org.apache.hive.service.cli.thrift.ThriftCLIService.FetchResults(ThriftCLIService.java:751)
 at org.apache.hive.service.rpc.thrift.TCLIService$Processor$FetchResults.getResult(TCLIService.java:1717)
 at org.apache.hive.service.rpc.thrift.TCLIService$Processor$FetchResults.getResult(TCLIService.java:1702)
 at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
 at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
 at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:56)
 at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 at java.lang.Thread.run(Thread.java:748)
Caused by: java.io.IOException: java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: part_ldts=2018-10-18 02:59:46
 at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextRow(FetchOperator.java:521)
 at org.apache.hadoop.hive.ql.exec.FetchOperator.pushRow(FetchOperator.java:428)
 at org.apache.hadoop.hive.ql.exec.FetchTask.fetch(FetchTask.java:147)
 at org.apache.hadoop.hive.ql.Driver.getResults(Driver.java:2208)
 at org.apache.hive.service.cli.operation.SQLOperation.getNextRowSet(SQLOperation.java:494)
 ... 24 more
Caused by: java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: part_ldts=2018-10-18 02:59:46
 at org.apache.hadoop.fs.Path.initialize(Path.java:254)
 at org.apache.hadoop.fs.Path.<init>(Path.java:212)
 at org.apache.hadoop.fs.Path.<init>(Path.java:120)
 at org.apache.hadoop.fs.Globber.doGlob(Globber.java:269)
 at org.apache.hadoop.fs.Globber.glob(Globber.java:148)
 at org.apache.hadoop.fs.FileSystem.globStatus(FileSystem.java:1705)
 at org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:2031)
 at org.apache.hadoop.mapred.FileInputFormat.singleThreadedListStatus(FileInputFormat.java:266)
 at org.apache.hadoop.mapred.FileInputFormat.listStatus(FileInputFormat.java:236)
 at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:322)
 at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextSplits(FetchOperator.java:372)
 at org.apache.hadoop.hive.ql.exec.FetchOperator.getRecordReader(FetchOperator.java:304)
 at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextRow(FetchOperator.java:459)
 ... 28 more
Caused by: java.lang.RuntimeException: java.net.URISyntaxException:Relative path in absolute URI: part_ldts=2018-10-18 02:59:46
 at java.net.URI.checkPath(URI.java:1823)
 at java.net.URI.<init>(URI.java:745)
 at org.apache.hadoop.fs.Path.initialize(Path.java:251)
 ... 40 more



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)