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

Balaji Varadarajan commented on HUDI-716:
-----------------------------------------

Spoke with Alex to debug this issue. It has to do with failed clean operation 
before upgrade to 0.5.[1,2].  Here is the context

```

Before 0.5.1, hudi was handling clean differently than commit action. If a 
clean failed during final step, there would be .clean.inflight files (empty or 
corrupted) lying around but pre 0.5.1 did not care. With 0.5.1 onwards, we are 
handling clean action consistently like commit. Hudi would first scan all files 
and store the plan (atomically) in .clean.requested and .clean.inflight before 
triggering the actual clean. If there are any intermittent failures, subsequent 
clean would read the cleaner plan again and retry.

 

In this case, Cleaner is reading an empty .clean.inflight plan and looking for 
Cleaner plan but it is empty. 

```

One workaround would be to catch exception in the below section and delete 
pending clean

```

// If there are inflight(failed) or previously requested clean operation, first 
perform them
table.getCleanTimeline().filterInflightsAndRequested().getInstants().forEach(hoodieInstant
 -> {
 LOG.info("There were previously unfinished cleaner operations. Finishing 
Instant=" + hoodieInstant);
 runClean(table, hoodieInstant);
});

```

 

 

> Exception: Not an Avro data file when running HoodieCleanClient.runClean
> ------------------------------------------------------------------------
>
>                 Key: HUDI-716
>                 URL: https://issues.apache.org/jira/browse/HUDI-716
>             Project: Apache Hudi (incubating)
>          Issue Type: Bug
>          Components: DeltaStreamer
>            Reporter: Alexander Filipchik
>            Assignee: lamber-ken
>            Priority: Major
>             Fix For: 0.6.0
>
>
> Just upgraded to upstream master from 0.5 and seeing an issue at the end of 
> the delta sync run: 
> 20/03/17 02:13:49 ERROR HoodieDeltaStreamer: Got error running delta sync 
> once. Shutting down20/03/17 02:13:49 ERROR HoodieDeltaStreamer: Got error 
> running delta sync once. Shutting 
> downorg.apache.hudi.exception.HoodieIOException: Not an Avro data file at 
> org.apache.hudi.client.HoodieCleanClient.runClean(HoodieCleanClient.java:144) 
> at 
> org.apache.hudi.client.HoodieCleanClient.lambda$clean$0(HoodieCleanClient.java:88)
>  at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382)
>  at 
> java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:580) 
> at org.apache.hudi.client.HoodieCleanClient.clean(HoodieCleanClient.java:86) 
> at org.apache.hudi.client.HoodieWriteClient.clean(HoodieWriteClient.java:843) 
> at 
> org.apache.hudi.client.HoodieWriteClient.postCommit(HoodieWriteClient.java:520)
>  at 
> org.apache.hudi.client.AbstractHoodieWriteClient.commit(AbstractHoodieWriteClient.java:168)
>  at 
> org.apache.hudi.client.AbstractHoodieWriteClient.commit(AbstractHoodieWriteClient.java:111)
>  at 
> org.apache.hudi.utilities.deltastreamer.DeltaSync.writeToSink(DeltaSync.java:395)
>  at 
> org.apache.hudi.utilities.deltastreamer.DeltaSync.syncOnce(DeltaSync.java:237)
>  at 
> org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.sync(HoodieDeltaStreamer.java:121)
>  at 
> org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.main(HoodieDeltaStreamer.java:294)
>  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:845)
>  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:920) 
> at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:929) at 
> org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)Caused by: 
> java.io.IOException: Not an Avro data file at 
> org.apache.avro.file.DataFileReader.openReader(DataFileReader.java:50) at 
> org.apache.hudi.common.util.AvroUtils.deserializeAvroMetadata(AvroUtils.java:147)
>  at 
> org.apache.hudi.common.util.CleanerUtils.getCleanerPlan(CleanerUtils.java:87) 
> at 
> org.apache.hudi.client.HoodieCleanClient.runClean(HoodieCleanClient.java:141) 
> ... 24 more
>  
> It is attempting to read an old cleanup file (2 month old) and crashing
>  



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

Reply via email to