Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

iceberg flink sink job can't restart due to metadata location not found #1688

Closed
wg1026688210 opened this issue Oct 29, 2020 · 3 comments
Closed
Labels

Comments

@wg1026688210
Copy link
Contributor

wg1026688210 commented Oct 29, 2020

flink sink job fail

we found job fail due to hive metastore server timeout
java.lang.RuntimeException: operation failed for flink_origin_db.iceberg_test_1504_10,localtion viewfs://AutoLfCluster/team/db/hive_db/flink_origin_db/iceberg_test_1504_10/metadata/00077-ca38eacc-c44f-4ea1-b702-de53de28154b.metadata.json=>viewfs://AutoLfCluster/team/db/hive_db/flink_origin_db/iceberg_test_1504_10/metadata/00078-399ca6c3-badf-4f84-9a9a-b58d5f5b02ef.metadata.json
	at org.apache.iceberg.hive.HiveTableOperations.doCommit(HiveTableOperations.java:198) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.BaseMetastoreTableOperations.commit(BaseMetastoreTableOperations.java:118) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.SnapshotProducer.lambda$commit$2(SnapshotProducer.java:293) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:404) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:213) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:197) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:189) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.SnapshotProducer.commit(SnapshotProducer.java:275) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.flink.sink.IcebergFilesCommitter.commitOperation(IcebergFilesCommitter.java:231) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.flink.sink.IcebergFilesCommitter.append(IcebergFilesCommitter.java:220) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.flink.sink.IcebergFilesCommitter.commitUpToCheckpoint(IcebergFilesCommitter.java:192) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.flink.sink.IcebergFilesCommitter.notifyCheckpointComplete(IcebergFilesCommitter.java:173) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.notifyCheckpointComplete(StreamOperatorWrapper.java:107) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.notifyCheckpointComplete(SubtaskCheckpointCoordinatorImpl.java:283) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.notifyCheckpointComplete(StreamTask.java:987) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$notifyCheckpointCompleteAsync$10(StreamTask.java:958) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$notifyCheckpointOperation$12(StreamTask.java:974) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:78) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:282) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxStep(MailboxProcessor.java:190) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:181) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:566) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:536) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at java.lang.Thread.run(Thread.java:745) ~[?:1.8.0_92]
Caused by: java.lang.RuntimeException: org.apache.thrift.TApplicationException: Internal error processing alter_table_with_environment_context
	at org.apache.iceberg.relocated.com.google.common.base.Throwables.propagate(Throwables.java:241) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.common.DynMethods$UnboundMethod.invoke(DynMethods.java:80) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.hive.HiveTableOperations.lambda$persistTable$1(HiveTableOperations.java:226) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.hive.ClientPool.run(ClientPool.java:54) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.hive.HiveTableOperations.persistTable(HiveTableOperations.java:222) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.hive.HiveTableOperations.doCommit(HiveTableOperations.java:176) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	... 26 more
Caused by: org.apache.thrift.TApplicationException: Internal error processing alter_table_with_environment_context
	at org.apache.thrift.TApplicationException.read(TApplicationException.java:111) ~[hive-exec-2.0.0.jar:2.0.0]
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:79) ~[hive-exec-2.0.0.jar:2.0.0]
	at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_alter_table_with_environment_context(ThriftHiveMetastore.java:1409) ~[hive-exec-2.0.0.jar:2.0.0]
	at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.alter_table_with_environment_context(ThriftHiveMetastore.java:1393) ~[hive-exec-2.0.0.jar:2.0.0]
	at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.alter_table(HiveMetaStoreClient.java:377) ~[hive-exec-2.0.0.jar:2.0.0]
	at sun.reflect.GeneratedMethodAccessor46.invoke(Unknown Source) ~[?:?]
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_92]
	at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_92]
	at org.apache.iceberg.common.DynMethods$UnboundMethod.invokeChecked(DynMethods.java:65) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.common.DynMethods$UnboundMethod.invoke(DynMethods.java:77) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.hive.HiveTableOperations.lambda$persistTable$1(HiveTableOperations.java:226) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.hive.ClientPool.run(ClientPool.java:54) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.hive.HiveTableOperations.persistTable(HiveTableOperations.java:222) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.hive.HiveTableOperations.doCommit(HiveTableOperations.java:176) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	... 26 more

error log in hive

Caused by: org.apache.thrift.transport.TTransportException: java.net.SocketTimeoutException: Read timed out

        at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:129)

        at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)

        at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:376)

        at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:453)

        at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:435)

        at org.apache.thrift.transport.TSaslClientTransport.read(TSaslClientTransport.java:37)

        at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)

        at org.apache.hadoop.hive.thrift.TFilterTransport.readAll(TFilterTransport.java:62)

        at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429)

        at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318)

        at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219)

        at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:77)

        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_alter_table_with_environment_context(ThriftHiveMetastore.java:1409)

        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.alter_table_with_environment_context(ThriftHiveMetastore.java:1393)

        at com.autohome.server.HMSFederationHandler.alter_table_with_environment_context(HMSFederationHa
ndler.java:255)

        ... 26 common frames omitted

Caused by: java.net.SocketTimeoutException: Read timed out

        at java.net.SocketInputStream.socketRead0(Native Method)

        at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)

        at java.net.SocketInputStream.read(SocketInputStream.java:170)

        at java.net.SocketInputStream.read(SocketInputStream.java:141)

        ... 40 common frames omitted

        at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:129)

        at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)

        at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:376)

        at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:453)

        at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:435)

        at org.apache.thrift.transport.TSaslClientTransport.read(TSaslClientTransport.java:37)

        at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)

        at org.apache.hadoop.hive.thrift.TFilterTransport.readAll(TFilterTransport.java:62)

        at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429)

        at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318)

        at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219)

        at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:77)

        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_alter_table_with_environment_context(ThriftHiveMetastore.java:1409)

        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.alter_table_with_environment_context(ThriftHiveMetastore.java:1393)

        at com.autohome.server.HMSFederationHandler.alter_table_with_environment_context(HMSFederationHandler.java:255)

        at sun.reflect.GeneratedMethodAccessor158.invoke(Unknown Source)

        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)

        at java.lang.reflect.Method.invoke(Method.java:498)

        at com.autohome.server.HMSFederationHandlerProxy.invoke(HMSFederationHandlerProxy.java:36)

        at com.sun.proxy.$Proxy90.alter_table_with_environment_context(Unknown Source)

        at sun.reflect.GeneratedMethodAccessor158.invoke(Unknown Source)

        at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invokeInternal(RetryingHMSHandler.java:140)

        at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:99)

        at com.sun.proxy.$Proxy90.alter_table_with_environment_context(Unknown Source)

        at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)

        at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)

        at org.apache.hadoop.hive.metastore.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:48)

        at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge.java:703)

        at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge.java:698)

        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:1657)

        at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:698)

        at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286)

        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.net.SocketTimeoutException: Read timed out

        at java.net.SocketInputStream.socketRead0(Native Method)

        at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)

        at java.net.SocketInputStream.read(SocketInputStream.java:170)

        at java.net.SocketInputStream.read(SocketInputStream.java:141)

        at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)



        at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:129)

        at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)

        at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:376)

        at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:453)

        at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:435)

        at org.apache.thrift.transport.TSaslClientTransport.read(TSaslClientTransport.java:37)

        at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)

        at org.apache.hadoop.hive.thrift.TFilterTransport.readAll(TFilterTransport.java:62)

        at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429)

        at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318)

        at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219)

        at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:77)

        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_alter_table_with_environment_context(ThriftHiveMetastore.java:1409)

        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.alter_table_with_environment_context(ThriftHiveMetastore.java:1393)

        at com.autohome.server.HMSFederationHandler.alter_table_with_environment_context(HMSFederationHandler.java:255)

        at sun.reflect.GeneratedMethodAccessor158.invoke(Unknown Source)

        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)

        at java.lang.reflect.Method.invoke(Method.java:498)

        at com.autohome.server.HMSFederationHandlerProxy.invoke(HMSFederationHandlerProxy.java:36)

        at com.sun.proxy.$Proxy90.alter_table_with_environment_context(Unknown Source)

        at sun.reflect.GeneratedMethodAccessor158.invoke(Unknown Source)

        at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invokeInternal(RetryingHMSHandler.java:140)

        at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:99)

        at com.sun.proxy.$Proxy90.alter_table_with_environment_context(Unknown Source)

        at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)

        at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)

        at org.apache.hadoop.hive.metastore.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:48)

        at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge.java:703)

        at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge.java:698)

        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:1657)

        at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:698)

        at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286)

        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.net.SocketTimeoutException: Read timed out

        at java.net.SocketInputStream.socketRead0(Native Method)

        at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)

        at java.net.SocketInputStream.read(SocketInputStream.java:170)

        at java.net.SocketInputStream.read(SocketInputStream.java:141)

        at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)

        at java.io.BufferedInputStream.read1(BufferedInputStream.java:286)


job can not restart from checkpoint

and then job can't restart from checkpoint automatically due to metadata location not found when hive catalog loads iceberg table

2020-10-28 14:05:36,665 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - IcebergFilesCommitter -> Sink: IcebergSink iceberg_catalog.flink_origin_db.iceberg_test_1504_10 (1/1) (f78a61f9b308af32c673641b23fe9c0f) switched from RUNNING to FAILED on org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot@16add4e5.
org.apache.iceberg.exceptions.NotFoundException: Failed to open input stream for file: viewfs://AutoLfCluster/team/db/hive_db/flink_origin_db/iceberg_test_1504_10/metadata/00078-399ca6c3-badf-4f84-9a9a-b58d5f5b02ef.metadata.json
	at org.apache.iceberg.hadoop.HadoopInputFile.newStream(HadoopInputFile.java:159) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:238) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:233) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.BaseMetastoreTableOperations.lambda$refreshFromMetadataLocation$0(BaseMetastoreTableOperations.java:167) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:404) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:213) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:197) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:189) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:166) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:148) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.hive.HiveTableOperations.doRefresh(HiveTableOperations.java:138) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.BaseMetastoreTableOperations.refresh(BaseMetastoreTableOperations.java:86) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.BaseMetastoreTableOperations.current(BaseMetastoreTableOperations.java:69) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.BaseMetastoreCatalog.loadTable(BaseMetastoreCatalog.java:102) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.flink.TableLoader$CatalogTableLoader.loadTable(TableLoader.java:113) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.iceberg.flink.sink.IcebergFilesCommitter.initializeState(IcebergFilesCommitter.java:115) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	at org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.initializeOperatorState(StreamOperatorStateHandler.java:106) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:258) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.OperatorChain.initializeStateAndOpenOperators(OperatorChain.java:290) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:479) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:475) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:528) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546) ~[flink-dist_2.11-1.11.2-auto1.1-SNAPSHOT.jar:1.11.2-auto1.1-SNAPSHOT]
	at java.lang.Thread.run(Thread.java:745) ~[?:1.8.0_92]
Caused by: java.io.FileNotFoundException: File does not exist: /team/db/hive_db/flink_origin_db/iceberg_test_1504_10/metadata/00078-399ca6c3-badf-4f84-9a9a-b58d5f5b02ef.metadata.json
	at org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:71)
	at org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:587)
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2049)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2045)
	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:1657)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2045)

	at sun.reflect.GeneratedConstructorAccessor47.newInstance(Unknown Source) ~[?:?]
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) ~[?:1.8.0_92]
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423) ~[?:1.8.0_92]
	at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1228) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1213) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1201) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength(DFSInputStream.java:306) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:272) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSInputStream.<init>(DFSInputStream.java:264) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:1526) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:304) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:299) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:312) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.fs.FilterFileSystem.open(FilterFileSystem.java:161) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.fs.viewfs.ChRootedFileSystem.open(ChRootedFileSystem.java:261) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.fs.viewfs.ViewFileSystem.open(ViewFileSystem.java:459) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:767) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.iceberg.hadoop.HadoopInputFile.newStream(HadoopInputFile.java:157) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	... 25 more
Caused by: org.apache.hadoop.ipc.RemoteException: File does not exist: /team/db/hive_db/flink_origin_db/iceberg_test_1504_10/metadata/00078-399ca6c3-badf-4f84-9a9a-b58d5f5b02ef.metadata.json
	at org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:71)
	at org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:587)
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:365)
	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2049)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2045)
	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:1657)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2045)

	at org.apache.hadoop.ipc.Client.call(Client.java:1475) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.ipc.Client.call(Client.java:1412) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229) ~[hadoop-common-2.7.2.jar:?]
	at com.sun.proxy.$Proxy26.getBlockLocations(Unknown Source) ~[?:?]
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(ClientNamenodeProtocolTranslatorPB.java:255) ~[hadoop-hdfs-2.7.2.jar:?]
	at sun.reflect.GeneratedMethodAccessor28.invoke(Unknown Source) ~[?:?]
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_92]
	at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_92]
	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:191) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102) ~[hadoop-common-2.7.2.jar:?]
	at com.sun.proxy.$Proxy27.getBlockLocations(Unknown Source) ~[?:?]
	at org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1226) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1213) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1201) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength(DFSInputStream.java:306) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:272) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSInputStream.<init>(DFSInputStream.java:264) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:1526) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:304) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:299) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:312) ~[hadoop-hdfs-2.7.2.jar:?]
	at org.apache.hadoop.fs.FilterFileSystem.open(FilterFileSystem.java:161) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.fs.viewfs.ChRootedFileSystem.open(ChRootedFileSystem.java:261) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.fs.viewfs.ViewFileSystem.open(ViewFileSystem.java:459) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:767) ~[hadoop-common-2.7.2.jar:?]
	at org.apache.iceberg.hadoop.HadoopInputFile.newStream(HadoopInputFile.java:157) ~[iceberg-flink-runtime-3d308d3.dirty.jar-1603859992059.jar:?]
	... 25 more

We found metadatafile will be deleted when alter hms table operation throwed exception (timeout) . but still alter operation still success when hive metastore api occur Internal error processing alter_table_with_environment_context.

ALTER_TABLE.invoke(client, database, tableName, hmsTable, envContext);

if (threw) {
// if anything went wrong, clean up the uncommitted metadata file
io().deleteFile(newMetadataLocation);
}

So we add a improvment on loading table in iceberg HiveTableOperations to do some fault-tolerants .
If metadataLocation is not found,

  1. Try to use previousMetadataLocation.
  2. Set the previousMetadataLocation path value to metadataLocation property of hive table , in case both location is unavaliable when job restart from checkpoint.

This work when the error occuring again .

@rdblue
Copy link
Contributor

rdblue commented Nov 1, 2020

I think a fix for this needs to happen during commit, not during read. Otherwise there is a correctness error that can only be resolved by a human. Please see my comment on #1687 for more detail.

@KarlManong
Copy link
Contributor

I think #3717 can fix this

Copy link

This issue has been automatically marked as stale because it has been open for 180 days with no activity. It will be closed in next 14 days if no further activity occurs. To permanently prevent this issue from being considered stale, add the label 'not-stale', but commenting on the issue is preferred when possible.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants