Skip to content

Commit

Permalink
AVRO-3594: FsInput to use openFile() API
Browse files Browse the repository at this point in the history
Boost performance reading from object stores in
hadoop by using the openFile builder API
and passing in the file length as an option
(can save a HEAD) and asks for adaptive IO
(sequential going to random if the client starts seeking)
  • Loading branch information
steveloughran committed Aug 4, 2022
1 parent b201f5d commit 19c0695
Showing 1 changed file with 11 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,8 @@

import org.apache.avro.file.SeekableInput;

import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;

/** Adapt an {@link FSDataInputStream} to {@link SeekableInput}. */
public class FsInput implements Closeable, SeekableInput {
private final FSDataInputStream stream;
Expand All @@ -41,7 +43,15 @@ public FsInput(Path path, Configuration conf) throws IOException {
/** Construct given a path and a {@code FileSystem}. */
public FsInput(Path path, FileSystem fileSystem) throws IOException {
this.len = fileSystem.getFileStatus(path).getLen();
this.stream = fileSystem.open(path);
// use the hadoop 3.3.0 openFile API and specify length
// and read policy. object stores can use these to
// optimize read performance.
// the read policy "adaptive" means "start sequential but
// go to random IO after backwards seeks"
// Filesystems which don't recognize the options will ignore them

this.stream = awaitFuture(fileSystem.openFile(path).opt("fs.option.openfile.read.policy", "adaptive")
.opt("fs.option.openfile.length", Long.toString(len)).build());
}

@Override
Expand Down

0 comments on commit 19c0695

Please sign in to comment.