Skip to content

Commit

Permalink
[HUDI-4282] Wrap FSDataInputStream by BoundedFsDataInputStream to rep…
Browse files Browse the repository at this point in the history
…air IOException in some dfs.
  • Loading branch information
5herhom committed Jul 10, 2022
1 parent c0e1587 commit 5db3df8
Show file tree
Hide file tree
Showing 3 changed files with 101 additions and 0 deletions.
@@ -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);
}

}
12 changes: 12 additions & 0 deletions hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
Expand Up @@ -85,6 +85,9 @@ public class FSUtils {

private static final PathFilter ALLOW_ALL_FILTER = file -> true;

private static final Set<StorageSchemes> WRAPPED_BOUNDED_FS_DATA_STREAM_SCHEMES =
Stream.of(StorageSchemes.CHDFS).collect(Collectors.toSet());

public static Configuration prepareHadoopConf(Configuration conf) {
// look for all properties, prefixed to be picked up
for (Entry<String, String> prop : System.getenv().entrySet()) {
Expand Down Expand Up @@ -632,6 +635,15 @@ public static boolean isGCSFileSystem(FileSystem fs) {
return fs.getScheme().equals(StorageSchemes.GCS.getScheme());
}

/**
* Some filesystem(such as chdfs) will throw {@code IOException} instead of {@code EOFException}. It will cause error in isBlockCorrupted().
* Wrapped by {@code BoundedFsDataInputStream}, to check whether the desired offset is out of the file size in advance.
*/
public static boolean shouldWrappedByBoundedDataStream(FileSystem fs) {
return WRAPPED_BOUNDED_FS_DATA_STREAM_SCHEMES.stream()
.map(e -> e.getScheme()).anyMatch(schema -> schema.equals(fs.getScheme()));
}

public static Configuration registerFileSystem(Path file, Configuration conf) {
Configuration returnConf = new Configuration(conf);
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
Expand Down
Expand Up @@ -18,6 +18,7 @@

package org.apache.hudi.common.table.log;

import org.apache.hudi.common.fs.BoundedFsDataInputStream;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.fs.SchemeAwareFSDataInputStream;
import org.apache.hudi.common.fs.TimedFSDataInputStream;
Expand Down Expand Up @@ -478,6 +479,10 @@ private static FSDataInputStream getFSDataInputStream(FileSystem fs,
return new SchemeAwareFSDataInputStream(getFSDataInputStreamForGCS(fsDataInputStream, logFile, bufferSize), true);
}

if (FSUtils.shouldWrappedByBoundedDataStream(fs)) {
return wrapStreamByBoundedFsDataInputStream(fs, logFile, fsDataInputStream, bufferSize);
}

if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) {
return new TimedFSDataInputStream(logFile.getPath(), new FSDataInputStream(
new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize)));
Expand Down Expand Up @@ -516,4 +521,20 @@ private static FSDataInputStream getFSDataInputStreamForGCS(FSDataInputStream fs

return fsDataInputStream;
}

/**
* Some filesystem(such as chdfs) will throw {@code IOException} instead of {@code EOFException}. It will cause error in isBlockCorrupted().
* Wrapped by {@code BoundedFsDataInputStream}, to check whether the desired offset is out of the file size in advance.
*/
private static FSDataInputStream wrapStreamByBoundedFsDataInputStream(FileSystem fs,
HoodieLogFile logFile,
FSDataInputStream fsDataInputStream,
int bufferSize) {
if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) {
return new TimedFSDataInputStream(logFile.getPath(), new BoundedFsDataInputStream(fs, logFile.getPath(), new FSDataInputStream(
new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize))));
}

return new BoundedFsDataInputStream(fs, logFile.getPath(), fsDataInputStream);
}
}

0 comments on commit 5db3df8

Please sign in to comment.