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/27 18:39:43 UTC

[GitHub] [hudi] rubenssoto opened a new issue, #5984: [SUPPORT] Error on GlobalSortPartitioner using 0.9.0

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

   Hello,
   
   We are trying to process a table and write to Hudi, the data size from source is 274GB but the data sink is only 300mb. We are using EMR on EKS:
   
   30 pods
   32gb each pod
   6000mb overhead memory
   5 Cores
   Hudi version: 0.9.0
   Emr: 6.2.0
   
   `Traceback (most recent call last):
     File "/tmp/spark-a3475e56-2060-433b-aa7a-569995900ac0/main.py", line 73, in <module>
       main(sys.argv[1])
     File "/tmp/spark-a3475e56-2060-433b-aa7a-569995900ac0/main.py", line 48, in main
       transition_table_updater.update()
     File "/tmp/spark-a3475e56-2060-433b-aa7a-569995900ac0/modules.zip/coordination/classes/item_inventory.py", line 179, in update
     File "/tmp/spark-a3475e56-2060-433b-aa7a-569995900ac0/modules.zip/infrastructure/write/write_to_hudi.py", line 57, in write_to_hudi
     File "/tmp/spark-a3475e56-2060-433b-aa7a-569995900ac0/modules.zip/infrastructure/write/write_to_hudi.py", line 90, in create_hudi_table
     File "/usr/lib/spark/python/lib/pyspark.zip/pyspark/sql/readwriter.py", line 827, in save
     File "/usr/lib/spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
     File "/usr/lib/spark/python/lib/pyspark.zip/pyspark/sql/utils.py", line 128, in deco
     File "/usr/lib/spark/python/lib/py4j-0.10.9-src.zip/py4j/protocol.py", line 328, in get_return_value
   py4j.protocol.Py4JJavaError: An error occurred while calling o576.save.
   : org.apache.spark.SparkException: Job aborted due to stage failure: ShuffleMapStage 26 (sortBy at GlobalSortPartitioner.java:41) has failed the maximum allowable number of times: 4. Most recent failure reason: org.apache.spark.shuffle.FetchFailedException: The relative remote executor(Id: 5), which maintains the block data to fetch is dead. 	at org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:748) 	at org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:663) 	at org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:70) 	at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:29) 	at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484) 	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490) 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) 	at org.apache.spark.util.CompletionIterator.has
 Next(CompletionIterator.scala:31) 	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37) 	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage9.sort_addToSorter_0$(Unknown Source) 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage9.processNext(Unknown Source) 	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) 	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729) 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage11.smj_findNextOuterJoinRows_0$(Unknown Source) 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage11.sort_addToSorter_0$(Unknown Source) 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegen
 Stage11.processNext(Unknown Source) 	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) 	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$2.hasNext(WholeStageCodegenExec.scala:748) 	at org.apache.spark.sql.execution.aggregate.SortAggregateExec.$anonfun$doExecute$2(SortAggregateExec.scala:76) 	at org.apache.spark.sql.execution.aggregate.SortAggregateExec.$anonfun$doExecute$2$adapted(SortAggregateExec.scala:73) 	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2(RDD.scala:859) 	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2$adapted(RDD.scala:859) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.
 apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.sql.execution.SQLExecutionRDD.compute(SQLExecutionRDD.scala:55) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org
 .apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59) 	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) 	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52) 	at org.apache.spark.scheduler.Task.run(Task.scala:127) 	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446) 	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377) 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449) 	at java.util.concurren
 t.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.spark.ExecutorDeadException: The relative remote executor(Id: 5), which maintains the block data to fetch is dead. 	at org.apache.spark.network.netty.NettyBlockTransferService$$anon$2.createAndStart(NettyBlockTransferService.scala:132) 	at org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:141) 	at org.apache.spark.network.shuffle.RetryingBlockFetcher.start(RetryingBlockFetcher.java:121) 	at org.apache.spark.network.netty.NettyBlockTransferService.fetchBlocks(NettyBlockTransferService.scala:143) 	at org.apache.spark.storage.ShuffleBlockFetcherIterator.sendRequest(ShuffleBlockFetcherIterator.scala:278) 	at org.apache.spark.storage.ShuffleBlockFetcherIterator.send$1(ShuffleBlockFetcherIterator.scala:721) 	at org.apache.spark.storag
 e.ShuffleBlockFetcherIterator.fetchUpToMaxBytes(ShuffleBlockFetcherIterator.scala:716) 	at org.apache.spark.storage.ShuffleBlockFetcherIterator.initialize(ShuffleBlockFetcherIterator.scala:530) 	at org.apache.spark.storage.ShuffleBlockFetcherIterator.<init>(ShuffleBlockFetcherIterator.scala:171) 	at org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:83) 	at org.apache.spark.sql.execution.ShuffledRowRDD.compute(ShuffledRowRDD.scala:207) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 	at org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 
 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313) 	... 37 more 
   	at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2215)
   	at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2164)
   	at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2163)
   	at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
   	at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
   	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
   	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2163)
   	at org.apache.spark.scheduler.DAGScheduler.handleTaskCompletion(DAGScheduler.scala:1654)
   	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2392)
   	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2344)
   	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2333)
   	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
   	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:815)
   	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2099)
   	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2120)
   	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2139)
   	at org.apache.spark.rdd.RDD.$anonfun$take$1(RDD.scala:1423)
   	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
   	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
   	at org.apache.spark.rdd.RDD.withScope(RDD.scala:388)
   	at org.apache.spark.rdd.RDD.take(RDD.scala:1396)
   	at org.apache.spark.rdd.RDD.$anonfun$isEmpty$1(RDD.scala:1531)
   	at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23)
   	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
   	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
   	at org.apache.spark.rdd.RDD.withScope(RDD.scala:388)
   	at org.apache.spark.rdd.RDD.isEmpty(RDD.scala:1531)
   	at org.apache.hudi.HoodieSparkSqlWriter$.commitAndPerformPostOperations(HoodieSparkSqlWriter.scala:609)
   	at org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:274)
   	at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:164)
   	at org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:46)
   	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
   	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
   	at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:90)
   	at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:180)
   	at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:218)
   	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
   	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:215)
   	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:176)
   	at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:124)
   	at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:123)
   	at org.apache.spark.sql.DataFrameWriter.$anonfun$runCommand$1(DataFrameWriter.scala:963)
   	at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:104)
   	at org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:227)
   	at org.apache.spark.sql.execution.SQLExecution$.executeQuery$1(SQLExecution.scala:107)
   	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:132)
   	at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:104)
   	at org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:227)
   	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:132)
   	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:248)
   	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:131)
   	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
   	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:68)
   	at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:963)
   	at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:415)
   	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:399)
   	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:288)
   	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 py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
   	at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
   	at py4j.Gateway.invoke(Gateway.java:282)
   	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
   	at py4j.commands.CallCommand.execute(CallCommand.java:79)
   	at py4j.GatewayConnection.run(GatewayConnection.java:238)
   	at java.lang.Thread.run(Thread.java:748)
   
   `
   
   
   <img width="1792" alt="Captura de Tela 2022-06-27 às 15 36 01" src="https://user-images.githubusercontent.com/36298331/176012394-eeb8701d-29d3-4967-8440-630e62a7e111.png">
   <img width="1792" alt="Captura de Tela 2022-06-27 às 15 35 29" src="https://user-images.githubusercontent.com/36298331/176012413-20122bf2-c8c1-4eec-95e4-53dc074ee7ab.png">
   


-- 
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] minihippo commented on issue #5984: [SUPPORT] Error on GlobalSortPartitioner using 0.9.0

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

   Could provide the size of shuffle data and the number of reducers? According to these and the executor memory, we could find out a suitable paralism.


-- 
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] nsivabalan commented on issue #5984: [SUPPORT] Error on GlobalSortPartitioner using 0.9.0

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

   @rubenssoto : can you respond to @minihippo 's request above. 


-- 
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 #5984: [SUPPORT] Error on GlobalSortPartitioner using 0.9.0

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

   @rubenssoto Based on your description, it looks like you tried to bulk insert the data from the source.  Likely the shuffle failure is due to out-of-memory on the executors.  What are the memory and parallelism configs (Hudi bulk insert shuffle parallelism and spark parallelism) you use?


-- 
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] nsivabalan commented on issue #5984: [SUPPORT] Error on GlobalSortPartitioner using 0.9.0

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

   @rubenssoto : if you can furnish more details, we can look into investigating 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

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


[GitHub] [hudi] nsivabalan commented on issue #5984: [SUPPORT] Error on GlobalSortPartitioner using 0.9.0

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

   @rubenssoto : hey hi. unless we get more info to reproduce, gonna be tough for us to make further investigation buddy. closing it due to no activity. 
   OOM w/ global sort partitioner is a known limitation. Recommendation is to go w/ NONE sort mode of partitioner sort. 
   thanks.
   


-- 
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 #5984: [SUPPORT] Error on GlobalSortPartitioner using 0.9.0

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

   cc @minihippo 


-- 
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] nsivabalan closed issue #5984: [SUPPORT] Error on GlobalSortPartitioner using 0.9.0

Posted by GitBox <gi...@apache.org>.
nsivabalan closed issue #5984: [SUPPORT] Error on GlobalSortPartitioner using 0.9.0
URL: https://github.com/apache/hudi/issues/5984


-- 
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] nsivabalan commented on issue #5984: [SUPPORT] Error on GlobalSortPartitioner using 0.9.0

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

   @rubenssoto : hey man. we might need more info about write configs used. whether are you using bulk_insert to write to hudi. or is it happening w/ clustering. would appreciate if you can furnish more details. 


-- 
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