Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
None
-
None
-
None
Description
For now, Flink job is forced to use embed timeline service, and will create new .hoodie/.aux/view_storage_conf.properties when create write client
so if an offline compaction job starts, it will overwrite the writing job's config file. it will work fine if no TMs failover. but if a TM failed, and JM is still alive, it will read the wrong config file written by offline compaction job, if offline job already done, then following error happens
org.apache.hudi.exception.HoodieRemoteException: Connect to 10.131.87.40:38605 [/10.131.87.40] failed: Connection refused (Connection refused) at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.refresh(RemoteHoodieTableFileSystemView.java:420) at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.sync(RemoteHoodieTableFileSystemView.java:484) at org.apache.hudi.common.table.view.PriorityBasedFileSystemView.sync(PriorityBasedFileSystemView.java:257) at org.apache.hudi.sink.partitioner.profile.WriteProfile.reload(WriteProfile.java:252) at org.apache.hudi.sink.partitioner.BucketAssigner.reload(BucketAssigner.java:211) at org.apache.hudi.sink.partitioner.BucketAssignFunction.notifyCheckpointComplete(BucketAssignFunction.java:234) at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.notifyCheckpointComplete(AbstractUdfStreamOperator.java:130) at org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.notifyCheckpointComplete(StreamOperatorWrapper.java:99) at org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.notifyCheckpointComplete(SubtaskCheckpointCoordinatorImpl.java:386) at org.apache.flink.streaming.runtime.tasks.StreamTask.notifyCheckpointComplete(StreamTask.java:1177) at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$notifyCheckpointCompleteAsync$10(StreamTask.java:1142) at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$notifyCheckpointOperation$12(StreamTask.java:1165) at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50) at org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90) at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMailsNonBlocking(MailboxProcessor.java:359) at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:323) at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:202) at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:689) at org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:644) at org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:655) at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:628) at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:782) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566) at java.lang.Thread.run(Thread.java:748) Caused by: org.apache.http.conn.HttpHostConnectException: Connect to 10.131.87.40:38605 [/10.131.87.40] failed: Connection refused (Connection refused) at org.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:151) at org.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:353) at org.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:380) at org.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) at org.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:184) at org.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:88) at org.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) at org.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:184) at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82) at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:107) at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55) at org.apache.http.client.fluent.Request.execute(Request.java:151) at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.executeRequest(RemoteHoodieTableFileSystemView.java:176) at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.refresh(RemoteHoodieTableFileSystemView.java:418) ... 23 more Caused by: java.net.ConnectException: Connection refused (Connection refused) at java.net.PlainSocketImpl.socketConnect(Native Method) at java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:350) at java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:206) at java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:188) at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) at java.net.Socket.connect(Socket.java:607) at org.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:74) at org.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:134) ... 36 more