非常感谢你的反馈,应该是真的有问题,我建个JIRA追踪下

https://issues.apache.org/jira/browse/FLINK-19166

会包含在即将发布的1.11.2中

Best,
Jingsong

On Wed, Sep 9, 2020 at 10:44 AM MuChen <9329...@qq.com> wrote:

> hi,Rui Li:
> 没有提交分区的目录是commited状态,手动add partition是可以正常查询的
>
> &nbsp;/user/hive/warehouse/rt_dwd.db/dwd_music_copyright_test/dt=2020-08-19/hour=07/part-b7d8f3c6-f1f3-40d4-a269-1ccf2c9a7720-0-1031
>
>
>
>
> ------------------&nbsp;原始邮件&nbsp;------------------
> 发件人:
>                                                   "user-zh"
>                                                                     <
> lirui.fu...@gmail.com&gt;;
> 发送时间:&nbsp;2020年9月8日(星期二) 晚上9:43
> 收件人:&nbsp;"MuChen"<9329...@qq.com&gt;;
> 抄送:&nbsp;"user-zh"<user-zh@flink.apache.org&gt;;"夏帅"<jkill...@dingtalk.com
> &gt;;
> 主题:&nbsp;Re: 回复:使用StreamingFileSink向hive metadata中增加分区部分失败
>
>
>
> 另外也list一下没有提交的分区目录吧,看看里面的文件是什么状态
>
> On Tue, Sep 8, 2020 at 9:19 PM Rui Li <lirui.fu...@gmail.com&gt; wrote:
>
> &gt; 作业有发生failover么?还是说作业能成功结束但是某些partition始终没提交?
> &gt;
> &gt; On Tue, Sep 8, 2020 at 5:20 PM MuChen <9329...@qq.com&gt; wrote:
> &gt;
> &gt;&gt; hi, Rui Li:
> &gt;&gt; 如你所说,的确有类似日志,但是只有成功增加的分区的日志,没有失败分区的日志:
> &gt;&gt; 2020-09-04 17:17:10,548 INFO
> org.apache.flink.streaming.api.operators.
> &gt;&gt; AbstractStreamOperator [] - Partition {dt=2020-08-22, hour=18} of
> table
> &gt;&gt; `hive_catalog`.`rt_dwd`.`dwd_music_copyright_test` is ready to be
> committed
> &gt;&gt; 2020-09-04 17:17:10,716 INFO org.apache.flink.table.filesystem.
> &gt;&gt; MetastoreCommitPolicy [] - Committed partition {dt=2020-08-22,
> hour=18}
> &gt;&gt; to metastore
> &gt;&gt; 2020-09-04 17:17:10,720 INFO org.apache.flink.table.filesystem.
> &gt;&gt; SuccessFileCommitPolicy [] - Committed partition {dt=2020-08-22,
> hour=18}
> &gt;&gt; with success file
> &gt;&gt; 2020-09-04 17:17:19,652 INFO
> org.apache.flink.streaming.api.operators.
> &gt;&gt; AbstractStreamOperator [] - Partition {dt=2020-08-22, hour=19} of
> table
> &gt;&gt; `hive_catalog`.`rt_dwd`.`dwd_music_copyright_test` is ready to be
> committed
> &gt;&gt; 2020-09-04 17:17:19,820 INFO org.apache.flink.table.filesystem.
> &gt;&gt; MetastoreCommitPolicy [] - Committed partition {dt=2020-08-22,
> hour=19}
> &gt;&gt; to metastore
> &gt;&gt; 2020-09-04 17:17:19,824 INFO org.apache.flink.table.filesystem.
> &gt;&gt; SuccessFileCommitPolicy [] - Committed partition {dt=2020-08-22,
> hour=19}
> &gt;&gt; with success file
> &gt;&gt;
> &gt;&gt; 写hdfs的日志是都有的:
> &gt;&gt; 2020-09-04 17:16:04,100 INFO org.apache.hadoop.hive.ql.io
> .parquet.write.
> &gt;&gt; ParquetRecordWriterWrapper [] - creating real writer to write at
> hdfs://
> &gt;&gt;
> Ucluster/user/hive/warehouse/rt_dwd.db/dwd_music_copyright_test/dt=2020-
> &gt;&gt; 08-22/hour=07/.part-b7d8f3c6-f1f3-40d4-a269-1ccf2c9a7720-0-1140
> &gt;&gt; .inprogress.1631ac6c-a07c-4ad7-86ff-cf0d4375d1de
> &gt;&gt; 2020-09-04 17:16:04,126 INFO org.apache.hadoop.hive.ql.io
> .parquet.write.
> &gt;&gt; ParquetRecordWriterWrapper [] - creating real writer to write at
> hdfs://
> &gt;&gt;
> Ucluster/user/hive/warehouse/rt_dwd.db/dwd_music_copyright_test/dt=2020-
> &gt;&gt; 08-22/hour=19/.part-b7d8f3c6-f1f3-40d4-a269-1ccf2c9a7720-0-1142
> &gt;&gt; .inprogress.2700eded-5ed0-4794-8ee9-21721c0c2ffd
> &gt;&gt;
> &gt;&gt; ------------------ 原始邮件 ------------------
> &gt;&gt; *发件人:* "Rui Li" <lirui.fu...@gmail.com&gt;;
> &gt;&gt; *发送时间:* 2020年9月8日(星期二) 中午12:09
> &gt;&gt; *收件人:* "user-zh"<user-zh@flink.apache.org&gt;;"夏帅"<
> jkill...@dingtalk.com&gt;;
> &gt;&gt; *抄送:* "MuChen"<9329...@qq.com&gt;;
> &gt;&gt; *主题:* Re: 回复:使用StreamingFileSink向hive metadata中增加分区部分失败
> &gt;&gt;
> &gt;&gt; streaming file committer在提交分区之前会打印这样的日志:
> &gt;&gt;
> &gt;&gt; LOG.info("Partition {} of table {} is ready to be committed",
> partSpec, tableIdentifier);
> &gt;&gt;
> &gt;&gt; partition commit policy会在成功提交分区以后打印这样的日志:
> &gt;&gt;
> &gt;&gt; LOG.info("Committed partition {} to metastore", partitionSpec);
> &gt;&gt;
> &gt;&gt; LOG.info("Committed partition {} with success file",
> context.partitionSpec());
> &gt;&gt;
> &gt;&gt; 可以检查一下这样的日志,看是不是卡在什么地方了
> &gt;&gt;
> &gt;&gt; On Tue, Sep 8, 2020 at 11:02 AM 夏帅 <jkill...@dingtalk.com.invalid&gt;
> wrote:
> &gt;&gt;
> &gt;&gt;&gt; 就第二次提供的日志看,好像是你的namenode出现的问题
> &gt;&gt;&gt;
> &gt;&gt;&gt;
> &gt;&gt;&gt;
> ------------------------------------------------------------------
> &gt;&gt;&gt; 发件人:MuChen <9329...@qq.com&gt;
> &gt;&gt;&gt; 发送时间:2020年9月8日(星期二) 10:56
> &gt;&gt;&gt; 收件人:user-zh@flink.apache.org 夏帅 <jkill...@dingtalk.com&gt;;
> user-zh <
> &gt;&gt;&gt; user-zh@flink.apache.org&gt;
> &gt;&gt;&gt; 主 题:回复: 回复:使用StreamingFileSink向hive metadata中增加分区部分失败
> &gt;&gt;&gt;
> &gt;&gt;&gt; 在checkpoint失败的时间,tm上还有一些info和warn级别的日志:
> &gt;&gt;&gt; 2020-09-04 17:17:59,520 INFO
> &gt;&gt;&gt; org.apache.hadoop.io.retry.RetryInvocationHandler [] -
> Exception while
> &gt;&gt;&gt; invoking create of class ClientNamenodeProtocolTranslatorPB
> over
> &gt;&gt;&gt; uhadoop-op3raf-master2/10.42.52.202:8020 after 14 fail over
> attempts.
> &gt;&gt;&gt; Trying to fail over immediately.
> &gt;&gt;&gt; java.io.IOException: java.lang.InterruptedException
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.ipc.Client.call(Client.java:1449)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.ipc.Client.call(Client.java:1401)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> com.sun.proxy.$Proxy26.create(Unknown Source) ~[?:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:295)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
> &gt;&gt;&gt; ~[?:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> &gt;&gt;&gt; ~[?:1.8.0_144]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_144]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> com.sun.proxy.$Proxy27.create(Unknown Source) ~[?:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:1721)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1657)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1582)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:397)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:393)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:393)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:337)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:908)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:889)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:786)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.hive.shaded.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:141)
> &gt;&gt;&gt; ~[flink-sql-connector-hive-1.2.2_2.11-1.11.0.jar:1.11.0]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.hive.shaded.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:37)
> &gt;&gt;&gt; ~[flink-sql-connector-hive-1.2.2_2.11-1.11.0.jar:1.11.0]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.table.filesystem.SuccessFileCommitPolicy.commit(SuccessFileCommitPolicy.java:45)
> &gt;&gt;&gt; ~[flink-table-blink_2.11-1.11.0.jar:1.11.0]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.table.filesystem.stream.StreamingFileCommitter.commitPartitions(StreamingFileCommitter.java:167)
> &gt;&gt;&gt; ~[flink-table-blink_2.11-1.11.0.jar:1.11.0]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.table.filesystem.stream.StreamingFileCommitter.processElement(StreamingFileCommitter.java:144)
> &gt;&gt;&gt; ~[flink-table-blink_2.11-1.11.0.jar:1.11.0]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:161)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at org.apache.flink.streaming.
> runtime.io
> .StreamTaskNetworkInput.processElement(StreamTaskNetworkInput.java:178)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at org.apache.flink.streaming.
> runtime.io
> .StreamTaskNetworkInput.emitNext(StreamTaskNetworkInput.java:153)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at org.apache.flink.streaming.
> runtime.io
> .StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:67)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:345)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxStep(MailboxProcessor.java:191)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:181)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:558)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:530)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.flink.runtime.taskmanager.Task.run(Task.java:546)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> java.lang.Thread.run(Thread.java:748) [?:1.8.0_144]
> &gt;&gt;&gt; Caused by: java.lang.InterruptedException
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> java.util.concurrent.FutureTask.awaitDone(FutureTask.java:404)
> &gt;&gt;&gt; ~[?:1.8.0_144]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> java.util.concurrent.FutureTask.get(FutureTask.java:191)
> &gt;&gt;&gt; ~[?:1.8.0_144]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.ipc.Client$Connection.sendRpcRequest(Client.java:1048)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.ipc.Client.call(Client.java:1443)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; ... 38 more
> &gt;&gt;&gt; 2020-09-04 17:17:59,522 WARN
> &gt;&gt;&gt; org.apache.hadoop.io.retry.RetryInvocationHandler [] -
> Exception while
> &gt;&gt;&gt; invoking class
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create
> &gt;&gt;&gt; over uhadoop-op3raf-master1/10.42.31.63:8020. Not retrying
> because
> &gt;&gt;&gt; failovers (15) exceeded maximum allowed (15)
> &gt;&gt;&gt; java.io.IOException: Failed on local exception:
> &gt;&gt;&gt; java.nio.channels.ClosedByInterruptException; Host Details :
> local host is:
> &gt;&gt;&gt; "uhadoop-op3raf-core13/10.42.99.178"; destination host is:
> &gt;&gt;&gt; "uhadoop-op3raf-master1":8020;
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at org.apache.hadoop.net
> .NetUtils.wrapException(NetUtils.java:772)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.ipc.Client.call(Client.java:1474)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.ipc.Client.call(Client.java:1401)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> com.sun.proxy.$Proxy26.create(Unknown Source) ~[?:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:295)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
> &gt;&gt;&gt; ~[?:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> &gt;&gt;&gt; ~[?:1.8.0_144]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_144]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> com.sun.proxy.$Proxy27.create(Unknown Source) ~[?:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:1721)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1657)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1582)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:397)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:393)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:393)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:337)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:908)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:889)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.fs.FileSystem.create(FileSystem.java:786)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.hive.shaded.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:141)
> &gt;&gt;&gt; ~[flink-sql-connector-hive-1.2.2_2.11-1.11.0.jar:1.11.0]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.hive.shaded.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:37)
> &gt;&gt;&gt; ~[flink-sql-connector-hive-1.2.2_2.11-1.11.0.jar:1.11.0]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.table.filesystem.SuccessFileCommitPolicy.commit(SuccessFileCommitPolicy.java:45)
> &gt;&gt;&gt; ~[flink-table-blink_2.11-1.11.0.jar:1.11.0]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.table.filesystem.stream.StreamingFileCommitter.commitPartitions(StreamingFileCommitter.java:167)
> &gt;&gt;&gt; ~[flink-table-blink_2.11-1.11.0.jar:1.11.0]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.table.filesystem.stream.StreamingFileCommitter.processElement(StreamingFileCommitter.java:144)
> &gt;&gt;&gt; ~[flink-table-blink_2.11-1.11.0.jar:1.11.0]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:161)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at org.apache.flink.streaming.
> runtime.io
> .StreamTaskNetworkInput.processElement(StreamTaskNetworkInput.java:178)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at org.apache.flink.streaming.
> runtime.io
> .StreamTaskNetworkInput.emitNext(StreamTaskNetworkInput.java:153)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at org.apache.flink.streaming.
> runtime.io
> .StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:67)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:345)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxStep(MailboxProcessor.java:191)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:181)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:558)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:530)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.flink.runtime.taskmanager.Task.run(Task.java:546)
> &gt;&gt;&gt; [music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> java.lang.Thread.run(Thread.java:748) [?:1.8.0_144]
> &gt;&gt;&gt; Caused by: java.nio.channels.ClosedByInterruptException
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202)
> &gt;&gt;&gt; ~[?:1.8.0_144]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:659)
> &gt;&gt;&gt; ~[?:1.8.0_144]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at org.apache.hadoop.net
> .SocketIOWithTimeout.connect(SocketIOWithTimeout.java:192)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at org.apache.hadoop.net
> .NetUtils.connect(NetUtils.java:530)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at org.apache.hadoop.net
> .NetUtils.connect(NetUtils.java:494)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:609)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:707)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> &gt;&gt;&gt;
> org.apache.hadoop.ipc.Client$Connection.access$2800(Client.java:370)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.ipc.Client.getConnection(Client.java:1523)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; at
> org.apache.hadoop.ipc.Client.call(Client.java:1440)
> &gt;&gt;&gt; ~[music_copyright-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
> &gt;&gt;&gt;&nbsp;&nbsp;&nbsp;&nbsp; ... 38 more
> &gt;&gt;&gt;
> &gt;&gt;&gt; 补充:程序多次执行,均会出现部分分区创建失败的情况,而且每次失败的分区是不同的
> &gt;&gt;&gt;
> &gt;&gt;&gt;
> &gt;&gt;&gt; ------------------ 原始邮件 ------------------
> &gt;&gt;&gt; 发件人: "user-zh@flink.apache.org 夏帅"
> <jkill...@dingtalk.com.INVALID&gt;;
> &gt;&gt;&gt; 发送时间: 2020年9月8日(星期二) 上午10:47
> &gt;&gt;&gt; 收件人: "user-zh"<user-zh@flink.apache.org&gt;;"MuChen"<
> 9329...@qq.com&gt;;
> &gt;&gt;&gt; 主题:&nbsp; 回复:使用StreamingFileSink向hive metadata中增加分区部分失败
> &gt;&gt;&gt;
> &gt;&gt;&gt; 异常日志只有这些么?有没有详细点的
> &gt;&gt;
> &gt;&gt;
> &gt;&gt;
> &gt;&gt; --
> &gt;&gt; Best regards!
> &gt;&gt; Rui Li
> &gt;&gt;
> &gt;
> &gt;
> &gt; --
> &gt; Best regards!
> &gt; Rui Li
> &gt;
>
>
> --
> Best regards!
> Rui Li



-- 
Best, Jingsong Lee

回复