Jelajahi Sumber

HADOOP-19548: [ABFS] Fix Logging in FSDataInputStream to Mention Correct Buffersize (#7642)

Contributed by: Manika Joshi
manika137 3 hari lalu
induk
melakukan
f9f7b292eb

+ 1 - 1
hadoop-tools/hadoop-azure/dev-support/testrun-scripts/configsupport.sh

@@ -115,7 +115,7 @@ promptNamespaceType() {
     printf "* In the left-hand menu, select 'Overview' section and look for 'Properties'. \n"
     printf "* Under 'Blob service', check if 'Hierarchical namespace' is enabled or disabled. \n"
     echo "$contactTeamMsg"
-    select namespaceType in "HNS" "NonHNS"
+    select namespaceType in "HNS" "NonHNS (FNS)"
     do
         case $namespaceType in
             HNS)

+ 5 - 18
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.fs.azurebfs;
 
-import javax.annotation.Nullable;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -42,6 +41,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import javax.annotation.Nullable;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -118,22 +118,7 @@ import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_DEFAULT;
 import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_APPEND;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_CREATE;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_CREATE_NON_RECURSIVE;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_DELETE;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_EXIST;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_GET_DELEGATION_TOKEN;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_GET_FILE_STATUS;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_LIST_STATUS;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_MKDIRS;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_OPEN;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_RENAME;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.DIRECTORIES_CREATED;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.DIRECTORIES_DELETED;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.ERROR_IGNORED;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.FILES_CREATED;
-import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.FILES_DELETED;
+import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CPK_IN_NON_HNS_ACCOUNT_ERROR_MESSAGE;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType.DFS;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.DATA_BLOCKS_BUFFER;
@@ -365,8 +350,10 @@ public class AzureBlobFileSystem extends FileSystem
 
   @Override
   public FSDataInputStream open(final Path path, final int bufferSize) throws IOException {
-    LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize);
     // bufferSize is unused.
+    LOG.debug(
+        "AzureBlobFileSystem.open path: {} bufferSize as configured in 'fs.azure.read.request.size': {}",
+        path, abfsStore.getAbfsConfiguration().getReadBufferSize());
     return open(path, Optional.empty());
   }
 

+ 29 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java

@@ -20,14 +20,18 @@ package org.apache.hadoop.fs.azurebfs;
 
 import java.util.Hashtable;
 
+import org.assertj.core.api.Assertions;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+
 /**
  * Test FileSystemProperties.
  */
@@ -135,4 +139,29 @@ public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest {
 
     assertEquals(properties, fetchedProperties);
   }
+
+  @Test
+  //Test to verify buffersize remains the same as set in the configuration, irrespective of the parameter passed to FSDataInputStream
+  public void testBufferSizeSet() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    AbfsConfiguration abfsConfig = fs.getAbfsStore().getAbfsConfiguration();
+    int bufferSizeConfig = 6 * ONE_MB;
+    int bufferSizeArg = 10 * ONE_MB;
+
+    Path testPath = path(TEST_PATH);
+    fs.create(testPath);
+
+    abfsConfig.setReadBufferSize(bufferSizeConfig);
+    FSDataInputStream inputStream = fs.open(testPath, bufferSizeArg);
+    AbfsInputStream abfsInputStream
+        = (AbfsInputStream) inputStream.getWrappedStream();
+    int actualBufferSize = abfsInputStream.getBufferSize();
+
+    Assertions.assertThat(actualBufferSize)
+        .describedAs("Buffer size should be set to the value in the configuration")
+        .isEqualTo(bufferSizeConfig);
+    Assertions.assertThat(actualBufferSize)
+        .describedAs("Buffer size should not be set to the value passed as argument")
+        .isNotEqualTo(bufferSizeArg);
+  }
 }