[ 
https://issues.apache.org/jira/browse/HADOOP-18291?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17742425#comment-17742425
 ] 

ASF GitHub Bot commented on HADOOP-18291:
-----------------------------------------

steveloughran commented on code in PR #5754:
URL: https://github.com/apache/hadoop/pull/5754#discussion_r1261160595


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java:
##########
@@ -1276,4 +1276,16 @@ private Constants() {
   public static final String 
STORE_CAPABILITY_DIRECTORY_MARKER_MULTIPART_UPLOAD_ENABLED =
       "fs.s3a.capability.multipart.uploads.enabled";
 
+  /**
+   * Prefetch max blocks count config.
+   * Value = {@value PREFETCH_MAX_BLOCKS_COUNT}
+   */
+  public static final String PREFETCH_MAX_BLOCKS_COUNT = 
"fs.s3a.prefetch.max.blocks.count";
+
+  /**
+   * Default value for max blocks count config.
+   * Value = {@value DEFAULT_PREFETCH_MAX_BLOCKS_COUNT}
+   */
+  public static final int DEFAULT_PREFETCH_MAX_BLOCKS_COUNT = 10;

Review Comment:
   this is per stream? it's big. we may want to see what happens in apps with 
many threads -risk of out of disk is high.



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java:
##########
@@ -77,13 +78,13 @@ public ITestS3APrefetchingInputStream() {
 
   private static final int TIMEOUT_MILLIS = 5000;
   private static final int INTERVAL_MILLIS = 500;
-
+  private static final int PREFETCH_MAX_BLOCKS = 3;
 
   @Override
   public Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
-    S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);

Review Comment:
   this needs to be restored, and add PREFETCH_MAX_BLOCKS_COUNT, as another to 
cut.
   
   in all Itests, assume that any option set in `createConfiguration()' MAY 
have a per-bucket override set by someone, so you MUST explicitly remove it.



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import 
org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
+import static 
org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE;
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+
+/**
+ * Test the prefetching input stream with LRU cache eviction on 
S3ACachingInputStream.
+ */
+@RunWith(Parameterized.class)
+public class ITestS3APrefetchingLruEviction extends AbstractS3ACostTest {
+
+  private final String maxBlocks;
+
+  @Parameterized.Parameters(name = "max-blocks-{0}")
+  public static Collection<Object[]> params() {
+    return Arrays.asList(new Object[][]{
+        {"1"},
+        {"2"},
+        {"3"}
+    });
+  }
+
+  public ITestS3APrefetchingLruEviction(final String maxBlocks) {
+    super(true);
+    this.maxBlocks = maxBlocks;
+  }
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestS3APrefetchingLruEviction.class);
+
+  private static final int S_1K = 1024;
+  // Path for file which should have length > block size so 
S3ACachingInputStream is used
+  private Path largeFile;
+  private FileSystem largeFileFS;
+  private int blockSize;
+
+  private static final int TIMEOUT_MILLIS = 5000;
+  private static final int INTERVAL_MILLIS = 500;
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setBoolean(PREFETCH_ENABLED_KEY, true);

Review Comment:
   again, remove base+bucket overrides of the options



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import 
org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
+import static 
org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE;
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+
+/**
+ * Test the prefetching input stream with LRU cache eviction on 
S3ACachingInputStream.
+ */
+@RunWith(Parameterized.class)
+public class ITestS3APrefetchingLruEviction extends AbstractS3ACostTest {
+
+  private final String maxBlocks;
+
+  @Parameterized.Parameters(name = "max-blocks-{0}")
+  public static Collection<Object[]> params() {
+    return Arrays.asList(new Object[][]{
+        {"1"},
+        {"2"},
+        {"3"}
+    });
+  }
+
+  public ITestS3APrefetchingLruEviction(final String maxBlocks) {
+    super(true);
+    this.maxBlocks = maxBlocks;
+  }
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestS3APrefetchingLruEviction.class);
+
+  private static final int S_1K = 1024;
+  // Path for file which should have length > block size so 
S3ACachingInputStream is used
+  private Path largeFile;
+  private FileSystem largeFileFS;
+  private int blockSize;
+
+  private static final int TIMEOUT_MILLIS = 5000;
+  private static final int INTERVAL_MILLIS = 500;
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setBoolean(PREFETCH_ENABLED_KEY, true);
+    conf.setInt(PREFETCH_MAX_BLOCKS_COUNT, Integer.parseInt(maxBlocks));
+    return conf;
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    cleanupWithLogger(LOG, largeFileFS);
+    largeFileFS = null;
+  }
+
+  private void openFS() throws Exception {
+    Configuration conf = getConfiguration();
+    String largeFileUri = S3ATestUtils.getCSVTestFile(conf);
+
+    largeFile = new Path(largeFileUri);
+    blockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, 
PREFETCH_BLOCK_DEFAULT_SIZE);
+    largeFileFS = new S3AFileSystem();
+    largeFileFS.initialize(new URI(largeFileUri), getConfiguration());
+  }
+
+  @Test
+  public void testSeeksWithLruEviction() throws Throwable {
+    IOStatistics ioStats;
+    openFS();
+
+    ExecutorService executorService = Executors.newFixedThreadPool(5,
+        new ThreadFactoryBuilder()
+            .setDaemon(true)
+            .setNameFormat("testSeeksWithLruEviction-%d")
+            .build());
+    CountDownLatch countDownLatch = new CountDownLatch(7);
+
+    try (FSDataInputStream in = largeFileFS.open(largeFile)) {
+      ioStats = in.getIOStatistics();
+      // tests to add multiple blocks in the prefetch cache
+      // and let LRU eviction take place as more cache entries
+      // are added with multiple block reads.
+
+      executorService.submit(() -> {

Review Comment:
   these submitted closures are almost identical. why not define one for 
readFully(), one for PositionedRedable.readFully() and submit them; maybe as 
some function
   
   ```
   Callable<Boolean> readFully(CountDownLatch countDownLatch, boolean 
positionedReadable, long offset, long size) {
     // return one of these with seek(offset) and read to a buffer of [size]
   }
   then submit these
   ```
   



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import 
org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
+import static 
org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE;
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+
+/**
+ * Test the prefetching input stream with LRU cache eviction on 
S3ACachingInputStream.
+ */
+@RunWith(Parameterized.class)
+public class ITestS3APrefetchingLruEviction extends AbstractS3ACostTest {
+
+  private final String maxBlocks;
+
+  @Parameterized.Parameters(name = "max-blocks-{0}")
+  public static Collection<Object[]> params() {
+    return Arrays.asList(new Object[][]{
+        {"1"},
+        {"2"},
+        {"3"}
+    });
+  }
+
+  public ITestS3APrefetchingLruEviction(final String maxBlocks) {
+    super(true);
+    this.maxBlocks = maxBlocks;
+  }
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestS3APrefetchingLruEviction.class);
+
+  private static final int S_1K = 1024;
+  // Path for file which should have length > block size so 
S3ACachingInputStream is used
+  private Path largeFile;
+  private FileSystem largeFileFS;
+  private int blockSize;
+
+  private static final int TIMEOUT_MILLIS = 5000;
+  private static final int INTERVAL_MILLIS = 500;
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setBoolean(PREFETCH_ENABLED_KEY, true);
+    conf.setInt(PREFETCH_MAX_BLOCKS_COUNT, Integer.parseInt(maxBlocks));
+    return conf;
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    cleanupWithLogger(LOG, largeFileFS);
+    largeFileFS = null;
+  }
+
+  private void openFS() throws Exception {
+    Configuration conf = getConfiguration();
+    String largeFileUri = S3ATestUtils.getCSVTestFile(conf);
+
+    largeFile = new Path(largeFileUri);
+    blockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, 
PREFETCH_BLOCK_DEFAULT_SIZE);
+    largeFileFS = new S3AFileSystem();
+    largeFileFS.initialize(new URI(largeFileUri), getConfiguration());
+  }
+
+  @Test
+  public void testSeeksWithLruEviction() throws Throwable {
+    IOStatistics ioStats;
+    openFS();
+
+    ExecutorService executorService = Executors.newFixedThreadPool(5,
+        new ThreadFactoryBuilder()
+            .setDaemon(true)
+            .setNameFormat("testSeeksWithLruEviction-%d")
+            .build());
+    CountDownLatch countDownLatch = new CountDownLatch(7);
+
+    try (FSDataInputStream in = largeFileFS.open(largeFile)) {
+      ioStats = in.getIOStatistics();
+      // tests to add multiple blocks in the prefetch cache
+      // and let LRU eviction take place as more cache entries
+      // are added with multiple block reads.
+
+      executorService.submit(() -> {
+        byte[] buffer = new byte[blockSize];
+        // Don't read block 0 completely
+        try {
+          in.read(buffer, 0, blockSize - S_1K * 10);
+          countDownLatch.countDown();
+          return true;
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      });
+
+      executorService.submit(() -> {
+        byte[] buffer = new byte[blockSize];
+        // Seek to block 1 and don't read completely
+        try {
+          in.seek(blockSize);

Review Comment:
   use PositionedReadable to include it in the test coverage
   ```
   readFully(blockSize * 4L, buffer, 0, 2 * S_1K);
   ```



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import 
org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
+import static 
org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE;
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+
+/**
+ * Test the prefetching input stream with LRU cache eviction on 
S3ACachingInputStream.
+ */
+@RunWith(Parameterized.class)
+public class ITestS3APrefetchingLruEviction extends AbstractS3ACostTest {
+
+  private final String maxBlocks;
+
+  @Parameterized.Parameters(name = "max-blocks-{0}")
+  public static Collection<Object[]> params() {
+    return Arrays.asList(new Object[][]{
+        {"1"},
+        {"2"},
+        {"3"}
+    });
+  }
+
+  public ITestS3APrefetchingLruEviction(final String maxBlocks) {
+    super(true);
+    this.maxBlocks = maxBlocks;
+  }
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestS3APrefetchingLruEviction.class);
+
+  private static final int S_1K = 1024;
+  // Path for file which should have length > block size so 
S3ACachingInputStream is used
+  private Path largeFile;
+  private FileSystem largeFileFS;
+  private int blockSize;
+
+  private static final int TIMEOUT_MILLIS = 5000;
+  private static final int INTERVAL_MILLIS = 500;
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setBoolean(PREFETCH_ENABLED_KEY, true);
+    conf.setInt(PREFETCH_MAX_BLOCKS_COUNT, Integer.parseInt(maxBlocks));
+    return conf;
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    cleanupWithLogger(LOG, largeFileFS);
+    largeFileFS = null;
+  }
+
+  private void openFS() throws Exception {
+    Configuration conf = getConfiguration();
+    String largeFileUri = S3ATestUtils.getCSVTestFile(conf);
+
+    largeFile = new Path(largeFileUri);
+    blockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, 
PREFETCH_BLOCK_DEFAULT_SIZE);
+    largeFileFS = new S3AFileSystem();
+    largeFileFS.initialize(new URI(largeFileUri), getConfiguration());
+  }
+
+  @Test
+  public void testSeeksWithLruEviction() throws Throwable {
+    IOStatistics ioStats;
+    openFS();
+
+    ExecutorService executorService = Executors.newFixedThreadPool(5,
+        new ThreadFactoryBuilder()
+            .setDaemon(true)
+            .setNameFormat("testSeeksWithLruEviction-%d")
+            .build());
+    CountDownLatch countDownLatch = new CountDownLatch(7);
+
+    try (FSDataInputStream in = largeFileFS.open(largeFile)) {
+      ioStats = in.getIOStatistics();
+      // tests to add multiple blocks in the prefetch cache
+      // and let LRU eviction take place as more cache entries
+      // are added with multiple block reads.
+
+      executorService.submit(() -> {
+        byte[] buffer = new byte[blockSize];
+        // Don't read block 0 completely
+        try {
+          in.read(buffer, 0, blockSize - S_1K * 10);

Review Comment:
   use readFully(); always.





> S3A prefetch - Implement LRU cache for SingleFilePerBlockCache
> --------------------------------------------------------------
>
>                 Key: HADOOP-18291
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18291
>             Project: Hadoop Common
>          Issue Type: Sub-task
>    Affects Versions: 3.4.0
>            Reporter: Ahmar Suhail
>            Assignee: Viraj Jasani
>            Priority: Major
>              Labels: pull-request-available
>
> Currently there is no limit on the size of disk cache. This means we could 
> have a large number of files on files, especially for access patterns that 
> are very random and do not always read the block fully. 
>  
> eg:
> in.seek(5);
> in.read(); 
> in.seek(blockSize + 10) // block 0 gets saved to disk as it's not fully read
> in.read();
> in.seek(2 * blockSize + 10) // block 1 gets saved to disk
> .. and so on
>  
> The in memory cache is bounded, and by default has a limit of 72MB (9 
> blocks). When a block is fully read, and a seek is issued it's released 
> [here|https://github.com/apache/hadoop/blob/feature-HADOOP-18028-s3a-prefetch/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingInputStream.java#L109].
>  We can also delete the on disk file for the block here if it exists. 
>  
> Also maybe add an upper limit on disk space, and delete the file which stores 
> data of the block furthest from the current block (similar to the in memory 
> cache) when this limit is reached. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to