You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/06/30 08:59:38 UTC

[GitHub] [hudi] eric9204 opened a new issue, #6011: [SUPPORT] HoodieFlinkCompactor failed

eric9204 opened a new issue, #6011:
URL: https://github.com/apache/hudi/issues/6011

   **Describe the problem you faced**
   
   Only the first `compaction.requested` can be compressed successfully. When the second compaction request  was executed , the following error occurred.
   
   **Environment Description**
   
   * Hudi version :  Hudi  master
   
   * Spark version : --
   
   * Hive version : --
   
   * Hadoop version :  Hadoop-3.3.0
   
   * Storage (HDFS/S3/GCS..) : HDFS
   
   * Running on Docker? (yes/no) :  NO
   
   **Additional context**
   
    * Flink-1.14.4
   
   **Stacktrace**
   
   The program finished with the following exception:
   
   org.apache.flink.client.program.ProgramInvocationException: The main method caused an error: org.apache.hudi.exception.HoodieException: Failed to scan metadata
           at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:372)
           at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:222)
           at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114)
           at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:812)
           at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:246)
           at org.apache.flink.client.cli.CliFrontend.parseAndRun(CliFrontend.java:1054)
           at org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1132)
           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:1876)
           at org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
           at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1132)
   Caused by: org.apache.hudi.exception.HoodieException: org.apache.hudi.exception.HoodieException: Failed to scan metadata
           at org.apache.hudi.sink.compact.HoodieFlinkCompactor.start(HoodieFlinkCompactor.java:85)
           at org.apache.hudi.sink.compact.HoodieFlinkCompactor.main(HoodieFlinkCompactor.java:73)
           at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
           at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
           at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
           at java.lang.reflect.Method.invoke(Method.java:498)
           at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:355)
           ... 11 more
   Caused by: java.util.concurrent.ExecutionException: org.apache.hudi.exception.HoodieException: Failed to scan metadata
           at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
           at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
           at org.apache.hudi.async.HoodieAsyncService.waitForShutdown(HoodieAsyncService.java:103)
           at org.apache.hudi.sink.compact.HoodieFlinkCompactor.start(HoodieFlinkCompactor.java:83)
           ... 17 more
   Caused by: org.apache.hudi.exception.HoodieException: Failed to scan metadata
           at org.apache.hudi.sink.compact.HoodieFlinkCompactor$AsyncCompactionService.lambda$startService$0(HoodieFlinkCompactor.java:193)
           at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   Caused by: org.apache.hudi.exception.HoodieIOException: Failed to scan metadata
           at org.apache.hudi.common.table.timeline.HoodieActiveTimeline.<init>(HoodieActiveTimeline.java:124)
           at org.apache.hudi.common.table.timeline.HoodieActiveTimeline.<init>(HoodieActiveTimeline.java:114)
           at org.apache.hudi.common.table.timeline.HoodieActiveTimeline.<init>(HoodieActiveTimeline.java:134)
           at org.apache.hudi.common.table.HoodieTableMetaClient.reloadActiveTimeline(HoodieTableMetaClient.java:348)
           at org.apache.hudi.sink.compact.HoodieFlinkCompactor$AsyncCompactionService.compact(HoodieFlinkCompactor.java:204)
           at org.apache.hudi.sink.compact.HoodieFlinkCompactor$AsyncCompactionService.lambda$startService$0(HoodieFlinkCompactor.java:188)
           ... 4 more
   Caused by: java.io.IOException: Filesystem closed
           at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:476)
           at org.apache.hadoop.hdfs.DFSClient.listPaths(DFSClient.java:1646)
           at org.apache.hadoop.hdfs.DFSClient.listPaths(DFSClient.java:1632)
           at org.apache.hadoop.hdfs.DistributedFileSystem.listStatusInternal(DistributedFileSystem.java:1082)
           at org.apache.hadoop.hdfs.DistributedFileSystem.access$600(DistributedFileSystem.java:140)
           at org.apache.hadoop.hdfs.DistributedFileSystem$24.doCall(DistributedFileSystem.java:1146)
           at org.apache.hadoop.hdfs.DistributedFileSystem$24.doCall(DistributedFileSystem.java:1143)
           at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
           at org.apache.hadoop.hdfs.DistributedFileSystem.listStatus(DistributedFileSystem.java:1153)
           at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1903)
           at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1945)
           at org.apache.hudi.common.fs.HoodieWrapperFileSystem.lambda$listStatus$19(HoodieWrapperFileSystem.java:581)
           at org.apache.hudi.common.fs.HoodieWrapperFileSystem.executeFuncWithTimeMetrics(HoodieWrapperFileSystem.java:101)
           at org.apache.hudi.common.fs.HoodieWrapperFileSystem.listStatus(HoodieWrapperFileSystem.java:580)
           at org.apache.hudi.common.table.HoodieTableMetaClient.scanFiles(HoodieTableMetaClient.java:498)
           at org.apache.hudi.common.table.HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(HoodieTableMetaClient.java:591)
           at org.apache.hudi.common.table.HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(HoodieTableMetaClient.java:574)
           at org.apache.hudi.common.table.timeline.HoodieActiveTimeline.<init>(HoodieActiveTimeline.java:122)
           ... 9 more
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] yuzhaojing commented on issue #6011: [SUPPORT] HoodieFlinkCompactor failed

Posted by GitBox <gi...@apache.org>.
yuzhaojing commented on issue #6011:
URL: https://github.com/apache/hudi/issues/6011#issuecomment-1200145602

   @eric9204 https://github.com/apache/hudi/pull/6253 Try this PR.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] yuzhaojing commented on issue #6011: [SUPPORT] HoodieFlinkCompactor failed

Posted by GitBox <gi...@apache.org>.
yuzhaojing commented on issue #6011:
URL: https://github.com/apache/hudi/issues/6011#issuecomment-1198812966

   @eric9204 Can you show the start command, I can't reproduce this problem locally, is it not a separate process that is started?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] yihua commented on issue #6011: [SUPPORT] HoodieFlinkCompactor failed

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #6011:
URL: https://github.com/apache/hudi/issues/6011#issuecomment-1171778309

   @danny0405 do you have any idea why the compaction fails in Flink?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] eric9204 commented on issue #6011: [SUPPORT] HoodieFlinkCompactor failed

Posted by GitBox <gi...@apache.org>.
eric9204 commented on issue #6011:
URL: https://github.com/apache/hudi/issues/6011#issuecomment-1285000129

   this issue has been solved in version hudi-0.12.0, so close this one out.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] eric9204 closed issue #6011: [SUPPORT] HoodieFlinkCompactor failed

Posted by GitBox <gi...@apache.org>.
eric9204 closed issue #6011: [SUPPORT] HoodieFlinkCompactor failed 
URL: https://github.com/apache/hudi/issues/6011


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] eric9204 commented on issue #6011: [SUPPORT] HoodieFlinkCompactor failed

Posted by GitBox <gi...@apache.org>.
eric9204 commented on issue #6011:
URL: https://github.com/apache/hudi/issues/6011#issuecomment-1198845658

   > @eric9204 Can you show the start command, I can't reproduce this problem locally, is it not a separate process that is started?
   
   @yuzhaojing   `bin/flink run -t yarn-per-job -d -c org.apache.hudi.sink.compact.HoodieFlinkCompactor lib/hudi-flink1.14-bundle_2.12-0.12.0-SNAPSHOT.jar --path /tmp/hudi/random_hudi --compaction-max-memory 4096 --service --min-compaction-interval-seconds 60`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on issue #6011: [SUPPORT] HoodieFlinkCompactor failed

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #6011:
URL: https://github.com/apache/hudi/issues/6011#issuecomment-1175897013

   It expects to be resolved already, cc @yuzhaojing any details here why the exception still exists ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] yihua commented on issue #6011: [SUPPORT] HoodieFlinkCompactor failed

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #6011:
URL: https://github.com/apache/hudi/issues/6011#issuecomment-1175779210

   cc @yuzhaojing 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] eric9204 commented on issue #6011: [SUPPORT] HoodieFlinkCompactor failed

Posted by GitBox <gi...@apache.org>.
eric9204 commented on issue #6011:
URL: https://github.com/apache/hudi/issues/6011#issuecomment-1200966001

   > @eric9204 #6253 Try this PR.
   
   @yuzhaojing The problem has not been solved,the first compaction was successful, the second one failed yet,the error has changed.
   
   2022-08-01 17:07:56,683 INFO  org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView [] - Sending request : (http://10.19.29.170:59932/v1/hoodie/vi     ew/compactions/pending/?basepath=%2Ftmp%2Ftest_houhang%2Fhudi%2Fhudi_table0801c&lastinstantts=20220801170448141&timelinehash=0a64ba8ceeb4ec20ad9743d990d256dcb     fdca1122f61aff27eb96a3939003b87&numinstants=4)
   1334 2022-08-01 17:07:57,215 ERROR org.apache.hudi.common.table.view.PriorityBasedFileSystemView [] - Got error running preferred function. Trying secondary
   1335 org.apache.hudi.exception.HoodieRemoteException: Connect to 10.19.29.170:59932 [/10.19.29.170] failed: Connection refused (Connection refused)
   1336         at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.getPendingCompactionOperations(RemoteHoodieTableFileSystemView.java:434) ~[hudi-f     link1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1337         at org.apache.hudi.common.table.view.PriorityBasedFileSystemView.execute(PriorityBasedFileSystemView.java:68) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.     jar:0.12.0-SNAPSHOT]
   1338         at org.apache.hudi.common.table.view.PriorityBasedFileSystemView.getPendingCompactionOperations(PriorityBasedFileSystemView.java:224) ~[hudi-flink1.14     -bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1339         at org.apache.hudi.table.action.clean.CleanPlanner.<init>(CleanPlanner.java:96) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1340         at org.apache.hudi.table.action.clean.CleanPlanActionExecutor.requestClean(CleanPlanActionExecutor.java:97) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.ja     r:0.12.0-SNAPSHOT]
   1341         at org.apache.hudi.table.action.clean.CleanPlanActionExecutor.requestClean(CleanPlanActionExecutor.java:141) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.j     ar:0.12.0-SNAPSHOT]
   1342         at org.apache.hudi.table.action.clean.CleanPlanActionExecutor.execute(CleanPlanActionExecutor.java:166) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.     12.0-SNAPSHOT]
   1343         at org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.scheduleCleaning(HoodieFlinkCopyOnWriteTable.java:316) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.ja     r:0.12.0-SNAPSHOT]
   1344         at org.apache.hudi.client.BaseHoodieWriteClient.scheduleTableServiceInternal(BaseHoodieWriteClient.java:1355) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.     jar:0.12.0-SNAPSHOT]
   1345         at org.apache.hudi.client.BaseHoodieWriteClient.clean(BaseHoodieWriteClient.java:867) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1346         at org.apache.hudi.client.BaseHoodieWriteClient.clean(BaseHoodieWriteClient.java:840) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1347         at org.apache.hudi.client.BaseHoodieWriteClient.clean(BaseHoodieWriteClient.java:894) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1348         at org.apache.hudi.client.BaseHoodieWriteClient.clean(BaseHoodieWriteClient.java:884) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1349         at org.apache.hudi.sink.compact.CompactionCommitSink.doCommit(CompactionCommitSink.java:168) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSH     OT]
   1350         at org.apache.hudi.sink.compact.CompactionCommitSink.commitIfNecessary(CompactionCommitSink.java:143) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12     .0-SNAPSHOT]
   1351         at org.apache.hudi.sink.compact.CompactionCommitSink.invoke(CompactionCommitSink.java:106) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT     ]  
   1352         at org.apache.hudi.sink.compact.CompactionCommitSink.invoke(CompactionCommitSink.java:57) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1353         at org.apache.flink.streaming.api.operators.StreamSink.processElement(StreamSink.java:54) ~[flink-dist_2.12-1.14.4.jar:1.14.4]
   1354         at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:233) ~[flink-dist_2.12-1.14.     4.jar:1.14.4]
   1355         at org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.processElement(AbstractStreamTaskNetworkInput.java:134) ~[flink-dist_2.12-1.14     .4.jar:1.14.4]
   1356         at org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.emitNext(AbstractStreamTaskNetworkInput.java:105) ~[flink-dist_2.12-1.14.4.jar     :1.14.4]
   1357         at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65) ~[flink-dist_2.12-1.14.4.jar:1.14.4]
   1358         at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:496) ~[flink-dist_2.12-1.14.4.jar:1.14.4]
   1359         at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:203) ~[flink-dist_2.12-1.14.4.jar:1.14.4]
   1360         at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:809) ~[flink-dist_2.12-1.14.4.jar:1.14.4]
   1361         at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:761) ~[flink-dist_2.12-1.14.4.jar:1.14.4]
   1362         at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958) [flink-dist_2.12-1.14.4.jar:1.14.4]
   1363         at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:937) [flink-dist_2.12-1.14.4.jar:1.14.4]
   1364         at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:766) [flink-dist_2.12-1.14.4.jar:1.14.4]
   1365         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575) [flink-dist_2.12-1.14.4.jar:1.14.4]
   1366         at java.lang.Thread.run(Thread.java:748) [?:1.8.0_271]
   1367 Caused by: org.apache.http.conn.HttpHostConnectException: Connect to 10.19.29.170:59932 [/10.19.29.170] failed: Connection refused (Connection refused)
   1368         at org.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:151) ~[hudi-flink1.14-bundle-0.12.0-     SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1369         at org.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:353) ~[hudi-flink1.14-bundle-0.12.0-SN     APSHOT.jar:0.12.0-SNAPSHOT]
   1370         at org.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:380) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1371         at org.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1372         at org.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:184) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1373         at org.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:88) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1374         at org.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1375         at org.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:184) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1376         at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1377         at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:107) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1378         at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1379         at org.apache.http.client.fluent.Request.execute(Request.java:151) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1380         at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.executeRequest(RemoteHoodieTableFileSystemView.java:174) ~[hudi-flink1.14-bundle-     0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1381         at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.getPendingCompactionOperations(RemoteHoodieTableFileSystemView.java:430) ~[hudi-f     link1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1382         ... 30 more
   1383 Caused by: java.net.ConnectException: Connection refused (Connection refused)
   1384         at java.net.PlainSocketImpl.socketConnect(Native Method) ~[?:1.8.0_271]
   1385         at java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:476) ~[?:1.8.0_271]
   1386         at java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:218) ~[?:1.8.0_271]
   1387         at java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:200) ~[?:1.8.0_271]
   1388         at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:394) ~[?:1.8.0_271]
   1389         at java.net.Socket.connect(Socket.java:606) ~[?:1.8.0_271]
   1390         at org.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:74) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHO     T.jar:0.12.0-SNAPSHOT]
   1391         at org.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:134) ~[hudi-flink1.14-bundle-0.12.0-     SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1392         at org.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:353) ~[hudi-flink1.14-bundle-0.12.0-SN     APSHOT.jar:0.12.0-SNAPSHOT]
   1393         at org.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:380) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1394         at org.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1395         at org.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:184) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1396         at org.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:88) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1397         at org.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1398         at org.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:184) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1399         at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1400         at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:107) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1401         at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1402         at org.apache.http.client.fluent.Request.execute(Request.java:151) ~[hudi-flink1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1403         at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.executeRequest(RemoteHoodieTableFileSystemView.java:174) ~[hudi-flink1.14-bundle-     0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1404         at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.getPendingCompactionOperations(RemoteHoodieTableFileSystemView.java:430) ~[hudi-f     link1.14-bundle-0.12.0-SNAPSHOT.jar:0.12.0-SNAPSHOT]
   1405         ... 30 more
   1406 2022-08-01 17:07:57,224 INFO  org.apache.hudi.table.action.clean.CleanPlanner              [] - No earliest commit to retain. No need to scan partitions !!
   1407 2022-08-01 17:07:57,225 INFO  org.apache.hudi.table.action.clean.CleanPlanner              [] - Nothing to clean here. It is already clean
   1408 2022-08-01 17:07:57,233 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20220801170510574__delta     commit__INFLIGHT]}


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org