-
Notifications
You must be signed in to change notification settings - Fork 2.5k
Description
In [https://github.com//pull/12376] - we attempted to revert the optimization for single file slice, and do the computation such as getRecordByKeys, etc. over executors even if it is for a single file slice. This means when listing files using metadata files index, even if the data partition has only one file slice, it happens over the executor and the request is sent to the timeline server (RemoteFileSystemView). However, we noticed that the timeline server did not respond and the request timed out in the case of bootstrap of a MOR table having multiple partition fields.
To reproduce locally, follow below steps:
First, revert the single file slice optimization in HoodieBackedTableMetadata. Look at this commit for ref - [https://github.com/codope/hudi/commit/e9f58e007b8428e52f7d3d60e655108376950679]
Now, run the TestBootstrapRead.testBootstrapFunctional. You will notice that COW case passes, MOR with 2 partition fields just hangs in fetching from fs view.
JIRA info
- Link: https://issues.apache.org/jira/browse/HUDI-8621
- Type: Sub-task
- Parent: https://issues.apache.org/jira/browse/HUDI-9096
- Fix version(s):
- 1.1.0
Comments
24/Jan/25 09:54;codope;Removing the optimization for single file slice led to increased CI runtime. Some tests took lot of time fetching from remote fs view ([timeline server read timeout|https://dev.azure.com/apachehudi/a1a51da7-8592-47d4-88dc-fd67bed336bb/_apis/build/builds/3055/logs/58], see stacktrace below). Reverting the change in [https://github.com//pull/12643] and reopening this ticket. We need to investigate why timeline server read timed out.
{code:java}
2025-01-23T22:18:28.6326078Z 334526 [ScalaTest-main-running-TestStreamSourceReadByStateTransitionTime] WARN org.apache.hudi.client.utils.ArchivalUtils [] - Error parsing instant time: 000000002
2025-01-23T22:23:28.8470539Z 634738 [Executor task launch worker for task 1.0 in stage 1335.0 (TID 2085)] ERROR org.apache.hudi.common.table.view.PriorityBasedFileSystemView [] - Got error running preferred function. Trying secondary
2025-01-23T22:23:28.8471638Z org.apache.hudi.exception.HoodieRemoteException: Read timed out
2025-01-23T22:23:28.8472396Z at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.getLatestBaseFilesFromParams(RemoteHoodieTableFileSystemView.java:241) ~[hudi-common-1.1.0-SNAPSHOT.jar:1.1.0-SNAPSHOT]
2025-01-23T22:23:28.8628636Z at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.getLatestBaseFilesBeforeOrOn(RemoteHoodieTableFileSystemView.java:260) ~[hudi-common-1.1.0-SNAPSHOT.jar:1.1.0-SNAPSHOT]
2025-01-23T22:23:28.8629289Z at org.apache.hudi.common.table.view.PriorityBasedFileSystemView.execute(PriorityBasedFileSystemView.java:103) ~[hudi-common-1.1.0-SNAPSHOT.jar:1.1.0-SNAPSHOT]
2025-01-23T22:23:28.8629855Z at org.apache.hudi.common.table.view.PriorityBasedFileSystemView.getLatestBaseFilesBeforeOrOn(PriorityBasedFileSystemView.java:148) ~[hudi-common-1.1.0-SNAPSHOT.jar:1.1.0-SNAPSHOT]
2025-01-23T22:23:28.8630433Z at org.apache.hudi.table.action.commit.UpsertPartitioner.getSmallFiles(UpsertPartitioner.java:306) ~[hudi-spark-client-1.1.0-SNAPSHOT.jar:1.1.0-SNAPSHOT]
2025-01-23T22:23:28.8630990Z at org.apache.hudi.table.action.commit.UpsertPartitioner.lambda$getSmallFilesForPartitions$6e0f90ba$1(UpsertPartitioner.java:288) ~[hudi-spark-client-1.1.0-SNAPSHOT.jar:1.1.0-SNAPSHOT]
2025-01-23T22:23:28.8631680Z at org.apache.hudi.client.common.HoodieSparkEngineContext.lambda$mapToPair$786cea6a$1(HoodieSparkEngineContext.java:176) ~[hudi-spark-client-1.1.0-SNAPSHOT.jar:1.1.0-SNAPSHOT]
2025-01-23T22:23:28.8632166Z at org.apache.spark.api.java.JavaPairRDD$.$anonfun$pairFunToScalaFun$1(JavaPairRDD.scala:1073) ~[spark-core_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8632553Z at scala.collection.Iterator$$anon$10.next(Iterator.scala:461) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8632906Z at scala.collection.Iterator.foreach(Iterator.scala:943) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8633238Z at scala.collection.Iterator.foreach$(Iterator.scala:943) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8635828Z at scala.collection.AbstractIterator.foreach(Iterator.scala:1431) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8636659Z at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8637044Z at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8637435Z at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8637833Z at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8638220Z at scala.collection.TraversableOnce.to(TraversableOnce.scala:366) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8638600Z at scala.collection.TraversableOnce.to$(TraversableOnce.scala:364) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8638967Z at scala.collection.AbstractIterator.to(Iterator.scala:1431) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8639339Z at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:358) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8639730Z at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:358) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8640110Z at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1431) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8641176Z at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:345) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8641576Z at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:339) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8641952Z at scala.collection.AbstractIterator.toArray(Iterator.scala:1431) ~[scala-library-2.12.18.jar:?]
2025-01-23T22:23:28.8642516Z at org.apache.spark.rdd.RDD.$anonfun$collect$2(RDD.scala:1049) ~[spark-core_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8642922Z at org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2433) ~[spark-core_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8643348Z at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93) ~[spark-core_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8643784Z at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:166) ~[spark-core_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8644180Z at org.apache.spark.scheduler.Task.run(Task.scala:141) ~[spark-core_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8644589Z at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:620) ~[spark-core_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8645048Z at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64) [spark-common-utils_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8645519Z at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61) [spark-common-utils_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8645950Z at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:94) [spark-core_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8646851Z at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:623) [spark-core_2.12-3.5.4.jar:3.5.4]
2025-01-23T22:23:28.8647264Z at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_432]
2025-01-23T22:23:28.8647643Z at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_432]
2025-01-23T22:23:28.8647978Z at java.lang.Thread.run(Thread.java:750) [?:1.8.0_432]
2025-01-23T22:23:28.8648257Z Caused by: java.net.SocketTimeoutException: Read timed out
2025-01-23T22:23:28.8648540Z at java.net.SocketInputStream.socketRead0(Native Method) ~[?:1.8.0_432]
2025-01-23T22:23:28.8648864Z at java.net.SocketInputStream.socketRead(SocketInputStream.java:116) ~[?:1.8.0_432]
2025-01-23T22:23:28.8649213Z at java.net.SocketInputStream.read(SocketInputStream.java:171) ~[?:1.8.0_432]
2025-01-23T22:23:28.8649547Z at java.net.SocketInputStream.read(SocketInputStream.java:141) ~[?:1.8.0_432]
2025-01-23T22:23:28.8649940Z at org.apache.http.impl.io.SessionInputBufferImpl.streamRead(SessionInputBufferImpl.java:137) ~[httpcore-4.4.16.jar:4.4.16]
2025-01-23T22:23:28.8650500Z at org.apache.http.impl.io.SessionInputBufferImpl.fillBuffer(SessionInputBufferImpl.java:153) ~[httpcore-4.4.16.jar:4.4.16]
2025-01-23T22:23:28.8650946Z at org.apache.http.impl.io.SessionInputBufferImpl.readLine(SessionInputBufferImpl.java:280) ~[httpcore-4.4.16.jar:4.4.16]
2025-01-23T22:23:28.8651401Z at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:138) ~[httpclient-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8651883Z at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:56) ~[httpclient-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8652334Z at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:259) ~[httpcore-4.4.16.jar:4.4.16]
2025-01-23T22:23:28.8652800Z at org.apache.http.impl.DefaultBHttpClientConnection.receiveResponseHeader(DefaultBHttpClientConnection.java:163) ~[httpcore-4.4.16.jar:4.4.16]
2025-01-23T22:23:28.8653375Z at org.apache.http.impl.conn.CPoolProxy.receiveResponseHeader(CPoolProxy.java:157) ~[httpclient-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8653787Z at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:273) ~[httpcore-4.4.16.jar:4.4.16]
2025-01-23T22:23:28.8654193Z at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:125) ~[httpcore-4.4.16.jar:4.4.16]
2025-01-23T22:23:28.8654591Z at org.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:272) ~[httpclient-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8654980Z at org.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) ~[httpclient-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8655426Z at org.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) ~[httpclient-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8655810Z at org.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) ~[httpclient-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8656207Z at org.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) ~[httpclient-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8656620Z at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) ~[httpclient-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8657026Z at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:56) ~[httpclient-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8657420Z at org.apache.http.client.fluent.Request.internalExecute(Request.java:173) ~[fluent-hc-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8657792Z at org.apache.http.client.fluent.Request.execute(Request.java:177) ~[fluent-hc-4.5.14.jar:4.5.14]
2025-01-23T22:23:28.8658231Z at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.get(RemoteHoodieTableFileSystemView.java:552) ~[hudi-common-1.1.0-SNAPSHOT.jar:1.1.0-SNAPSHOT]
2025-01-23T22:23:28.8658764Z at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.executeRequest(RemoteHoodieTableFileSystemView.java:190) ~[hudi-common-1.1.0-SNAPSHOT.jar:1.1.0-SNAPSHOT]
2025-01-23T22:23:28.8659311Z at org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView.getLatestBaseFilesFromParams(RemoteHoodieTableFileSystemView.java:237) ~[hudi-common-1.1.0-SNAPSHOT.jar:1.1.0-SNAPSHOT]
2025-01-23T22:23:28.8660076Z ... 37 more {code}
;;;