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/01/09 10:07:45 UTC

[GitHub] [incubator-hudi] RonBarabash opened a new issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

RonBarabash opened a new issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202
 
 
   Hey Guys, 
   We encountered this error running Hudi with MergeOnRead and inline compaction.
   I havent find anything related to this error + a way to recover from it.
   Have u ever encountered this? What is the best way to recover from a corrupted log fie? 
   The first error to appear is `com.uber.hoodie.exception.HoodieIOException`
   and the following one is:
   `com.uber.hoodie.exception.CorruptedLogFileException`
   
   
   
   
   **Environment Description**
   
   * Hudi version : 0.47
   
   * Spark version : 2.4.4
   
   * Hive version :1.3
   
   * Hadoop version : 2.9
   
   * Storage (HDFS/S3/GCS..) :s3
   
   * Running on Docker? (yes/no) : yes
   
   
   **Additional context**
   
   Add any other context about the problem here.
   
   **Stacktrace**
   
   ``` com.uber.hoodie.exception.HoodieIOException: IOException when reading log file 
   	at com.uber.hoodie.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:244)
   	at com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:76)
   	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.compact(HoodieRealtimeTableCompactor.java:124)
   	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.lambda$compact$44896304$1(HoodieRealtimeTableCompactor.java:95)
   	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$12.nextCur(Iterator.scala:435)
   	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441)
   	at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221)
   	at org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:349)
   	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182)
   	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156)
   	at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091)
   	at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156)
   	at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882)
   	at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335)
   	at org.apache.spark.rdd.RDD.iterator(RDD.scala:286)
   	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
   	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
   	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:1360)
   	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)```
   
   ```
   
   com.uber.hoodie.exception.CorruptedLogFileException: HoodieLogFile{pathStr='s3://yotpo-data-lake/tableViews/orders/order_lines_stream/table_view/0/.0a7846c4-66e3-4851-95eb-0b2c05c4b91a-0_20200108092342.log.19_710-645-102267', fileLen=0}could not be read. Did not find the magic bytes at the start of the block 	at com.uber.hoodie.common.table.log.HoodieLogFileReader.readMagic(HoodieLogFileReader.java:298) 	at com.uber.hoodie.common.table.log.HoodieLogFileReader.hasNext(HoodieLogFileReader.java:280) 	at com.uber.hoodie.common.table.log.HoodieLogFormatReader.hasNext(HoodieLogFormatReader.java:101) 	at com.uber.hoodie.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:128) 	at com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:76) 	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.compact(HoodieRealtimeTableCompactor.java:124) 	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.lambda$compact$44896304$1(HoodieRealtimeTableCompactor.java:95) 	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$12.nextCur(Iterator.scala:435) 	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441) 	at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221) 	at org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:349) 	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182) 	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) 	at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) 	at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) 	at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882) 	at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:286) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) 	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:1360) 	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)
   --
   
   
   
   ```

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] RonBarabash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
RonBarabash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202#issuecomment-574166330
 
 
   Input -> kafka, CDC events logs generated by debezium
   Processing -> spark structured streaming, we do some spark sql on the events
   Output -> Writing using spark to s3 with Hudi MergeOnRead
   this is are the hudi config:
   ``` "options": {
           "hoodie.compaction.strategy": "com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy",
           "hoodie.fail.on.timeline.archiving": "false",
           "hoodie.cleaner.commits.retained": "1",
           "hoodie.datasource.hive_sync.enable": "false",
           "hoodie.copyonwrite.record.size.estimate": "60",
           "hoodie.copyonwrite.insert.auto.split": "true",
           "hoodie.parquet.compression.codec": "snappy",
           "hoodie.index.bloom.num_entries": "1000000",
           "hoodie.compact.inline.max.delta.commits": "1",
         }```

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] RonBarabash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
RonBarabash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202#issuecomment-576832495
 
 
   not relevant anymore

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] RonBarabash closed issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
RonBarabash closed issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202
 
 
   

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] RonBarabash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
RonBarabash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202#issuecomment-574166866
 
 
   this is the first error we are getting:
   ```
   
   com.uber.hoodie.exception.CorruptedLogFileException: HoodieLogFile{pathStr='s3://yotpo-data-lake/table-views/orders/order_lines_stream/table_view/0/.d11ada9b-5e05-4649-9bbb-1b7f85f647fc-0_20200114075517.log.1_26-21-15497', fileLen=0}could not be read. Did not find the magic bytes at the start of the block 	at com.uber.hoodie.common.table.log.HoodieLogFileReader.readMagic(HoodieLogFileReader.java:298) 	at com.uber.hoodie.common.table.log.HoodieLogFileReader.hasNext(HoodieLogFileReader.java:280) 	at com.uber.hoodie.common.table.log.HoodieLogFormatReader.hasNext(HoodieLogFormatReader.java:84) 	at com.uber.hoodie.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:128) 	at com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:76) 	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.compact(HoodieRealtimeTableCompactor.java:124) 	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.lambda$compact$44896304$1(HoodieRealtimeTableCompactor.java:95) 	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$12.nextCur(Iterator.scala:435) 	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441) 	at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221) 	at org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:349) 	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182) 	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) 	at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) 	at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) 	at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882) 	at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:286) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) 	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:1360) 	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)
   --
   
   
   ```

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] RonBarabash edited a comment on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
RonBarabash edited a comment on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202#issuecomment-572489315
 
 
   ```
   com.uber.hoodie.exception.CorruptedLogFileException: HoodieLogFile{pathStr='s3://yotpo-data-lake/tableViews/orders/order_lines_stream/table_view/0/.0a7846c4-66e3-4851-95eb-0b2c05c4b91a-0_20200108092342.log.19_710-645-102267', fileLen=0}could not be read. Did not find the magic bytes at the start of the block 	at com.uber.hoodie.common.table.log.HoodieLogFileReader.readMagic(HoodieLogFileReader.java:298) 	at com.uber.hoodie.common.table.log.HoodieLogFileReader.hasNext(HoodieLogFileReader.java:280) 	at com.uber.hoodie.common.table.log.HoodieLogFormatReader.hasNext(HoodieLogFormatReader.java:101) 	at com.uber.hoodie.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:128) 	at com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:76) 	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.compact(HoodieRealtimeTableCompactor.java:124) 	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.lambda$compact$44896304$1(HoodieRealtimeTableCompactor.java:95) 	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$12.nextCur(Iterator.scala:435) 	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441) 	at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221) 	at org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:349) 	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182) 	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) 	at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) 	at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) 	at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882) 	at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:286) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) 	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:1360) 	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)
   ```

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] RonBarabash edited a comment on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
RonBarabash edited a comment on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202#issuecomment-572489315
 
 
   com.uber.hoodie.exception.CorruptedLogFileException: HoodieLogFile{pathStr='s3://yotpo-data-lake/tableViews/orders/order_lines_stream/table_view/0/.0a7846c4-66e3-4851-95eb-0b2c05c4b91a-0_20200108092342.log.19_710-645-102267', fileLen=0}could not be read. Did not find the magic bytes at the start of the block 	at com.uber.hoodie.common.table.log.HoodieLogFileReader.readMagic(HoodieLogFileReader.java:298) 	at com.uber.hoodie.common.table.log.HoodieLogFileReader.hasNext(HoodieLogFileReader.java:280) 	at com.uber.hoodie.common.table.log.HoodieLogFormatReader.hasNext(HoodieLogFormatReader.java:101) 	at com.uber.hoodie.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:128) 	at com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:76) 	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.compact(HoodieRealtimeTableCompactor.java:124) 	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.lambda$compact$44896304$1(HoodieRealtimeTableCompactor.java:95) 	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$12.nextCur(Iterator.scala:435) 	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441) 	at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221) 	at org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:349) 	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182) 	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) 	at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) 	at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) 	at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882) 	at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:286) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) 	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:1360) 	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)
   

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
n3nash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202#issuecomment-575745358
 
 
   @RonBarabash @vinothchandar yes, there are checks in place to be able to recover from it. As you pointed out, this might be an underlying infra issue that hopefully goes away after your config changes

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] cdmikechen commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
cdmikechen commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202#issuecomment-572542639
 
 
   Maybe you can upgrade to hudi 0.5.1 ?
   

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] RonBarabash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
RonBarabash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202#issuecomment-574294028
 
 
   Hey, im thinking this might be an s3 issue with inconsistency, we turned on `hoodie.consistency.check.enabled` and im thinking it solved, if it will not reproduce in the next couple of days i will close the issue.
   10x
   

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202#issuecomment-572624688
 
 
   `Did not find the magic bytes at the start of the block` ; this happens when a delta commit failed midway writing a block.. the scanner would be able to recover by seeking to next magic bytes.. Is that not happening? 
   
   if you don't mind, could you provide more context i.e series of events, workload characteristics, failure symptoms ? cc @n3nash I bet this sort of thing happens and we can recover out of it..

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


With regards,
Apache Git Services

[GitHub] [incubator-hudi] RonBarabash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File

Posted by GitBox <gi...@apache.org>.
RonBarabash commented on issue #1202: [SUPPORT] com.uber.hoodie.exception.HoodieIOException: IOException when reading log file + Corrupted Log File
URL: https://github.com/apache/incubator-hudi/issues/1202#issuecomment-572489315
 
 
   ```
   
   com.uber.hoodie.exception.CorruptedLogFileException: HoodieLogFile{pathStr='s3://yotpo-data-lake/tableViews/orders/order_lines_stream/table_view/0/.0a7846c4-66e3-4851-95eb-0b2c05c4b91a-0_20200108092342.log.19_710-645-102267', fileLen=0}could not be read. Did not find the magic bytes at the start of the block 	at com.uber.hoodie.common.table.log.HoodieLogFileReader.readMagic(HoodieLogFileReader.java:298) 	at com.uber.hoodie.common.table.log.HoodieLogFileReader.hasNext(HoodieLogFileReader.java:280) 	at com.uber.hoodie.common.table.log.HoodieLogFormatReader.hasNext(HoodieLogFormatReader.java:101) 	at com.uber.hoodie.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:128) 	at com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:76) 	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.compact(HoodieRealtimeTableCompactor.java:124) 	at com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor.lambda$compact$44896304$1(HoodieRealtimeTableCompactor.java:95) 	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$12.nextCur(Iterator.scala:435) 	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441) 	at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221) 	at org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:349) 	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182) 	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) 	at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) 	at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) 	at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882) 	at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:286) 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) 	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:1360) 	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)
   --
   
   
   ```

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


With regards,
Apache Git Services