[ 
https://issues.apache.org/jira/browse/HUDI-690?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

leesf resolved HUDI-690.
------------------------
    Resolution: Fixed

Fixed via master: 6c450957ced051de6231ad047bce22752210b786

> filtercompletedInstants in HudiSnapshotCopier not working as expected for MOR 
> tables
> ------------------------------------------------------------------------------------
>
>                 Key: HUDI-690
>                 URL: https://issues.apache.org/jira/browse/HUDI-690
>             Project: Apache Hudi
>          Issue Type: Improvement
>            Reporter: Jasmine Omeke
>            Assignee: Raymond Xu
>            Priority: Major
>              Labels: bug-bash-0.6.0, pull-request-available
>             Fix For: 0.6.0
>
>
> Hi. I encountered an error while using the HudiSnapshotCopier class to make a 
> Backup of merge on read tables: 
> [https://github.com/apache/incubator-hudi/blob/release-0.5.0/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java]
>  
> The error:
>  
> {code:java}
> 20/03/09 15:43:19 INFO AmazonHttpClient: Configuring Proxy. Proxy Host: 
> web-proxy.bt.local Proxy Port: 3128
> 20/03/09 15:43:19 INFO HoodieTableConfig: Loading dataset properties from 
> <s3Path>/.hoodie/hoodie.properties
> 20/03/09 15:43:19 INFO AmazonHttpClient: Configuring Proxy. Proxy Host: 
> web-proxy.bt.local Proxy Port: 3128
> 20/03/09 15:43:19 INFO HoodieTableMetaClient: Finished Loading Table of type 
> MERGE_ON_READ from <s3Path>
> 20/03/09 15:43:20 INFO HoodieActiveTimeline: Loaded instants 
> java.util.stream.ReferencePipeline$Head@77f7352a
> 20/03/09 15:43:21 INFO YarnSchedulerBackend$YarnDriverEndpoint: Registered 
> executor NettyRpcEndpointRef(spark-client://Executor) (10.49.26.74:40894) 
> with ID 2
> 20/03/09 15:43:21 INFO ExecutorAllocationManager: New executor 2 has 
> registered (new total is 1)
> 20/03/09 15:43:21 INFO BlockManagerMasterEndpoint: Registering block manager 
> ip-10-49-26-74.us-east-2.compute.internal:32831 with 12.4 GB RAM, 
> BlockManagerId(2, ip-10-49-26-74.us-east-2.compute.internal, 3283
> 1, None)
> 20/03/09 15:43:21 INFO YarnSchedulerBackend$YarnDriverEndpoint: Registered 
> executor NettyRpcEndpointRef(spark-client://Executor) (10.49.26.74:40902) 
> with ID 4
> 20/03/09 15:43:21 INFO ExecutorAllocationManager: New executor 4 has 
> registered (new total is 2)Exception in thread "main" 
> java.lang.IllegalStateException: Hudi File Id 
> (HoodieFileGroupId{partitionPath='created_at_month=2020-03', 
> fileId='7104bb0b-20f6-4dec-981b-c11bf20ade4a-0'}) has more than 1 pending
> compactions. Instants: (20200309011643,{"baseInstantTime": "20200308213934", 
> "deltaFilePaths": 
> [".7104bb0b-20f6-4dec-981b-c11bf20ade4a-0_20200308213934.log.1_3-751289-170568496",
>  ".7104bb0b-20f6-4dec-981b-c11
> bf20ade4a-0_20200308213934.log.2_3-761601-172985464", 
> ".7104bb0b-20f6-4dec-981b-c11bf20ade4a-0_20200308213934.log.3_1-772174-175483657",
>  ".7104bb0b-20f6-4dec-981b-c11bf20ade4a-0_20200308213934.log.4_2-782377-
> 177872977", 
> ".7104bb0b-20f6-4dec-981b-c11bf20ade4a-0_20200308213934.log.5_1-790994-179909226"],
>  "dataFilePath": 
> "7104bb0b-20f6-4dec-981b-c11bf20ade4a-0_0-746201-169642460_20200308213934.parquet",
>  "fileId": "7
> 104bb0b-20f6-4dec-981b-c11bf20ade4a-0", "partitionPath": 
> "created_at_month=2020-03", "metrics": {"TOTAL_LOG_FILES": 5.0, 
> "TOTAL_IO_READ_MB": 512.0, "TOTAL_LOG_FILES_SIZE": 33789.0, 
> "TOTAL_IO_WRITE_MB": 512.0,
>  "TOTAL_IO_MB": 1024.0, "TOTAL_LOG_FILE_SIZE": 33789.0}}), 
> (20200308213934,{"baseInstantTime": "20200308180755", "deltaFilePaths": 
> [".7104bb0b-20f6-4dec-981b-c11bf20ade4a-0_20200308180755.log.1_3-696047-158157865",
>  
> ".7104bb0b-20f6-4dec-981b-c11bf20ade4a-0_20200308180755.log.2_2-706457-160605423",
>  
> ".7104bb0b-20f6-4dec-981b-c11bf20ade4a-0_20200308180755.log.3_1-716977-163056814",
>  ".7104bb0b-20f6-4dec-981b-c11bf20ad
> e4a-0_20200308180755.log.4_3-727192-165430450", 
> ".7104bb0b-20f6-4dec-981b-c11bf20ade4a-0_20200308180755.log.5_3-737755-167913339"],
>  "dataFilePath": "7104bb0b-20f6-4dec-981b-c11bf20ade4a-0_0-690668-157158597_2
> 0200308180755.parquet", "fileId": "7104bb0b-20f6-4dec-981b-c11bf20ade4a-0", 
> "partitionPath": "created_at_month=2020-03", "metrics": {"TOTAL_LOG_FILES": 
> 5.0, "TOTAL_IO_READ_MB": 512.0, "TOTAL_LOG_FILES_SIZE":
> 44197.0, "TOTAL_IO_WRITE_MB": 511.0, "TOTAL_IO_MB": 1023.0, 
> "TOTAL_LOG_FILE_SIZE": 44197.0}})        at 
> org.apache.hudi.common.util.CompactionUtils.lambda$getAllPendingCompactionOperations$5(CompactionUtils.java:161)
>         at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>         at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>         at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>         at java.util.Iterator.forEachRemaining(Iterator.java:116)
>         at 
> java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
>         at 
> java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>         at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>         at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>         at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>         at 
> java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>         at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
>         at 
> java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272)
>         at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382)
>         at 
> java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>         at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>         at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>         at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>         at 
> java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>         at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
>         at 
> org.apache.hudi.common.util.CompactionUtils.getAllPendingCompactionOperations(CompactionUtils.java:149)
>         at 
> org.apache.hudi.common.table.view.AbstractTableFileSystemView.init(AbstractTableFileSystemView.java:95)
>         at 
> org.apache.hudi.common.table.view.HoodieTableFileSystemView.init(HoodieTableFileSystemView.java:87)
>         at 
> org.apache.hudi.common.table.view.HoodieTableFileSystemView.<init>(HoodieTableFileSystemView.java:81)
>         at 
> org.apache.hudi.common.table.view.HoodieTableFileSystemView.<init>(HoodieTableFileSystemView.java:72)
>         at 
> org.apache.hudi.utilities.HoodieSnapshotCopier.snapshot(HoodieSnapshotCopier.java:74)
>         at 
> org.apache.hudi.utilities.HoodieSnapshotCopier.main(HoodieSnapshotCopier.java:176)
>         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.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
>         at 
> org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:853)
>         at 
> org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:161)
>         at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:184)
>         at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
>         at 
> org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:928)
>         at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:937)
>         at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> 20/03/09 15:43:21 INFO SparkContext: Invoking stop() from shutdown hook
> 20/03/09 15:43:21 INFO SparkUI: Stopped Spark web UI at 
> http://ip-10-49-25-197.us-east-2.compute.internal:4041
> 20/03/09 15:43:21 INFO YarnClientSchedulerBackend: Interrupting monitor thread
> 20/03/09 15:43:21 INFO YarnClientSchedulerBackend: Shutting down all executors
> 20/03/09 15:43:21 INFO YarnSchedulerBackend$YarnDriverEndpoint: Asking each 
> executor to shut down
> 20/03/09 15:43:21 INFO SchedulerExtensionServices: Stopping 
> SchedulerExtensionServices
> (serviceOption=None,
>  services=List(), {code}
>  
>  
> This is only occurring for larger tables. We notice that this class always 
> chooses the last Instant to fetch the latest Hudi commit.
> Option<HoodieInstant> latestCommit =
>         
> tableMetadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant();
>  
>  
> And wonder if this could be made more resilient in the case of this error; 
> instead continuing to seek backwards until it finds a valid Instant that is 
> not being processed for compaction. {{filtercompletedInstants}} isn’t working 
> as expected and is *not* filtering out an {{Instant}} in process of being 
> compacted and leading to this error.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to