forked from apache/hudi
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[HUDI-4282] Repair IOException in CHDFS when check block corrupted in…
… HoodieLogFileReader (apache#6031) Co-authored-by: Y Ethan Guo <[email protected]> (cherry picked from commit eaa2f8e)
- Loading branch information
1 parent
fa0b60b
commit 6da4bf4
Showing
3 changed files
with
81 additions
and
1 deletion.
There are no files selected for viewing
68 changes: 68 additions & 0 deletions
68
hudi-common/src/main/java/org/apache/hudi/common/fs/BoundedFsDataInputStream.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,68 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
*/ | ||
|
||
package org.apache.hudi.common.fs; | ||
|
||
import org.apache.hadoop.fs.FSDataInputStream; | ||
import org.apache.hadoop.fs.FileSystem; | ||
import org.apache.hadoop.fs.Path; | ||
|
||
import java.io.EOFException; | ||
import java.io.IOException; | ||
import java.io.InputStream; | ||
|
||
public class BoundedFsDataInputStream extends FSDataInputStream { | ||
private FileSystem fs; | ||
private Path file; | ||
private long fileLen = -1L; | ||
|
||
public BoundedFsDataInputStream(FileSystem fs, Path file, InputStream in) { | ||
super(in); | ||
this.fs = fs; | ||
this.file = file; | ||
} | ||
|
||
@Override | ||
public boolean markSupported() { | ||
return false; | ||
} | ||
|
||
/* Return the file length */ | ||
private long getFileLength() throws IOException { | ||
if (fileLen == -1L) { | ||
fileLen = fs.getContentSummary(file).getLength(); | ||
} | ||
return fileLen; | ||
} | ||
|
||
@Override | ||
public synchronized void seek(long pos) throws IOException { | ||
if (pos < 0 || pos > getFileLength()) { | ||
throw new EOFException("Try to seek pos[" + pos + "] , but fileSize is " + getFileLength()); | ||
} | ||
super.seek(pos); | ||
} | ||
|
||
@Override | ||
public synchronized long skip(long n) throws IOException { | ||
long curPos = getPos(); | ||
long fileLength = getFileLength(); | ||
if (n + curPos > fileLength) { | ||
n = fileLength - curPos; | ||
} | ||
return super.skip(n); | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters