Skip to content

Commit f9f7b29

Browse files
authored
HADOOP-19548: [ABFS] Fix Logging in FSDataInputStream to Mention Correct Buffersize (#7642)
Contributed by: Manika Joshi
1 parent 7c7adef commit f9f7b29

File tree

3 files changed

+35
-19
lines changed

3 files changed

+35
-19
lines changed

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

+1-1
Original file line numberDiff line numberDiff line change
@@ -115,7 +115,7 @@ promptNamespaceType() {
115115
printf "* In the left-hand menu, select 'Overview' section and look for 'Properties'. \n"
116116
printf "* Under 'Blob service', check if 'Hierarchical namespace' is enabled or disabled. \n"
117117
echo "$contactTeamMsg"
118-
select namespaceType in "HNS" "NonHNS"
118+
select namespaceType in "HNS" "NonHNS (FNS)"
119119
do
120120
case $namespaceType in
121121
HNS)

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

+5-18
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818

1919
package org.apache.hadoop.fs.azurebfs;
2020

21-
import javax.annotation.Nullable;
2221
import java.io.File;
2322
import java.io.FileNotFoundException;
2423
import java.io.IOException;
@@ -42,6 +41,7 @@
4241
import java.util.concurrent.ExecutorService;
4342
import java.util.concurrent.Executors;
4443
import java.util.concurrent.Future;
44+
import javax.annotation.Nullable;
4545

4646
import org.slf4j.Logger;
4747
import org.slf4j.LoggerFactory;
@@ -118,22 +118,7 @@
118118
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL;
119119
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_DEFAULT;
120120
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS;
121-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_APPEND;
122-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_CREATE;
123-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_CREATE_NON_RECURSIVE;
124-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_DELETE;
125-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_EXIST;
126-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_GET_DELEGATION_TOKEN;
127-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_GET_FILE_STATUS;
128-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_LIST_STATUS;
129-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_MKDIRS;
130-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_OPEN;
131-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_RENAME;
132-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.DIRECTORIES_CREATED;
133-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.DIRECTORIES_DELETED;
134-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.ERROR_IGNORED;
135-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.FILES_CREATED;
136-
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.FILES_DELETED;
121+
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*;
137122
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CPK_IN_NON_HNS_ACCOUNT_ERROR_MESSAGE;
138123
import static org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType.DFS;
139124
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.DATA_BLOCKS_BUFFER;
@@ -365,8 +350,10 @@ public void registerListener(Listener listener1) {
365350

366351
@Override
367352
public FSDataInputStream open(final Path path, final int bufferSize) throws IOException {
368-
LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize);
369353
// bufferSize is unused.
354+
LOG.debug(
355+
"AzureBlobFileSystem.open path: {} bufferSize as configured in 'fs.azure.read.request.size': {}",
356+
path, abfsStore.getAbfsConfiguration().getReadBufferSize());
370357
return open(path, Optional.empty());
371358
}
372359

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

+29
Original file line numberDiff line numberDiff line change
@@ -20,14 +20,18 @@
2020

2121
import java.util.Hashtable;
2222

23+
import org.assertj.core.api.Assertions;
2324
import org.junit.Test;
2425

2526
import org.apache.hadoop.fs.FSDataInputStream;
2627
import org.apache.hadoop.fs.FSDataOutputStream;
2728
import org.apache.hadoop.fs.FileStatus;
2829
import org.apache.hadoop.fs.Path;
30+
import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
2931
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
3032

33+
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
34+
3135
/**
3236
* Test FileSystemProperties.
3337
*/
@@ -135,4 +139,29 @@ public void testSetFileSystemProperties() throws Exception {
135139

136140
assertEquals(properties, fetchedProperties);
137141
}
142+
143+
@Test
144+
//Test to verify buffersize remains the same as set in the configuration, irrespective of the parameter passed to FSDataInputStream
145+
public void testBufferSizeSet() throws Exception {
146+
final AzureBlobFileSystem fs = getFileSystem();
147+
AbfsConfiguration abfsConfig = fs.getAbfsStore().getAbfsConfiguration();
148+
int bufferSizeConfig = 6 * ONE_MB;
149+
int bufferSizeArg = 10 * ONE_MB;
150+
151+
Path testPath = path(TEST_PATH);
152+
fs.create(testPath);
153+
154+
abfsConfig.setReadBufferSize(bufferSizeConfig);
155+
FSDataInputStream inputStream = fs.open(testPath, bufferSizeArg);
156+
AbfsInputStream abfsInputStream
157+
= (AbfsInputStream) inputStream.getWrappedStream();
158+
int actualBufferSize = abfsInputStream.getBufferSize();
159+
160+
Assertions.assertThat(actualBufferSize)
161+
.describedAs("Buffer size should be set to the value in the configuration")
162+
.isEqualTo(bufferSizeConfig);
163+
Assertions.assertThat(actualBufferSize)
164+
.describedAs("Buffer size should not be set to the value passed as argument")
165+
.isNotEqualTo(bufferSizeArg);
166+
}
138167
}

0 commit comments

Comments
 (0)