You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by sa...@apache.org on 2021/02/26 22:44:01 UTC

[hudi] branch master updated: [Hudi-1583]: Fix bug that Hudi will skip remaining log files if there is logFile with zero size in logFileList when merge on read. (#2584)

This is an automated email from the ASF dual-hosted git repository.

satish pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new be257b5  [Hudi-1583]: Fix bug that Hudi will skip remaining log files if there is logFile with zero size in logFileList when merge on read. (#2584)
be257b5 is described below

commit be257b58c689510a21529019a766b7a2bfc7ebe6
Author: Liulietong <li...@sina.com>
AuthorDate: Sat Feb 27 06:43:47 2021 +0800

    [Hudi-1583]: Fix bug that Hudi will skip remaining log files if there is logFile with zero size in logFileList when merge on read. (#2584)
    
    Co-authored-by: liulietong <li...@bytedance.com>
---
 .../java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java
index a5834d2..7267227 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java
@@ -104,7 +104,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
         throw new HoodieIOException("unable to initialize read with log file ", io);
       }
       LOG.info("Moving to the next reader for logfile " + currentReader.getLogFile());
-      return this.currentReader.hasNext();
+      return hasNext();
     }
     return false;
   }