[ 
https://issues.apache.org/jira/browse/HUDI-3636?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17746149#comment-17746149
 ] 

sivabalan narayanan commented on HUDI-3636:
-------------------------------------------

* In Deltastreamer, we re-instantiate WriteClient whenever schema changes. Same 
write client is used by all async table services as well. This poses an issue, 
because the new write client when re-instantiated is intimated to the async 
table service. but if the async table service is in the middle of compaction, 
uses a local copy of write client. and hence may not be able to reach the 
timeline server and will run into connection issues. We are fixing this in this 
patch.
 * Spark streaming sink flow: We start a new write client during first batch 
and close it at the end. But keep re-using the same instance of writeClient for 
subsequent batches. Only core entity that is impacted here was the embedded 
timeline server since we were closing it when write client was closed. So, 
after batch1, if timeline server was enabled, pipeline will fail since timeline 
server is shutdown. 
 *  

> Clustering fails due to marker creation failure
> -----------------------------------------------
>
>                 Key: HUDI-3636
>                 URL: https://issues.apache.org/jira/browse/HUDI-3636
>             Project: Apache Hudi
>          Issue Type: Bug
>          Components: multi-writer
>            Reporter: Ethan Guo
>            Assignee: Ethan Guo
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 0.14.0
>
>
> Scenario: multi-writer test, one writer doing ingesting with Deltastreamer 
> continuous mode, COW, inserts, async clustering and cleaning (partitions 
> under 2022/1, 2022/2), another writer with Spark datasource doing backfills 
> to different partitions (2021/12).  
> 0.10.0 no MT, clustering instant is inflight (failing it in the middle before 
> upgrade) ➝ 0.11 MT, with multi-writer configuration the same as before.
> The clustering/replace instant cannot make progress due to marker creation 
> failure, failing the DS ingestion as well.  Need to investigate if this is 
> timeline-server-based marker related or MT related.
> {code:java}
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 2 in 
> stage 46.0 failed 1 times, most recent failure: Lost task 2.0 in stage 46.0 
> (TID 277) (192.168.70.231 executor driver): java.lang.RuntimeException: 
> org.apache.hudi.exception.HoodieException: 
> org.apache.hudi.exception.HoodieException: 
> java.util.concurrent.ExecutionException: 
> org.apache.hudi.exception.HoodieRemoteException: Failed to create marker file 
> 2022/1/24/aa2f24d3-882f-4d48-b20e-9fcd3540c7a7-0_2-46-277_20220314101326706.parquet.marker.CREATE
> Connect to localhost:26754 [localhost/127.0.0.1, localhost/0:0:0:0:0:0:0:1] 
> failed: Connection refused (Connection refused)
>     at 
> org.apache.hudi.client.utils.LazyIterableIterator.next(LazyIterableIterator.java:121)
>     at 
> scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:46)
>     at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:486)
>     at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:492)
>     at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
>     at scala.collection.Iterator.foreach(Iterator.scala:943)
>     at scala.collection.Iterator.foreach$(Iterator.scala:943)
>     at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
>     at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62)
>     at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53)
>     at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105)
>     at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49)
>     at scala.collection.TraversableOnce.to(TraversableOnce.scala:366)
>     at scala.collection.TraversableOnce.to$(TraversableOnce.scala:364)
>     at scala.collection.AbstractIterator.to(Iterator.scala:1431)
>     at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:358)
>     at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:358)
>     at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1431)
>     at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:345)
>     at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:339)
>     at scala.collection.AbstractIterator.toArray(Iterator.scala:1431)
>     at org.apache.spark.rdd.RDD.$anonfun$collect$2(RDD.scala:1030)
>     at 
> org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2254)
>     at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>     at org.apache.spark.scheduler.Task.run(Task.scala:131)
>     at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>     at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>     at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>     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.HoodieException: 
> org.apache.hudi.exception.HoodieException: 
> java.util.concurrent.ExecutionException: 
> org.apache.hudi.exception.HoodieRemoteException: Failed to create marker file 
> 2022/1/24/aa2f24d3-882f-4d48-b20e-9fcd3540c7a7-0_2-46-277_20220314101326706.parquet.marker.CREATE
> Connect to localhost:26754 [localhost/127.0.0.1, localhost/0:0:0:0:0:0:0:1] 
> failed: Connection refused (Connection refused)
>     at 
> org.apache.hudi.execution.SparkLazyInsertIterable.computeNext(SparkLazyInsertIterable.java:94)
>     at 
> org.apache.hudi.execution.SparkLazyInsertIterable.computeNext(SparkLazyInsertIterable.java:37)
>     at 
> org.apache.hudi.client.utils.LazyIterableIterator.next(LazyIterableIterator.java:119)
>     ... 30 more
> Caused by: org.apache.hudi.exception.HoodieException: 
> java.util.concurrent.ExecutionException: 
> org.apache.hudi.exception.HoodieRemoteException: Failed to create marker file 
> 2022/1/24/aa2f24d3-882f-4d48-b20e-9fcd3540c7a7-0_2-46-277_20220314101326706.parquet.marker.CREATE
> Connect to localhost:26754 [localhost/127.0.0.1, localhost/0:0:0:0:0:0:0:1] 
> failed: Connection refused (Connection refused)
>     at 
> org.apache.hudi.common.util.queue.BoundedInMemoryExecutor.execute(BoundedInMemoryExecutor.java:160)
>     at 
> org.apache.hudi.execution.SparkLazyInsertIterable.computeNext(SparkLazyInsertIterable.java:90)
>     ... 32 more
> Caused by: java.util.concurrent.ExecutionException: 
> org.apache.hudi.exception.HoodieRemoteException: Failed to create marker file 
> 2022/1/24/aa2f24d3-882f-4d48-b20e-9fcd3540c7a7-0_2-46-277_20220314101326706.parquet.marker.CREATE
> Connect to localhost:26754 [localhost/127.0.0.1, localhost/0:0:0:0:0:0:0:1] 
> failed: Connection refused (Connection refused)
>     at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>     at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>     at 
> org.apache.hudi.common.util.queue.BoundedInMemoryExecutor.execute(BoundedInMemoryExecutor.java:154)
>     ... 33 more
> Caused by: org.apache.hudi.exception.HoodieRemoteException: Failed to create 
> marker file 
> 2022/1/24/aa2f24d3-882f-4d48-b20e-9fcd3540c7a7-0_2-46-277_20220314101326706.parquet.marker.CREATE
> Connect to localhost:26754 [localhost/127.0.0.1, localhost/0:0:0:0:0:0:0:1] 
> failed: Connection refused (Connection refused)
>     at 
> org.apache.hudi.table.marker.TimelineServerBasedWriteMarkers.create(TimelineServerBasedWriteMarkers.java:149)
>     at org.apache.hudi.table.marker.WriteMarkers.create(WriteMarkers.java:64)
>     at 
> org.apache.hudi.io.HoodieWriteHandle.createMarkerFile(HoodieWriteHandle.java:181)
>     at 
> org.apache.hudi.io.HoodieCreateHandle.<init>(HoodieCreateHandle.java:99)
>     at 
> org.apache.hudi.io.HoodieCreateHandle.<init>(HoodieCreateHandle.java:73)
>     at 
> org.apache.hudi.io.CreateHandleFactory.create(CreateHandleFactory.java:46)
>     at 
> org.apache.hudi.execution.CopyOnWriteInsertHandler.consumeOneRecord(CopyOnWriteInsertHandler.java:83)
>     at 
> org.apache.hudi.execution.CopyOnWriteInsertHandler.consumeOneRecord(CopyOnWriteInsertHandler.java:40)
>     at 
> org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer.consume(BoundedInMemoryQueueConsumer.java:37)
>     at 
> org.apache.hudi.common.util.queue.BoundedInMemoryExecutor.lambda$null$2(BoundedInMemoryExecutor.java:134)
>     at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>     ... 3 more
> Caused by: org.apache.http.conn.HttpHostConnectException: Connect to 
> localhost:26754 [localhost/127.0.0.1, localhost/0:0:0:0:0:0:0:1] failed: 
> Connection refused (Connection refused)
>     at 
> org.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156)
>     at 
> org.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376)
>     at 
> org.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:393)
>     at 
> org.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236)
>     at 
> org.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186)
>     at org.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89)
>     at 
> org.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110)
>     at 
> org.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185)
>     at 
> org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83)
>     at 
> org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108)
>     at 
> org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:56)
>     at org.apache.http.client.fluent.Request.execute(Request.java:151)
>     at 
> org.apache.hudi.table.marker.TimelineServerBasedWriteMarkers.executeRequestToTimelineServer(TimelineServerBasedWriteMarkers.java:177)
>     at 
> org.apache.hudi.table.marker.TimelineServerBasedWriteMarkers.create(TimelineServerBasedWriteMarkers.java:145)
>     ... 13 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:75)
>     at 
> org.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142)
>     ... 26 more {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to