laystax created FLINK-20710:
-------------------------------

             Summary: HiveTableInputFormat throws IllegalArgumentException when 
creating splits for tables with partitions on different hdfs nameservices
                 Key: FLINK-20710
                 URL: https://issues.apache.org/jira/browse/FLINK-20710
             Project: Flink
          Issue Type: Bug
          Components: Connectors / Hive
    Affects Versions: 1.11.3, 1.11.2, 1.12.0
         Environment: Flink 1.11.2 on Yarn.
            Reporter: laystax


HiveTableInputFormat will throws the IllegalArgumentException if partitions of 
a table are in different HDFS nameservices:
{code:java}
Caused by: java.lang.IllegalArgumentException: Wrong FS: 
hdfs://ns1/hive/warehouse/test.db/flink_test/day=4, expected: hdfs://ns2
        at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:662)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.getPathName(DistributedFileSystem.java:222)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.access$000(DistributedFileSystem.java:114)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$20.doCall(DistributedFileSystem.java:1266)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$20.doCall(DistributedFileSystem.java:1262)
        at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1262)
        at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1418)
        at 
org.apache.flink.connectors.hive.read.HiveTableInputFormat.createInputSplits(HiveTableInputFormat.java:299)
        at 
org.apache.flink.connectors.hive.read.HiveTableInputFormat.createInputSplits(HiveTableInputFormat.java:282)
        at 
org.apache.flink.connectors.hive.HiveTableSource.createBatchSource(HiveTableSource.java:214)
        at 
org.apache.flink.connectors.hive.HiveTableSource.getDataStream(HiveTableSource.java:188)
 {code}
The FLINK-16197 introduced this issue.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to