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

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

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


##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java:
##########
@@ -67,6 +70,10 @@ public class SingleFilePerBlockCache implements BlockCache {
 
   private final PrefetchingStatistics prefetchingStatistics;
 
+  // File attributes attached to any intermediate temporary file created 
during index creation.

Review Comment:
   use javadocs



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.File;
+import java.net.URI;
+
+import org.junit.Before;
+import org.junit.Test;
+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.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+
+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.io.IOUtils.cleanupWithLogger;
+
+/**
+ * Test the cache file behaviour with prefetching input stream.
+ */
+public class ITestS3APrefetchingCacheFiles extends AbstractS3ACostTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestS3APrefetchingCacheFiles.class);
+
+  private Path testFile;
+  private FileSystem fs;
+  private int prefetchBlockSize;
+
+  public ITestS3APrefetchingCacheFiles() {
+    super(true);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setup();
+    Configuration conf = getConfiguration();
+    String testFileUri = S3ATestUtils.getCSVTestFile(conf);
+
+    testFile = new Path(testFileUri);
+    prefetchBlockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, 
PREFETCH_BLOCK_DEFAULT_SIZE);
+    fs = new S3AFileSystem();
+    fs.initialize(new URI(testFileUri), getConfiguration());
+  }
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
+    conf.setBoolean(PREFETCH_ENABLED_KEY, true);
+    return conf;
+  }
+
+  @Override
+  public synchronized void teardown() throws Exception {
+    super.teardown();
+    File tmpFileDir = new File("target/build/test");

Review Comment:
   ok, here is why the tests are stamping on each other.
   if you look at `S3ATestUtils.prepareTestConfiguration()` you can see how it 
patches test configs with custom temp dirs. extract that from the fs used and 
delete it *only*



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java:
##########
@@ -125,13 +130,16 @@ public void testArgChecks() throws Exception {
   private static final class BlockManagerForTesting
       extends S3ACachingBlockManager {
 
+    private static final Configuration CONF = new Configuration();

Review Comment:
   use prepareTestConfiguration() to isolate the temp dir. this always sets 
BUFFER_DIR so no need for the checks on L142



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java:
##########
@@ -221,8 +234,10 @@ protected void writeFile(Path path, ByteBuffer buffer) 
throws IOException {
     writeChannel.close();
   }
 
-  protected Path getCacheFilePath() throws IOException {
-    return getTempFilePath();
+  protected Path getCacheFilePath(final Configuration conf,

Review Comment:
   javadocs



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java:
##########
@@ -110,11 +117,17 @@ public S3APrefetchingInputStream(
           streamStatistics);
     } else {
       LOG.debug("Creating in caching input stream for {}", context.getPath());
+      final String bufferDir =
+          conf.get(BUFFER_DIR) != null ? BUFFER_DIR : HADOOP_TMP_DIR;
+      final LocalDirAllocator localDirAllocator =

Review Comment:
   these are pretty expensive. s3afs creates one in directoryAllocator on 
demand...that code could be pulled out from createTmpFileForWrite() and also 
passed down here.



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java:
##########
@@ -63,24 +64,25 @@ public void testArgChecks() throws Exception {
     S3AInputStreamStatistics stats =
         readContext.getS3AStatisticsContext().newInputStreamStatistics();
 
+    Configuration conf = new Configuration();

Review Comment:
   same comments as above on prepareTestConfiguration()



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java:
##########
@@ -323,9 +338,10 @@ private String getStats() {
 
   private static final String CACHE_FILE_PREFIX = "fs-cache-";
 
-  public static boolean isCacheSpaceAvailable(long fileSize) {
+  public static boolean isCacheSpaceAvailable(long fileSize, Configuration 
conf,

Review Comment:
   javadocs



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.File;
+import java.net.URI;
+
+import org.junit.Before;
+import org.junit.Test;
+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.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+
+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.io.IOUtils.cleanupWithLogger;
+
+/**
+ * Test the cache file behaviour with prefetching input stream.
+ */
+public class ITestS3APrefetchingCacheFiles extends AbstractS3ACostTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestS3APrefetchingCacheFiles.class);
+
+  private Path testFile;
+  private FileSystem fs;
+  private int prefetchBlockSize;
+
+  public ITestS3APrefetchingCacheFiles() {
+    super(true);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setup();
+    Configuration conf = getConfiguration();
+    String testFileUri = S3ATestUtils.getCSVTestFile(conf);
+
+    testFile = new Path(testFileUri);
+    prefetchBlockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, 
PREFETCH_BLOCK_DEFAULT_SIZE);
+    fs = new S3AFileSystem();

Review Comment:
   superclass setup creates an fs already; no need for another one.





> SingleFilePerBlockCache to use LocalDirAllocator for file allocation
> --------------------------------------------------------------------
>
>                 Key: HADOOP-18399
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18399
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 3.4.0
>            Reporter: Steve Loughran
>            Assignee: Viraj Jasani
>            Priority: Major
>              Labels: pull-request-available
>
> prefetching stream's SingleFilePerBlockCache uses Files.tempFile() to 
> allocate a temp file.
> it should be using LocalDirAllocator to allocate space from a list of dirs, 
> taking a config key to use. for s3a we will use the Constants.BUFFER_DIR 
> option, which on yarn deployments is fixed under the env.LOCAL_DIR path, so 
> automatically cleaned up on container exit



--
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