You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@uniffle.apache.org by ck...@apache.org on 2023/02/16 06:23:20 UTC

[incubator-uniffle] branch branch-0.5 created (now 1554f9f5)

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

ckj pushed a change to branch branch-0.5
in repository https://gitbox.apache.org/repos/asf/incubator-uniffle.git


      at 1554f9f5 [Bug] Fix skip() api maybe skip unexpected bytes which makes inconsistent data (#40) (#52)

This branch includes the following new commits:

     new 1554f9f5 [Bug] Fix skip() api maybe skip unexpected bytes which makes inconsistent data (#40) (#52)

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[incubator-uniffle] 01/01: [Bug] Fix skip() api maybe skip unexpected bytes which makes inconsistent data (#40) (#52)

Posted by ck...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ckj pushed a commit to branch branch-0.5
in repository https://gitbox.apache.org/repos/asf/incubator-uniffle.git

commit 1554f9f5f6a6e117174794526b3a20699078efae
Author: Colin <co...@tencent.com>
AuthorDate: Tue Jul 12 16:32:15 2022 +0800

    [Bug] Fix skip() api maybe skip unexpected bytes which makes inconsistent data (#40) (#52)
    
    ### What changes were proposed in this pull request?
    Fix bug when call `inputstream.skip()` which may return unexpected result
    
    
    ### Why are the changes needed?
    Get exception messages as following, and it maybe caused by unexpected data from `Local` storage
    ```
    com.tencent.rss.common.exception.RssException: Unexpected crc value for blockId[9992363390829154], expected:2562548848, actual:2244862586
            at com.tencent.rss.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:184)
            at org.apache.spark.shuffle.reader.RssShuffleDataIterator.hasNext(RssShuffleDataIterator.java:99)
            at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
            at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
    ```
    
    
    ### Does this PR introduce _any_ user-facing change?
    No
    
    
    ### How was this patch tested?
    With current UTs
---
 .../rss/storage/handler/impl/LocalFileReader.java       | 17 ++++++++++++++++-
 1 file changed, 16 insertions(+), 1 deletion(-)

diff --git a/storage/src/main/java/com/tencent/rss/storage/handler/impl/LocalFileReader.java b/storage/src/main/java/com/tencent/rss/storage/handler/impl/LocalFileReader.java
index 8ff2f5fb..f16846a7 100644
--- a/storage/src/main/java/com/tencent/rss/storage/handler/impl/LocalFileReader.java
+++ b/storage/src/main/java/com/tencent/rss/storage/handler/impl/LocalFileReader.java
@@ -42,7 +42,22 @@ public class LocalFileReader implements FileReader, Closeable {
 
   public byte[] read(long offset, int length) {
     try {
-      dataInputStream.skip(offset);
+      long targetSkip = offset;
+      // comments from skip API:
+      // The skip method may, for a variety of reasons,
+      // end up skipping over some smaller number of bytes, possibly 0
+      // the result should be checked and try again until skip expectation length
+      while (targetSkip > 0) {
+        long realSkip = dataInputStream.skip(targetSkip);
+        if (realSkip == -1) {
+          throw new RuntimeException("Unexpected EOF when skip bytes");
+        }
+        targetSkip -= realSkip;
+        if (targetSkip > 0) {
+          LOG.warn("Got unexpected skip for path:" + path + " with offset["
+              + offset + "], length[" + length + "], remain[" + targetSkip + "]");
+        }
+      }
       byte[] buf = new byte[length];
       dataInputStream.readFully(buf);
       return buf;