[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-04-17 Thread Vinoth Chandar (Jira)


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

Vinoth Chandar commented on HUDI-716:
-

[~afilipchik] if you are still facing this issue. please share a listing of 
.hoodie with us, before and after migration.. that should give us signals to 
troubleshoot

> 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
>  Labels: pull-request-available
> Fix For: 0.6.0
>
> Attachments: image-2020-03-21-02-45-25-099.png, 
> image-2020-03-21-13-37-17-039.png
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> 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)


[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-04-16 Thread lamber-ken (Jira)


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

lamber-ken commented on HUDI-716:
-

[https://github.com/apache/incubator-hudi/pull/1432]

> 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
>  Labels: pull-request-available
> Fix For: 0.6.0
>
> Attachments: image-2020-03-21-02-45-25-099.png, 
> image-2020-03-21-13-37-17-039.png
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> 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)


[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-04-16 Thread Alexander Filipchik (Jira)


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

Alexander Filipchik commented on HUDI-716:
--

[~lamber-ken] what class? Don't see a fix in the HoodieActiveTimeline and test 
deployment is breaking

> 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
>  Labels: pull-request-available
> Fix For: 0.6.0
>
> Attachments: image-2020-03-21-02-45-25-099.png, 
> image-2020-03-21-13-37-17-039.png
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> 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)


[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-03-20 Thread lamber-ken (Jira)


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

lamber-ken commented on HUDI-716:
-

The life cycle of *.clean files: (base on hudi-0.5.0) 
[https://github.com/apache/incubator-hudi/blob/release-0.5.0|https://github.com/apache/incubator-hudi/blob/release-0.5.0/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java]

!image-2020-03-21-13-37-17-039.png!

> 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
>
> Attachments: image-2020-03-21-02-45-25-099.png, 
> image-2020-03-21-13-37-17-039.png
>
>
> 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)


[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-03-20 Thread lamber-ken (Jira)


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

lamber-ken commented on HUDI-716:
-

Hi [~vbalaji], I agree with your point. from code we can see that create new 
file first, then write content.

 

HoodieActiveTimeline#createFileInPath

!image-2020-03-21-02-45-25-099.png!

> 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
>
> Attachments: image-2020-03-21-02-45-25-099.png
>
>
> 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)


[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-03-20 Thread Balaji Varadarajan (Jira)


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

Balaji Varadarajan commented on HUDI-716:
-

[~lamber-ken]: My understanding is that this is due to crashes when we create 
clean inflight and transition to complete files (in 0.5.0 code). But, It would 
greatly help if you can objectively take a new look at the code and try repro 
why this is happening.

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


[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-03-20 Thread lamber-ken (Jira)


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

lamber-ken commented on HUDI-716:
-

Also, we need to figure it out what jobs created these zero bytes files

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


[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-03-18 Thread Balaji Varadarajan (Jira)


[ 
https://issues.apache.org/jira/browse/HUDI-716?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-03-18 Thread Vinoth Chandar (Jira)


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

Vinoth Chandar commented on HUDI-716:
-

I think it happens only when there is an inflight .clean file and the upgrade 
happens? 
[~vbalaji] 

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


[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-03-18 Thread lamber-ken (Jira)


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

lamber-ken commented on HUDI-716:
-

I tried to reproduce it, but it works ok.

*Step1: Use hudi 0.5.0 generate old datas*
{code:java}
export SPARK_HOME=/work/BigData/install/spark/spark-2.4.4-bin-hadoop2.7
${SPARK_HOME}/bin/spark-shell \
  --packages org.apache.hudi:hudi-spark-bundle:0.5.0-incubating \
  --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer'

val tableName = "hudi_mor_table"
val basePath = "file:///tmp/hudi_mor_table"
var datas = List("""{ "name": "kenken", "ts": "qwer", "age": 12, "location": 
"latitude"}""")

val df = spark.read.json(spark.sparkContext.parallelize(datas, 2))
df.write.format("org.apache.hudi").
option("hoodie.insert.shuffle.parallelism", "10").
option("hoodie.upsert.shuffle.parallelism", "10").
option("hoodie.delete.shuffle.parallelism", "10").
option("hoodie.bulkinsert.shuffle.parallelism", "10").
option("hoodie.datasource.write.recordkey.field", "name").
option("hoodie.datasource.write.partitionpath.field", "location").
option("hoodie.datasource.write.precombine.field", "ts").
option("hoodie.table.name", tableName).
mode("Overwrite").
save(basePath)

var datas = List.tabulate(30)(i => List(s"""{ "name": "kenken${i}", "ts": 
"zasz", "age": 123, "location": "latitude"}"""))
for (data <- datas) {
  val df = spark.read.json(spark.sparkContext.parallelize(data, 2))
  df.write.format("org.apache.hudi").
option("hoodie.insert.shuffle.parallelism", "10").
option("hoodie.upsert.shuffle.parallelism", "10").
option("hoodie.delete.shuffle.parallelism", "10").
option("hoodie.bulkinsert.shuffle.parallelism", "10").
option("hoodie.datasource.write.recordkey.field", "name").
option("hoodie.datasource.write.partitionpath.field", "location").
option("hoodie.datasource.write.precombine.field", "ts").
option("hoodie.table.name", tableName).
option("hoodie.keep.max.commits", "5").
option("hoodie.keep.min.commits", "4").
option("hoodie.cleaner.commits.retained", "3").
mode("Append").
save(basePath)
}

spark.read.format("org.apache.hudi").load(basePath + "/*/").show()
{code}

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

[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-03-18 Thread lamber-ken (Jira)


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

lamber-ken commented on HUDI-716:
-

[~vinoth] willing to take it. :) 

 

Hi [~afilipchik], waiting for you, need more detail infomation as balaji 
mentioned, thanks.

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


[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-03-16 Thread Balaji Varadarajan (Jira)


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

Balaji Varadarajan commented on HUDI-716:
-

[~afilipchik]: Can you attach the listing of .hoodie folder and also contents 
of hoodie.properties.

Also, Any chance do you know what version of hoodie was running 2 months back ?

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


[jira] [Commented] (HUDI-716) Exception: Not an Avro data file when running HoodieCleanClient.runClean

2020-03-16 Thread Vinoth Chandar (Jira)


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

Vinoth Chandar commented on HUDI-716:
-

[~vbalaji] [~nagarwal] one more report of the same thing.. 

 

Alex mentioned that this is a zero size file.. 

 

cc @lamber-ken  are you up for taking a crack at this? (since you love these 
challenging ones :)) 

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