Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-18659

FileNotFoundException when writing Hive orc tables

    XMLWordPrintableJSON

Details

    Description

      Writing Hive orc tables with Hive 1.1 version, will be:

      Caused by: java.io.FileNotFoundException: File does not exist: hdfs://xxx/warehouse2/tmp_table/.part-6b51dbc2-e169-43a8-93b2-eb8d2be45054-0-0.inprogress.d77fa76c-4760-4cb6-bb5b-97d70afff000	at org.apache.hadoop.hdfs.DistributedFileSystem$19.doCall(DistributedFileSystem.java:1218)	at org.apache.hadoop.hdfs.DistributedFileSystem$19.doCall(DistributedFileSystem.java:1210)	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1210)	at org.apache.flink.connectors.hive.write.HiveBulkWriterFactory$1.getSize(HiveBulkWriterFactory.java:54)	at org.apache.flink.formats.hadoop.bulk.HadoopPathBasedPartFileWriter.getSize(HadoopPathBasedPartFileWriter.java:84)	at org.apache.flink.table.filesystem.FileSystemTableSink$TableRollingPolicy.shouldRollOnEvent(FileSystemTableSink.java:451)	at org.apache.flink.table.filesystem.FileSystemTableSink$TableRollingPolicy.shouldRollOnEvent(FileSystemTableSink.java:421)	at org.apache.flink.streaming.api.functions.sink.filesystem.Bucket.write(Bucket.java:193)	at org.apache.flink.streaming.api.functions.sink.filesystem.Buckets.onElement(Buckets.java:282)	at org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSinkHelper.onElement(StreamingFileSinkHelper.java:104)	at org.apache.flink.table.filesystem.stream.StreamingFileWriter.processElement(StreamingFileWriter.java:118)	at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:717)	at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:692)	at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:672)	at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:52)	at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:30)
      

      This maybe due to lazy init in Orc writer. Until first record comes, orc writer not create this file.

      Attachments

        Issue Links

          Activity

            People

              lirui Rui Li
              lzljs3620320 Jingsong Lee
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: