You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2022/02/09 18:39:51 UTC

[GitHub] [hadoop] yzhangal commented on a change in pull request #3736: HADOOP-18028. improve S3 read speed using prefetching & caching

yzhangal commented on a change in pull request #3736:
URL: https://github.com/apache/hadoop/pull/3736#discussion_r802970939



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/SingleFilePerBlockCache.java
##########
@@ -0,0 +1,328 @@
+/*
+ * 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 limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.common;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.nio.file.attribute.FileAttribute;
+import java.nio.file.attribute.PosixFilePermission;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Provides functionality necessary for caching blocks of data read from FileSystem.
+ * Each cache block is stored on the local disk as a separate file.
+ */
+public class SingleFilePerBlockCache implements BlockCache {
+  private static final Logger LOG = LoggerFactory.getLogger(SingleFilePerBlockCache.class);
+
+  // Blocks stored in this cache.
+  private Map<Integer, Entry> blocks = new ConcurrentHashMap<>();
+
+  // Number of times a block was read from this cache.
+  // Used for determining cache utilization factor.
+  private int numGets = 0;
+
+  private boolean closed;
+
+  // Cache entry.
+  // Each block is stored as a separate file.
+  private static class Entry {
+    private final int blockNumber;
+    private final Path path;
+    private final int size;
+    private final long checksum;
+
+    Entry(int blockNumber, Path path, int size, long checksum) {
+      this.blockNumber = blockNumber;
+      this.path = path;
+      this.size = size;
+      this.checksum = checksum;
+    }
+
+    @Override
+    public String toString() {
+      return String.format(
+          "([%03d] %s: size = %d, checksum = %d)",
+          this.blockNumber, this.path, this.size, this.checksum);
+    }
+  }
+
+  public SingleFilePerBlockCache() {
+  }
+
+  /**
+   * Indicates whether the given block is in this cache.
+   */
+  @Override
+  public boolean containsBlock(int blockNumber) {
+    return this.blocks.containsKey(blockNumber);
+  }
+
+  /**
+   * Gets the blocks in this cache.
+   */
+  @Override
+  public Iterable<Integer> blocks() {
+    return Collections.unmodifiableList(new ArrayList<Integer>(this.blocks.keySet()));
+  }
+
+  /**
+   * Gets the number of blocks in this cache.
+   */
+  @Override
+  public int size() {
+    return this.blocks.size();
+  }
+
+  /**
+   * Gets the block having the given {@code blockNumber}.
+   *
+   * @throws IllegalArgumentException if buffer is null.
+   */
+  @Override
+  public void get(int blockNumber, ByteBuffer buffer) throws IOException {
+    if (this.closed) {
+      return;
+    }
+
+    Validate.checkNotNull(buffer, "buffer");
+
+    Entry entry = this.getEntry(blockNumber);
+    buffer.clear();
+    this.readFile(entry.path, buffer);
+    buffer.rewind();
+
+    validateEntry(entry, buffer);
+  }
+
+  protected int readFile(Path path, ByteBuffer buffer) throws IOException {
+    int numBytesRead = 0;
+    int numBytes;
+    FileChannel channel = FileChannel.open(path, StandardOpenOption.READ);
+    while ((numBytes = channel.read(buffer)) > 0) {
+      numBytesRead += numBytes;
+    }
+    buffer.limit(buffer.position());
+    channel.close();
+    return numBytesRead;
+  }
+
+  private Entry getEntry(int blockNumber) {
+    Validate.checkNotNegative(blockNumber, "blockNumber");
+
+    Entry entry = this.blocks.get(blockNumber);
+    if (entry == null) {
+      throw new IllegalStateException(String.format("block %d not found in cache", blockNumber));
+    }
+    this.numGets++;
+    return entry;
+  }
+
+  /**
+   * Puts the given block in this cache.
+   *
+   * @throws IllegalArgumentException if buffer is null.
+   * @throws IllegalArgumentException if buffer.limit() is zero or negative.
+   */
+  @Override
+  public void put(int blockNumber, ByteBuffer buffer) throws IOException {
+    if (this.closed) {
+      return;
+    }
+
+    Validate.checkNotNull(buffer, "buffer");
+
+    if (this.blocks.containsKey(blockNumber)) {
+      Entry entry = this.blocks.get(blockNumber);
+      validateEntry(entry, buffer);
+      return;
+    }
+
+    Validate.checkPositiveInteger(buffer.limit(), "buffer.limit()");
+
+    Path blockFilePath = getCacheFilePath();
+    long size = Files.size(blockFilePath);
+    if (size != 0) {
+      String message =
+          String.format("[%d] temp file already has data. %s (%d)",
+          blockNumber, blockFilePath, size);
+      throw new IllegalStateException(message);
+    }
+
+    this.writeFile(blockFilePath, buffer);
+    long checksum = BufferData.getChecksum(buffer);
+    Entry entry = new Entry(blockNumber, blockFilePath, buffer.limit(), checksum);
+    this.blocks.put(blockNumber, entry);
+  }
+
+  private static final Set<? extends OpenOption> CREATE_OPTIONS =
+      EnumSet.of(StandardOpenOption.WRITE,
+          StandardOpenOption.CREATE,
+          StandardOpenOption.TRUNCATE_EXISTING);
+
+  protected void writeFile(Path path, ByteBuffer buffer) throws IOException {
+    buffer.rewind();
+    WritableByteChannel writeChannel = Files.newByteChannel(path, CREATE_OPTIONS);
+    while (buffer.hasRemaining()) {
+      writeChannel.write(buffer);
+    }
+    writeChannel.close();
+  }
+
+  protected Path getCacheFilePath() throws IOException {
+    return getTempFilePath();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.closed) {
+      return;
+    }
+
+    this.closed = true;
+
+    LOG.info(this.getStats());
+    int numFilesDeleted = 0;
+
+    for (Entry entry : this.blocks.values()) {
+      try {
+        Files.deleteIfExists(entry.path);
+        numFilesDeleted++;
+      } catch (IOException e) {
+        // Ignore while closing so that we can delete as many cache files as possible.
+      }
+    }
+
+    if (numFilesDeleted > 0) {
+      LOG.info("Deleted {} cache files", numFilesDeleted);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("stats: ");
+    sb.append(getStats());
+    sb.append(", blocks:[");
+    sb.append(this.getIntList(this.blocks()));
+    sb.append("]");
+    return sb.toString();
+  }
+
+  private void validateEntry(Entry entry, ByteBuffer buffer) {
+    if (entry.size != buffer.limit()) {
+      String message = String.format(
+          "[%d] entry.size(%d) != buffer.limit(%d)",
+          entry.blockNumber, entry.size, buffer.limit());
+      throw new IllegalStateException(message);
+    }
+
+    long checksum = BufferData.getChecksum(buffer);
+    if (entry.checksum != checksum) {
+      String message = String.format(
+          "[%d] entry.checksum(%d) != buffer checksum(%d)",
+          entry.blockNumber, entry.checksum, checksum);
+      throw new IllegalStateException(message);
+    }
+  }
+
+  private String getIntList(Iterable<Integer> nums) {
+    List<String> numList = new ArrayList<>();
+    List<Integer> numbers = new ArrayList<Integer>();
+    for (Integer n : nums) {

Review comment:
       nums.forEach(numbers::add) can be used to replace the loop.

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
##########
@@ -276,6 +279,19 @@
   private TransferManager transfers;
   private ExecutorService boundedThreadPool;
   private ThreadPoolExecutor unboundedThreadPool;
+
+  // S3 reads are prefetched asynchronously using this future pool.
+  private FuturePool futurePool;
+
+  // If true, the prefetching input stream is used for reads.
+  private boolean prefetchEnabled;
+
+  // Size in bytes of a single prefetch block.
+  private int prefetchBlockSize;
+
+  // Size of prefetch queue (in number of blocks).
+  private int prefetchBlockCount;

Review comment:
       suggest to change prefetchBlockCount to prefetchThreadCount.

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java
##########
@@ -67,10 +69,10 @@ public void testUnbuffer() throws IOException {
 
     // Call read and then unbuffer
     FSDataInputStream stream = fs.open(path);
-    assertEquals(0, stream.read(new byte[8])); // mocks read 0 bytes
+    assertEquals(-1, stream.read(new byte[8])); // mocks read 0 bytes

Review comment:
       Please add comment to explain why return value is -1 instead of 0 here

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockData.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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 limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.common;
+
+/**
+ * Holds information about blocks of data in a file.
+ */
+public class BlockData {

Review comment:
       Suggest to add more detailed description of the values, for example, NOT_READY, what's the meaning of ready? the meaning of queued, what's the diff between ready and cached?

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/SingleFilePerBlockCache.java
##########
@@ -0,0 +1,328 @@
+/*
+ * 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 limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.common;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.nio.file.attribute.FileAttribute;
+import java.nio.file.attribute.PosixFilePermission;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Provides functionality necessary for caching blocks of data read from FileSystem.
+ * Each cache block is stored on the local disk as a separate file.
+ */
+public class SingleFilePerBlockCache implements BlockCache {
+  private static final Logger LOG = LoggerFactory.getLogger(SingleFilePerBlockCache.class);
+
+  // Blocks stored in this cache.
+  private Map<Integer, Entry> blocks = new ConcurrentHashMap<>();
+
+  // Number of times a block was read from this cache.
+  // Used for determining cache utilization factor.
+  private int numGets = 0;
+
+  private boolean closed;
+
+  // Cache entry.
+  // Each block is stored as a separate file.
+  private static class Entry {
+    private final int blockNumber;
+    private final Path path;
+    private final int size;
+    private final long checksum;
+
+    Entry(int blockNumber, Path path, int size, long checksum) {
+      this.blockNumber = blockNumber;
+      this.path = path;
+      this.size = size;
+      this.checksum = checksum;
+    }
+
+    @Override
+    public String toString() {
+      return String.format(
+          "([%03d] %s: size = %d, checksum = %d)",
+          this.blockNumber, this.path, this.size, this.checksum);
+    }
+  }
+
+  public SingleFilePerBlockCache() {
+  }
+
+  /**
+   * Indicates whether the given block is in this cache.
+   */
+  @Override
+  public boolean containsBlock(int blockNumber) {
+    return this.blocks.containsKey(blockNumber);
+  }
+
+  /**
+   * Gets the blocks in this cache.
+   */
+  @Override
+  public Iterable<Integer> blocks() {
+    return Collections.unmodifiableList(new ArrayList<Integer>(this.blocks.keySet()));
+  }
+
+  /**
+   * Gets the number of blocks in this cache.
+   */
+  @Override
+  public int size() {
+    return this.blocks.size();
+  }
+
+  /**
+   * Gets the block having the given {@code blockNumber}.
+   *
+   * @throws IllegalArgumentException if buffer is null.
+   */
+  @Override
+  public void get(int blockNumber, ByteBuffer buffer) throws IOException {
+    if (this.closed) {
+      return;
+    }
+
+    Validate.checkNotNull(buffer, "buffer");
+
+    Entry entry = this.getEntry(blockNumber);
+    buffer.clear();
+    this.readFile(entry.path, buffer);
+    buffer.rewind();
+
+    validateEntry(entry, buffer);
+  }
+
+  protected int readFile(Path path, ByteBuffer buffer) throws IOException {
+    int numBytesRead = 0;
+    int numBytes;
+    FileChannel channel = FileChannel.open(path, StandardOpenOption.READ);
+    while ((numBytes = channel.read(buffer)) > 0) {
+      numBytesRead += numBytes;
+    }
+    buffer.limit(buffer.position());
+    channel.close();
+    return numBytesRead;
+  }
+
+  private Entry getEntry(int blockNumber) {
+    Validate.checkNotNegative(blockNumber, "blockNumber");
+
+    Entry entry = this.blocks.get(blockNumber);
+    if (entry == null) {
+      throw new IllegalStateException(String.format("block %d not found in cache", blockNumber));
+    }
+    this.numGets++;
+    return entry;
+  }
+
+  /**
+   * Puts the given block in this cache.
+   *
+   * @throws IllegalArgumentException if buffer is null.
+   * @throws IllegalArgumentException if buffer.limit() is zero or negative.
+   */
+  @Override
+  public void put(int blockNumber, ByteBuffer buffer) throws IOException {
+    if (this.closed) {
+      return;
+    }
+
+    Validate.checkNotNull(buffer, "buffer");
+
+    if (this.blocks.containsKey(blockNumber)) {
+      Entry entry = this.blocks.get(blockNumber);
+      validateEntry(entry, buffer);
+      return;
+    }
+
+    Validate.checkPositiveInteger(buffer.limit(), "buffer.limit()");
+
+    Path blockFilePath = getCacheFilePath();
+    long size = Files.size(blockFilePath);
+    if (size != 0) {
+      String message =
+          String.format("[%d] temp file already has data. %s (%d)",
+          blockNumber, blockFilePath, size);
+      throw new IllegalStateException(message);
+    }
+
+    this.writeFile(blockFilePath, buffer);
+    long checksum = BufferData.getChecksum(buffer);
+    Entry entry = new Entry(blockNumber, blockFilePath, buffer.limit(), checksum);
+    this.blocks.put(blockNumber, entry);
+  }
+
+  private static final Set<? extends OpenOption> CREATE_OPTIONS =
+      EnumSet.of(StandardOpenOption.WRITE,
+          StandardOpenOption.CREATE,
+          StandardOpenOption.TRUNCATE_EXISTING);
+
+  protected void writeFile(Path path, ByteBuffer buffer) throws IOException {
+    buffer.rewind();
+    WritableByteChannel writeChannel = Files.newByteChannel(path, CREATE_OPTIONS);
+    while (buffer.hasRemaining()) {
+      writeChannel.write(buffer);
+    }
+    writeChannel.close();
+  }
+
+  protected Path getCacheFilePath() throws IOException {
+    return getTempFilePath();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.closed) {
+      return;
+    }
+
+    this.closed = true;
+
+    LOG.info(this.getStats());
+    int numFilesDeleted = 0;
+
+    for (Entry entry : this.blocks.values()) {
+      try {
+        Files.deleteIfExists(entry.path);
+        numFilesDeleted++;
+      } catch (IOException e) {
+        // Ignore while closing so that we can delete as many cache files as possible.
+      }
+    }
+
+    if (numFilesDeleted > 0) {
+      LOG.info("Deleted {} cache files", numFilesDeleted);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("stats: ");
+    sb.append(getStats());
+    sb.append(", blocks:[");
+    sb.append(this.getIntList(this.blocks()));
+    sb.append("]");
+    return sb.toString();
+  }
+
+  private void validateEntry(Entry entry, ByteBuffer buffer) {
+    if (entry.size != buffer.limit()) {
+      String message = String.format(
+          "[%d] entry.size(%d) != buffer.limit(%d)",
+          entry.blockNumber, entry.size, buffer.limit());
+      throw new IllegalStateException(message);
+    }
+
+    long checksum = BufferData.getChecksum(buffer);
+    if (entry.checksum != checksum) {
+      String message = String.format(
+          "[%d] entry.checksum(%d) != buffer checksum(%d)",
+          entry.blockNumber, entry.checksum, checksum);
+      throw new IllegalStateException(message);
+    }
+  }
+
+  private String getIntList(Iterable<Integer> nums) {
+    List<String> numList = new ArrayList<>();
+    List<Integer> numbers = new ArrayList<Integer>();
+    for (Integer n : nums) {
+      numbers.add(n);
+    }
+    Collections.sort(numbers);
+
+    int index = 0;
+    while (index < numbers.size()) {

Review comment:
       The logic in this loop is not straightforward, suggest to add a comment what it does

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3EInputStream.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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 limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.read;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CanSetReadahead;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.common.Validate;
+import org.apache.hadoop.fs.s3a.S3AInputStream;
+import org.apache.hadoop.fs.s3a.S3AReadOpContext;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * Enhanced {@code InputStream} for reading from S3.
+ *
+ * This implementation provides improved read throughput by asynchronously prefetching
+ * blocks of configurable size from the underlying S3 file.
+ */
+public class S3EInputStream

Review comment:
       Thanks for the good work here Kumar. 
   
   Suggest to change S3EInputStream to S3AInputStreamWithPrefetch, say, it also log S3AStatistics, it's implemented in S3AFileSystem, it's better to have a signatur    e S3A in its name, rather than S3E, and it would be good to stick to the categorization described in https://web.archive.org/web/20170718025436/https://aws.amazon.    com/premiumsupport/knowledge-center/emr-file-system-s3/

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/SingleFilePerBlockCache.java
##########
@@ -0,0 +1,328 @@
+/*
+ * 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 limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.common;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.nio.file.attribute.FileAttribute;
+import java.nio.file.attribute.PosixFilePermission;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Provides functionality necessary for caching blocks of data read from FileSystem.
+ * Each cache block is stored on the local disk as a separate file.
+ */
+public class SingleFilePerBlockCache implements BlockCache {
+  private static final Logger LOG = LoggerFactory.getLogger(SingleFilePerBlockCache.class);
+
+  // Blocks stored in this cache.
+  private Map<Integer, Entry> blocks = new ConcurrentHashMap<>();
+
+  // Number of times a block was read from this cache.
+  // Used for determining cache utilization factor.
+  private int numGets = 0;
+
+  private boolean closed;
+
+  // Cache entry.
+  // Each block is stored as a separate file.
+  private static class Entry {
+    private final int blockNumber;
+    private final Path path;
+    private final int size;
+    private final long checksum;
+
+    Entry(int blockNumber, Path path, int size, long checksum) {
+      this.blockNumber = blockNumber;
+      this.path = path;
+      this.size = size;
+      this.checksum = checksum;
+    }
+
+    @Override
+    public String toString() {
+      return String.format(
+          "([%03d] %s: size = %d, checksum = %d)",
+          this.blockNumber, this.path, this.size, this.checksum);
+    }
+  }
+
+  public SingleFilePerBlockCache() {
+  }
+
+  /**
+   * Indicates whether the given block is in this cache.
+   */
+  @Override
+  public boolean containsBlock(int blockNumber) {
+    return this.blocks.containsKey(blockNumber);
+  }
+
+  /**
+   * Gets the blocks in this cache.
+   */
+  @Override
+  public Iterable<Integer> blocks() {
+    return Collections.unmodifiableList(new ArrayList<Integer>(this.blocks.keySet()));
+  }
+
+  /**
+   * Gets the number of blocks in this cache.
+   */
+  @Override
+  public int size() {
+    return this.blocks.size();
+  }
+
+  /**
+   * Gets the block having the given {@code blockNumber}.
+   *
+   * @throws IllegalArgumentException if buffer is null.
+   */
+  @Override
+  public void get(int blockNumber, ByteBuffer buffer) throws IOException {
+    if (this.closed) {
+      return;
+    }
+
+    Validate.checkNotNull(buffer, "buffer");
+
+    Entry entry = this.getEntry(blockNumber);
+    buffer.clear();
+    this.readFile(entry.path, buffer);
+    buffer.rewind();
+
+    validateEntry(entry, buffer);
+  }
+
+  protected int readFile(Path path, ByteBuffer buffer) throws IOException {
+    int numBytesRead = 0;
+    int numBytes;
+    FileChannel channel = FileChannel.open(path, StandardOpenOption.READ);
+    while ((numBytes = channel.read(buffer)) > 0) {
+      numBytesRead += numBytes;
+    }
+    buffer.limit(buffer.position());
+    channel.close();
+    return numBytesRead;
+  }
+
+  private Entry getEntry(int blockNumber) {
+    Validate.checkNotNegative(blockNumber, "blockNumber");
+
+    Entry entry = this.blocks.get(blockNumber);
+    if (entry == null) {
+      throw new IllegalStateException(String.format("block %d not found in cache", blockNumber));
+    }
+    this.numGets++;
+    return entry;
+  }
+
+  /**
+   * Puts the given block in this cache.
+   *
+   * @throws IllegalArgumentException if buffer is null.
+   * @throws IllegalArgumentException if buffer.limit() is zero or negative.
+   */
+  @Override
+  public void put(int blockNumber, ByteBuffer buffer) throws IOException {
+    if (this.closed) {
+      return;
+    }
+
+    Validate.checkNotNull(buffer, "buffer");
+
+    if (this.blocks.containsKey(blockNumber)) {
+      Entry entry = this.blocks.get(blockNumber);
+      validateEntry(entry, buffer);
+      return;
+    }
+
+    Validate.checkPositiveInteger(buffer.limit(), "buffer.limit()");
+
+    Path blockFilePath = getCacheFilePath();
+    long size = Files.size(blockFilePath);
+    if (size != 0) {
+      String message =
+          String.format("[%d] temp file already has data. %s (%d)",
+          blockNumber, blockFilePath, size);
+      throw new IllegalStateException(message);
+    }
+
+    this.writeFile(blockFilePath, buffer);
+    long checksum = BufferData.getChecksum(buffer);
+    Entry entry = new Entry(blockNumber, blockFilePath, buffer.limit(), checksum);
+    this.blocks.put(blockNumber, entry);
+  }
+
+  private static final Set<? extends OpenOption> CREATE_OPTIONS =
+      EnumSet.of(StandardOpenOption.WRITE,
+          StandardOpenOption.CREATE,
+          StandardOpenOption.TRUNCATE_EXISTING);
+
+  protected void writeFile(Path path, ByteBuffer buffer) throws IOException {
+    buffer.rewind();
+    WritableByteChannel writeChannel = Files.newByteChannel(path, CREATE_OPTIONS);
+    while (buffer.hasRemaining()) {
+      writeChannel.write(buffer);
+    }
+    writeChannel.close();
+  }
+
+  protected Path getCacheFilePath() throws IOException {
+    return getTempFilePath();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.closed) {
+      return;
+    }
+
+    this.closed = true;
+
+    LOG.info(this.getStats());
+    int numFilesDeleted = 0;
+
+    for (Entry entry : this.blocks.values()) {
+      try {
+        Files.deleteIfExists(entry.path);
+        numFilesDeleted++;
+      } catch (IOException e) {
+        // Ignore while closing so that we can delete as many cache files as possible.
+      }
+    }
+
+    if (numFilesDeleted > 0) {
+      LOG.info("Deleted {} cache files", numFilesDeleted);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("stats: ");
+    sb.append(getStats());
+    sb.append(", blocks:[");
+    sb.append(this.getIntList(this.blocks()));
+    sb.append("]");
+    return sb.toString();
+  }
+
+  private void validateEntry(Entry entry, ByteBuffer buffer) {
+    if (entry.size != buffer.limit()) {
+      String message = String.format(
+          "[%d] entry.size(%d) != buffer.limit(%d)",
+          entry.blockNumber, entry.size, buffer.limit());
+      throw new IllegalStateException(message);
+    }
+
+    long checksum = BufferData.getChecksum(buffer);
+    if (entry.checksum != checksum) {
+      String message = String.format(
+          "[%d] entry.checksum(%d) != buffer checksum(%d)",
+          entry.blockNumber, entry.checksum, checksum);
+      throw new IllegalStateException(message);
+    }
+  }
+
+  private String getIntList(Iterable<Integer> nums) {

Review comment:
       Suggest to add javadoc to describe what the method does

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3EInputStream.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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 limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.read;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CanSetReadahead;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.common.Validate;
+import org.apache.hadoop.fs.s3a.S3AInputStream;
+import org.apache.hadoop.fs.s3a.S3AReadOpContext;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * Enhanced {@code InputStream} for reading from S3.
+ *
+ * This implementation provides improved read throughput by asynchronously prefetching
+ * blocks of configurable size from the underlying S3 file.
+ */
+public class S3EInputStream

Review comment:
       It would be nice to evaluate and describe the number of S3 accesses implication of this change so users can be aware of. See https://issues.apache.org/jira/browse/HADOOP-14965?focusedCommentId=17489161&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17489161 for reference.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org