zherenyu831 opened a new issue #2707:
URL: https://github.com/apache/hudi/issues/2707


   **_Tips before filing an issue_**
   
   - Have you gone through our 
[FAQs](https://cwiki.apache.org/confluence/display/HUDI/FAQ)?
   
   - Join the mailing list to engage in conversations and get faster support at 
[email protected].
   
   - If you have triaged this as a bug, then file an 
[issue](https://issues.apache.org/jira/projects/HUDI/issues) directly.
   
   **Describe the problem you faced**
   Configs
   ```
   hoodie.cleaner.incremental.mode -> true
   hoodie.insert.shuffle.parallelism -> 20
   hoodie.datasource.write.precombine.field -> daas_internal_ts
   hoodie.clean.automatic -> false
   hoodie.datasource.write.operation -> insert_overwrite_table
   hoodie.datasource.write.recordkey.field -> guid
   hoodie.table.name -> xxxxx
   hoodie.datasource.write.table.type -> MERGE_ON_READ
   hoodie.datasource.write.hive_style_partitioning -> true
   hoodie.consistency.check.enabled -> true
   hoodie.cleaner.policy -> KEEP_LATEST_FILE_VERSIONS
   hoodie.datasource.write.keygenerator.class -> 
org.apache.hudi.keygen.ComplexKeyGenerator
   hoodie.keep.max.commits -> 3
   hoodie.cleaner.commits.retained -> 1
   hoodie.keep.min.commits -> 2
   hoodie.datasource.write.partitionpath.field -> 
   hoodie.compact.inline.max.delta.commits -> 1
   ```
   
   after write 5 times we got below error
   ```
   21/03/23 09:29:38 ERROR MonitoringUtils$: sendFailureMetric - One of the 
batch jobs failed
   org.apache.hudi.exception.HoodieCommitException: Failed to archive commits
        at 
org.apache.hudi.table.HoodieTimelineArchiveLog.archive(HoodieTimelineArchiveLog.java:322)
        at 
org.apache.hudi.table.HoodieTimelineArchiveLog.archiveIfRequired(HoodieTimelineArchiveLog.java:138)
        at 
org.apache.hudi.client.AbstractHoodieWriteClient.postCommit(AbstractHoodieWriteClient.java:426)
        at 
org.apache.hudi.client.AbstractHoodieWriteClient.commitStats(AbstractHoodieWriteClient.java:188)
        at 
org.apache.hudi.client.SparkRDDWriteClient.commit(SparkRDDWriteClient.java:110)
        at 
org.apache.hudi.HoodieSparkSqlWriter$.commitAndPerformPostOperations(HoodieSparkSqlWriter.scala:442)
        at 
org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:218)
        at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:134)
        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:86)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:131)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(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:83)
        at 
org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:83)
        at 
org.apache.spark.sql.DataFrameWriter.$anonfun$runCommand$1(DataFrameWriter.scala:676)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:84)
        at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:165)
        at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:74)
        at 
org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:676)
        at 
org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:290)
        at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:271)
        at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:229)
        at 
jp.ne.paypay.daas.dataloader.writer.HudiDataWriter.write(HudiDataWriter.scala:272)
        at 
jp.ne.paypay.daas.dataloader.writer.HudiDataWriter.insertOverrideTable(HudiDataWriter.scala:161)
        at 
jp.ne.paypay.daas.dataloader.FileSystemJob$.mainProcedure(FileSystemJob.scala:107)
        at 
jp.ne.paypay.daas.dataloader.FileSystemJob$.main(FileSystemJob.scala:38)
        at jp.ne.paypay.daas.dataloader.FileSystemJob.main(FileSystemJob.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: java.lang.IllegalArgumentException: Positive number of partitions 
required
        at 
org.apache.spark.rdd.ParallelCollectionRDD$.slice(ParallelCollectionRDD.scala:119)
        at 
org.apache.spark.rdd.ParallelCollectionRDD.getPartitions(ParallelCollectionRDD.scala:97)
        at org.apache.spark.rdd.RDD.$anonfun$partitions$2(RDD.scala:253)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.rdd.RDD.partitions(RDD.scala:251)
        at 
org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49)
        at org.apache.spark.rdd.RDD.$anonfun$partitions$2(RDD.scala:253)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.rdd.RDD.partitions(RDD.scala:251)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2126)
        at org.apache.spark.rdd.RDD.$anonfun$collect$1(RDD.scala:945)
        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:363)
        at org.apache.spark.rdd.RDD.collect(RDD.scala:944)
        at org.apache.spark.api.java.JavaRDDLike.collect(JavaRDDLike.scala:361)
        at org.apache.spark.api.java.JavaRDDLike.collect$(JavaRDDLike.scala:360)
        at 
org.apache.spark.api.java.AbstractJavaRDDLike.collect(JavaRDDLike.scala:45)
        at 
org.apache.hudi.client.common.HoodieSparkEngineContext.map(HoodieSparkEngineContext.java:73)
        at 
org.apache.hudi.client.ReplaceArchivalHelper.deleteReplacedFileGroups(ReplaceArchivalHelper.java:72)
        at 
org.apache.hudi.table.HoodieTimelineArchiveLog.deleteReplacedFileGroups(HoodieTimelineArchiveLog.java:341)
        at 
org.apache.hudi.table.HoodieTimelineArchiveLog.archive(HoodieTimelineArchiveLog.java:303)
        ... 36 more
   ```
   
   
   What we found:
   Seems first commit's don't have any partitionToReplaceFileIds, and it was 
required when archiving files
   
   ```
   client.getCommitTimeline.getInstants.forEach(instant => {
         val metadata = 
HoodieReplaceCommitMetadata.fromBytes(client.getActiveTimeline.getInstantDetails(instant).get,
 classOf[HoodieReplaceCommitMetadata])
         val a = metadata.getPartitionToReplaceFileIds()
         println(s"${instant.toString}: size : ${a.size}")
       })
   
   // Exiting paste mode, now interpreting.
   
   [20210323080718__replacecommit__COMPLETED]: size : 0
   [20210323081449__replacecommit__COMPLETED]: size : 1
   [20210323082046__replacecommit__COMPLETED]: size : 1
   [20210323082758__replacecommit__COMPLETED]: size : 1
   [20210323084004__replacecommit__COMPLETED]: size : 1
   [20210323085044__replacecommit__COMPLETED]: size : 1
   [20210323085823__replacecommit__COMPLETED]: size : 1
   [20210323090550__replacecommit__COMPLETED]: size : 1
   [20210323091700__replacecommit__COMPLETED]: size : 1
   ```
   
   A clear and concise description of the problem.
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1. set the config as we set
   2. write 5 times
   
   
   **Expected behavior**
   
   A clear and concise description of what you expected to happen.
   
   **Environment Description**
   
   * Hudi version :
   
   * Spark version :
   
   * Hive version :
   
   * Hadoop version :
   
   * Storage (HDFS/S3/GCS..) :
   
   * Running on Docker? (yes/no) :
   
   
   **Additional context**
   
   Add any other context about the problem here.
   
   **Stacktrace**
   
   ```Add the stacktrace of the error.```
   
   


-- 
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:
[email protected]


Reply via email to