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 2020/06/24 04:40:17 UTC

[GitHub] [hudi] zuyanton opened a new issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

zuyanton opened a new issue #1764:
URL: https://github.com/apache/hudi/issues/1764


   
   **Describe the problem you faced**
   We are running MoR table on EMR+Hudi+S3 with ```hoodie.consistency.check.enabled```set to true with compaction set to be executed inline.We update table every ten minutes with new data. We see following issue (actually two issues) . 
   First issue is that compaction fails from time to time with exception ```HoodieCommitException: Failed to complete commit 20200624012710 due to finalize errors. caused by HoodieIOException: Consistency check failed to ensure all files APPEAR.``` Looks like Hudi tries to clean up duplicate data files created due to spark retries but consistency check fails because files are not there. This error does not appear when we disable consistency check by setting up hoodie.consistency.check.enabled to false, cause Hudi proceeds with attempt to delete non existing duplicate files and wraps up commit successfully, however since we use S3, having consistency check disabled is not ideal. First issue more often happens on bigger tables (>400gb) then small ones (<100gbs) 
   Second issue is that after First issue happens, Hudi never changes commit status and it stays INFLIGHT forever, which causes several other issues, like log files with the same fileID as parquet files that were part of failed compaction never get compacted, Hudi start ignoring cleaning settings and stops removing all the commits that happen after failed commit.  Although second issue in our case is caused by first issue, its still doen't seem right to leave compaction in INFLIGHT status after failure. 
   
   **To Reproduce**
   Create a MoR table with ~100 partitions saved to S3, run updates for a while with consistency check enabled and compaction set to be run inline. eventually one of the compaction jobs should fail and compaction commit should stay in INFLIGHT status
   
   
   **Environment Description**
   
   * Hudi version : 0.5.3
   
   * Spark version :2.4.4
   
   * Hive version :
   
   * Hadoop version :2.8.5
   
   * Storage (HDFS/S3/GCS..) : S3
   
   * Running on Docker? (yes/no) : no
   
   
   **Additional context**
   
   Hudi settings that we use:
             "hoodie.consistency.check.enabled"->"true",
             "hoodie.compact.inline.max.delta.commits"->"12",
             "hoodie.compact.inline"->"true",
             "hoodie.clean.automatic"->"true",
             "hoodie.cleaner.commits.retained"->"2",
             DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY → "true",
   
   **Stacktrace**
   
   ```  
   20/06/24 01:38:05 INFO HoodieTable: Removing duplicate data files created due to spark retries before committing. Paths=[s3://bucketName/tableName/30/5bb5c4d5-a54a-4682-93d1-98ef3222d887-1_0-30-9408_20200624012710.parquet]
   20/06/24 01:42:22 ERROR ApplicationMaster: User class threw exception: org.apache.hudi.exception.HoodieCommitException: Failed to complete commit 20200624012710 due to finalize errors.
   org.apache.hudi.exception.HoodieCommitException: Failed to complete commit 20200624012710 due to finalize errors.
   	at org.apache.hudi.client.AbstractHoodieWriteClient.finalizeWrite(AbstractHoodieWriteClient.java:204)
   	at org.apache.hudi.client.HoodieWriteClient.doCompactionCommit(HoodieWriteClient.java:1129)
   	at org.apache.hudi.client.HoodieWriteClient.commitCompaction(HoodieWriteClient.java:1089)
   	at org.apache.hudi.client.HoodieWriteClient.runCompaction(HoodieWriteClient.java:1072)
   	at org.apache.hudi.client.HoodieWriteClient.compact(HoodieWriteClient.java:1043)
   	at org.apache.hudi.client.HoodieWriteClient.lambda$forceCompact$12(HoodieWriteClient.java:1158)
   	at org.apache.hudi.common.util.Option.ifPresent(Option.java:96)
   	at org.apache.hudi.client.HoodieWriteClient.forceCompact(HoodieWriteClient.java:1155)
   	at org.apache.hudi.client.HoodieWriteClient.postCommit(HoodieWriteClient.java:502)
   	at org.apache.hudi.client.AbstractHoodieWriteClient.commit(AbstractHoodieWriteClient.java:157)
   	at org.apache.hudi.client.AbstractHoodieWriteClient.commit(AbstractHoodieWriteClient.java:101)
   	at org.apache.hudi.client.AbstractHoodieWriteClient.commit(AbstractHoodieWriteClient.java:92)
   	at org.apache.hudi.HoodieSparkSqlWriter$.checkWriteStatus(HoodieSparkSqlWriter.scala:268)
   	at org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:188)
   	at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:108)
   	at org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:45)
   	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:86)
   	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
   	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
   	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:156)
   	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
   	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
   	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
   	at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80)
   	at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80)
   	at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
   	at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
   	at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78)
   	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
   	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73)
   	at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:676)
   	at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:285)
   	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:271)
   	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:229)
   	at com.amazon.fdl.components.compaction.job.CompactionHudiJob2$.main(CompactionHudiJob2.scala:147)
   	at com.amazon.fdl.components.compaction.job.CompactionHudiJob2.main(CompactionHudiJob2.scala)
   	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.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:684)
   Caused by: org.apache.hudi.exception.HoodieIOException: Consistency check failed to ensure all files APPEAR
   	at org.apache.hudi.table.HoodieTable.waitForAllFiles(HoodieTable.java:431)
   	at org.apache.hudi.table.HoodieTable.cleanFailedWrites(HoodieTable.java:379)
   	at org.apache.hudi.table.HoodieTable.finalizeWrite(HoodieTable.java:315)
   	at org.apache.hudi.table.HoodieMergeOnReadTable.finalizeWrite(HoodieMergeOnReadTable.java:319)
   	at org.apache.hudi.client.AbstractHoodieWriteClient.finalizeWrite(AbstractHoodieWriteClient.java:195)
   	... 42 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.

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



[GitHub] [hudi] vinothchandar commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   > just skipping (and not failing the job) it should work better as an interim solution given the current design of using marker files
   
   may be we should discuss the scenario here more explicitly.. are you referring to listing after marker file creation? so, by skipping, you mean - move on to writing, then hope the marker file is indeed available, by the time we finalize?
   
   @bvaradar can you also chime in with your thoughts? 


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

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



[GitHub] [hudi] umehrot2 commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   So this seems like a bug in Hudi. The simplest and reasonable solution seems to me to move `createMarkerFile` call after the `storageWriter` is instantiated so that we never create lingering marker files.
   https://github.com/apache/hudi/blob/master/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java#L70
   
   If will make this change and test for my failing workload, and put up a PR if this seems fine. cc @vinothchandar


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

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



[GitHub] [hudi] umehrot2 commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   @vinothchandar Agreed, I realized this soon after so didn't proceed with this approach. I am just thinking if we really need to wait for all files to appear here, or even if we need to wait, if at the end of the wait period the file is not present it should be safe to assume that file never got created.
   
   Atleast for S3 I can say that it is eventually consistent in the order of few 100 milliseconds. If after waiting so much (7 consistency checks by default) the file is not present, we can assume that it was never created.
   
   Another, approach can be that if parquet file didn't get created then we try to delete the marker file. But then again it can potentially fail to delete the marker as well.
   
   We can possibly do a combination of both.


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

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



[GitHub] [hudi] vinothchandar edited a comment on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

Posted by GitBox <gi...@apache.org>.
vinothchandar edited a comment on issue #1764:
URL: https://github.com/apache/hudi/issues/1764#issuecomment-648882567


   @zuyanton thanks for reporting this.. let's work together to resolve this. can you please paste the `.hoodie/hoodie.properties` file? The .inflight file hanging around could be normal or not, depending on the timeline layout version.. So checking for that. 
   
   The consistency check itself is tunable.. See https://github.com/apache/hudi/blob/release-0.5.3/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java#L92 although I would expect the defaults to be sufficient most of the time (have not seen a lot of issues like this so far).. cc @umehrot2  any suggestions here? 
   
   On the second problem, 
   >like log files with the same fileID as parquet files that were part of failed compaction never get compacted,
   The next run of compaction should try and attempt to complete this inflight compaction again.. cc @bvaradar can you confirm 
   
   a) with inline compaction, a failed compaction would be re-attempted in the next run 
   b) we will perform clean up from .aux for all marker files that we may not have deleted yet (due to such finalize errors).. 
    
   
   


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

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



[GitHub] [hudi] umehrot2 commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   I think my assumption is right. The root cause is S3 throttling that causes intermittent tasks to be failed and retried while writing the parquet files.
   
   ```
   20/06/26 02:19:06 WARN TaskSetManager: Lost task 539.0 in stage 1.0 (TID 9300, ip-172-30-0-24.ec2.internal, executor 74): com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.services.s3.model.AmazonS3Exception: Slow Down (Service: Amazon S3; Status Code: 503; Error Code: 503 Slow Down; Request ID: FAE1300E894176E8; S3 Extended Request ID: y4HlnhS5ClPb+DlERbIYW4kGOa2EqP1Ghio0krjgu+dBhlgPzwhNRnN5OL8h9vCCLfaiv8/0HTk=), S3 Extended Request ID: y4HlnhS5ClPb+DlERbIYW4kGOa2EqP1Ghio0krjgu+dBhlgPzwhNRnN5OL8h9vCCLfaiv8/0HTk=
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1742)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1371)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1347)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1127)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:784)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:752)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:726)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:686)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:668)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:532)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:512)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5052)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4998)
   	at com.amazon.ws.emr.hadoop.fs.shaded.com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1335)
   	at com.amazon.ws.emr.hadoop.fs.s3.lite.call.GetObjectMetadataCall.perform(GetObjectMetadataCall.java:22)
   	at com.amazon.ws.emr.hadoop.fs.s3.lite.call.GetObjectMetadataCall.perform(GetObjectMetadataCall.java:8)
   	at com.amazon.ws.emr.hadoop.fs.s3.lite.executor.GlobalS3Executor.execute(GlobalS3Executor.java:114)
   	at com.amazon.ws.emr.hadoop.fs.s3.lite.AmazonS3LiteClient.invoke(AmazonS3LiteClient.java:189)
   	at com.amazon.ws.emr.hadoop.fs.s3.lite.AmazonS3LiteClient.invoke(AmazonS3LiteClient.java:184)
   	at com.amazon.ws.emr.hadoop.fs.s3.lite.AmazonS3LiteClient.getObjectMetadata(AmazonS3LiteClient.java:96)
   	at com.amazon.ws.emr.hadoop.fs.s3.lite.AbstractAmazonS3Lite.getObjectMetadata(AbstractAmazonS3Lite.java:43)
   	at com.amazon.ws.emr.hadoop.fs.s3n.Jets3tNativeFileSystemStore.getFileMetadataFromCacheOrS3(Jets3tNativeFileSystemStore.java:497)
   	at com.amazon.ws.emr.hadoop.fs.s3n.Jets3tNativeFileSystemStore.retrieveMetadata(Jets3tNativeFileSystemStore.java:223)
   	at com.amazon.ws.emr.hadoop.fs.s3n.S3NativeFileSystem.getFileStatus(S3NativeFileSystem.java:597)
   	at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1440)
   	at com.amazon.ws.emr.hadoop.fs.s3.upload.plan.RegularUploadPlanner.checkExistenceIfNotOverwriting(RegularUploadPlanner.java:35)
   	at com.amazon.ws.emr.hadoop.fs.s3.upload.plan.RegularUploadPlanner.plan(RegularUploadPlanner.java:30)
   	at com.amazon.ws.emr.hadoop.fs.s3.upload.plan.UploadPlannerChain.plan(UploadPlannerChain.java:37)
   	at com.amazon.ws.emr.hadoop.fs.s3n.S3NativeFileSystem.create(S3NativeFileSystem.java:433)
   	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:932)
   	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:913)
   	at com.amazon.ws.emr.hadoop.fs.EmrFileSystem.create(EmrFileSystem.java:252)
   	at org.apache.hudi.common.fs.HoodieWrapperFileSystem.create(HoodieWrapperFileSystem.java:221)
   	at org.apache.parquet.hadoop.util.HadoopOutputFile.create(HadoopOutputFile.java:74)
   	at org.apache.parquet.hadoop.ParquetFileWriter.<init>(ParquetFileWriter.java:248)
   	at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:280)
   	at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:227)
   	at org.apache.hudi.io.storage.HoodieParquetWriter.<init>(HoodieParquetWriter.java:59)
   	at org.apache.hudi.io.storage.HoodieStorageWriterFactory.newParquetStorageWriter(HoodieStorageWriterFactory.java:67)
   	at org.apache.hudi.io.storage.HoodieStorageWriterFactory.getStorageWriter(HoodieStorageWriterFactory.java:48)
   	at org.apache.hudi.io.HoodieCreateHandle.<init>(HoodieCreateHandle.java:83)
   	at org.apache.hudi.io.HoodieBootstrapHandle.<init>(HoodieBootstrapHandle.java:33)
   	at org.apache.hudi.table.action.bootstrap.BootstrapCommitActionExecutor.handleMetadataBootstrap(BootstrapCommitActionExecutor.java:246)
   	at org.apache.hudi.table.action.bootstrap.BootstrapCommitActionExecutor.lambda$runMetadataBootstrap$237db4ee$1(BootstrapCommitActionExecutor.java:345)
   	at org.apache.spark.api.java.JavaPairRDD$$anonfun$toScalaFunction$1.apply(JavaPairRDD.scala:1040)
   	at scala.collection.Iterator$$anon$11.next(Iterator.scala:410)
   	at scala.collection.Iterator$$anon$11.next(Iterator.scala:410)
   	at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:222)
   	at org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:349)
   	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1181)
   	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1155)
   	at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1090)
   	at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1155)
   	at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:881)
   	at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:357)
   	at org.apache.spark.rdd.RDD.iterator(RDD.scala:308)
   	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
   	at org.apache.spark.scheduler.Task.run(Task.scala:123)
   	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
   	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1405)
   	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
   	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)
   ```
   
   As we can see that S3 throttled while trying to create the output parquet file, and hence it was not created in the first place. Later on when the same task was retried it succeeded. But from the previous retry we have a lingering `marker file` with no corresponding `parquet file`
   
   
   
   


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

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



[GitHub] [hudi] umehrot2 commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   @vinothchandar @bvaradar looking at the logic we are forming the list of invalid data file paths to be deleted from the marker file paths. One possible reason that seems to me can be that marker file got created but corresponding data file was never written by spark because failure happened before the file was written. Now we are expecting that file to appear, but it was never created in the first place. Do you guys think its possible ? I will also dive more into the marker file code to understand.
   
   On a similar note regarding handling of marker files, I have narrowed down some performance issues with S3 in the marker files clean up code. https://issues.apache.org/jira/browse/HUDI-1054 @zuyanton might be of interest to you.


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

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



[GitHub] [hudi] bvaradar commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   We discussed this in yesterday's community weekly sync. We have opened a blocker in 0.6 (Please see https://issues.apache.org/jira/browse/HUDI-1098) to provide ways to control this behavior.  Please subscribe to the ticket for progress. 


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

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



[GitHub] [hudi] umehrot2 edited a comment on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

Posted by GitBox <gi...@apache.org>.
umehrot2 edited a comment on issue #1764:
URL: https://github.com/apache/hudi/issues/1764#issuecomment-649932495


   So this seems like a bug in Hudi. The simplest and reasonable solution seems to me to move `createMarkerFile` call after the `storageWriter` is instantiated so that we never create lingering marker files.
   https://github.com/apache/hudi/blob/master/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java#L70
   
   I will make this change and test for my failing workload, and put up a PR if this seems fine. cc @vinothchandar


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

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



[GitHub] [hudi] vinothchandar commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   @zuyanton thanks for reporting this.. let's work together to resolve this. can you please paste the `.hoodie/hoodie.properties` file? The .inflight file hanging around could be normal or not, depending on the timeline layout version.. So checking for that. 
   
   The consistency check itself is tunable.. See https://github.com/apache/hudi/blob/release-0.5.3/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java#L92 although I would expect the defaults to be sufficient most of the time (have not seen a lot of issues like this so far).. cc @umehrot2  any suggestions here? 
   
   On the second problem, 
   >like log files with the same fileID as parquet files that were part of failed compaction never get compacted,
   The next run of compaction should try and attempt to complete this inflight compaction again.. cc @bvaradar can you confirm 
   a) with inline compaction, a failed compaction would be re-attempted in the next run 
   b) we will perform clean up from .aux for all marker files that we may not have deleted yet (due to such finalize errors).. 
    
   
   


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

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



[GitHub] [hudi] vinothchandar commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   @umehrot2 if we reverse the order, then it might violate guarantee that if there was a file created in storage, then there is a marker file involved... the task can open a parquet file, and crash without writing a marker right?


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

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



[GitHub] [hudi] bvaradar closed issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

Posted by GitBox <gi...@apache.org>.
bvaradar closed issue #1764:
URL: https://github.com/apache/hudi/issues/1764


   


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

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



[GitHub] [hudi] umehrot2 commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   Actually this is not just a problem with `Throttling`. AWS S3 can throw intermittent `Throttling` and well as `Internal Errors` which can potentially succeed upon retrying.
   
   I wish we were able to use https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-spark-s3-optimized-committer.html which solved a lot of the S3 related commit problems. The EMR file system will not commit the file until the spark task commit succeeds, essentially making this file commit atomic. Unfortunately, Hudi does not depend on sparks commit mechanisms to be able to leverage this.
   
   Yes, I think waiting for 1 attempt and if it does not appear, just skipping (**and not failing the job**) it should work better as an interim solution given the current design of using marker files. @bvaradar also has some thoughts of improving this in the long run.


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

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



[GitHub] [hudi] vinothchandar commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   >I am just thinking if we really need to wait for all files to appear here, or even if we need to wait, if at the end of the wait period the file is not present it should be safe to assume that file never got created.
   
   the marker file being like a write ahead log for parquet file is probably a better design to keep.. In the case, that happy path of checking after 2000ms 
   
   ```
     // time between successive attempts to ensure written data's metadata is consistent on storage
     private static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP =
         "hoodie.consistency.check.initial_interval_ms";
     private static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 2000L;
   ```
   
   should succeed most of the time right - in just one attempt right? are you suggesting we don't even do this check to overcome the throttling issue


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

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



[GitHub] [hudi] bvaradar commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   Thanks @zuyanton for reporting the issue. Regarding failed compaction jobs getting retried - As async compaction is the usual compaction mode run by users, we did not notice this earlier. Have created a patch to retry failed compactions first in inline compaction mode against 0.5.3 release.
   
   https://github.com/apache/hudi/pull/1765
   
    We will also fix this in 0.6.0.  Can you give this a try ?


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

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



[GitHub] [hudi] bvaradar edited a comment on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

Posted by GitBox <gi...@apache.org>.
bvaradar edited a comment on issue #1764:
URL: https://github.com/apache/hudi/issues/1764#issuecomment-658778968


   We discussed this in yesterday's community weekly sync. We have opened a blocker in 0.6 (Please see https://issues.apache.org/jira/browse/HUDI-1098) to provide ways to control this behavior.  Please subscribe to the ticket for tracking progress. 


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

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



[GitHub] [hudi] umehrot2 commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   Wow, I just started running into this same issue as I was running performance tests for RFC-12 bootstrap implementation. cc @bvaradar 
   ```
   org.apache.hudi.exception.HoodieCommitException: Failed to complete commit 00000000000001 due to finalize errors.
     at org.apache.hudi.table.action.commit.BaseCommitActionExecutor.finalizeWrite(BaseCommitActionExecutor.java:255)
     at org.apache.hudi.table.action.commit.BaseCommitActionExecutor.commit(BaseCommitActionExecutor.java:216)
     at org.apache.hudi.table.action.bootstrap.BootstrapCommitActionExecutor.commit(BootstrapCommitActionExecutor.java:205)
     at org.apache.hudi.table.action.commit.BaseCommitActionExecutor.commitOnAutoCommit(BaseCommitActionExecutor.java:178)
     at org.apache.hudi.table.action.commit.BaseCommitActionExecutor.updateIndexAndCommitIfNeeded(BaseCommitActionExecutor.java:172)
     at org.apache.hudi.table.action.bootstrap.BootstrapCommitActionExecutor.metadataBootstrap(BootstrapCommitActionExecutor.java:178)
     at org.apache.hudi.table.action.bootstrap.BootstrapCommitActionExecutor.execute(BootstrapCommitActionExecutor.java:136)
     at org.apache.hudi.table.HoodieCopyOnWriteTable.bootstrap(HoodieCopyOnWriteTable.java:143)
     at org.apache.hudi.client.HoodieWriteClient.bootstrap(HoodieWriteClient.java:158)
     at org.apache.hudi.HoodieSparkSqlWriter$.bootstrap(HoodieSparkSqlWriter.scala:203)
     at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:139)
     at org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:45)
     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:86)
     at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:173)
     at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:169)
     at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:197)
     at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
     at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:194)
     at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:169)
     at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:114)
     at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:112)
     at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
     at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:676)
     at org.apache.spark.sql.execution.SQLExecution$.org$apache$spark$sql$execution$SQLExecution$$executeQuery$1(SQLExecution.scala:83)
     at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1$$anonfun$apply$1.apply(SQLExecution.scala:94)
     at org.apache.spark.sql.execution.QueryExecutionMetrics$.withMetrics(QueryExecutionMetrics.scala:141)
     at org.apache.spark.sql.execution.SQLExecution$.org$apache$spark$sql$execution$SQLExecution$$withMetrics(SQLExecution.scala:178)
     at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:93)
     at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:200)
     at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:92)
     at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:676)
     at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:285)
     at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:271)
     at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:229)
     ... 49 elided
   Caused by: org.apache.hudi.exception.HoodieIOException: Consistency check failed to ensure all files APPEAR
     at org.apache.hudi.table.HoodieTable.waitForAllFiles(HoodieTable.java:553)
     at org.apache.hudi.table.HoodieTable.cleanFailedWrites(HoodieTable.java:495)
     at org.apache.hudi.table.HoodieTable.finalizeWrite(HoodieTable.java:398)
     at org.apache.hudi.table.action.commit.BaseCommitActionExecutor.finalizeWrite(BaseCommitActionExecutor.java:252)
     ... 84 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.

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



[GitHub] [hudi] zuyanton commented on issue #1764: [SUPPORT] Commits stays INFLIGHT forever after S3 consistency check fails when Hudi tries to delete duplicate datafiles

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


   Thank you for quick response.
   
   @vinothchandar  posting hoodie.properties file zipped. However when I open file on my end with sublime I only see bunch of hex numbers, not sure if thats how it supposed to be or its due to S3 encryption, let me know if you can make sense out of it.
   [hoodie.zip](https://github.com/apache/hudi/files/4826981/hoodie.zip) 
   
   In regards to increasing wait time for consistency checks - I dont think thats the issue. I think files are either never were there or they got deleted prior to compaction finalizing step. I created a bug report few days before [here](https://issues.apache.org/jira/browse/HUDI-1030) (I probably should have asked support first though). The same exception appears almost always when we try to bulk_import large amount of data. I was checking s3 location 10-20-40 minutes after failed import and files that Hudi tried to delete but failed never appeared in S3. So I strongly believe its not S3 taking too long to show up files.  
   
   to the second issue: let me test @bvaradar patch 
   
   


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

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