Ver Fonte

HADOOP-17422: ABFS: Set default ListMaxResults to max server limit (#2535)
Contributed by Sumangala Patki

TEST RESULTS:

namespace.enabled=true
auth.type=SharedKey
-------------------
$mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount=8 clean verify
Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
Tests run: 462, Failures: 0, Errors: 0, Skipped: 24
Tests run: 208, Failures: 0, Errors: 0, Skipped: 24

namespace.enabled=true
auth.type=OAuth
-------------------
$mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount=8 clean verify
Tests run: 90, Failures: 0, Errors: 0, Skipped: 0
Tests run: 462, Failures: 0, Errors: 0, Skipped: 70
Tests run: 208, Failures: 0, Errors: 0, Skipped: 141

Sumangala há 4 anos atrás
pai
commit
a35fc3871b

+ 1 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java

@@ -63,7 +63,7 @@ public final class FileSystemConfigurations {
   public static final int MAX_BUFFER_SIZE = 100 * ONE_MB;  // 100 MB
   public static final long MAX_AZURE_BLOCK_SIZE = 256 * 1024 * 1024L; // changing default abfs blocksize to 256MB
   public static final String AZURE_BLOCK_LOCATION_HOST_DEFAULT = "localhost";
-  public static final int DEFAULT_AZURE_LIST_MAX_RESULTS = 500;
+  public static final int DEFAULT_AZURE_LIST_MAX_RESULTS = 5000;
 
   public static final int MAX_CONCURRENT_READ_THREADS = 12;
   public static final int MAX_CONCURRENT_WRITE_THREADS = 8;

+ 1 - 1
hadoop-tools/hadoop-azure/src/site/markdown/abfs.md

@@ -848,7 +848,7 @@ Please refer the following links for further information.
 listStatus API fetches the FileStatus information from server in a page by page
 manner. The config `fs.azure.list.max.results` used to set the maxResults URI
  param which sets the pagesize(maximum results per call). The value should
- be >  0. By default this will be 500. Server has a maximum value for this
+ be >  0. By default this will be 5000. Server has a maximum value for this
  parameter as 5000. So even if the config is above 5000 the response will only
 contain 5000 entries. Please refer the following link for further information.
 https://docs.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/list

+ 7 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java

@@ -29,12 +29,15 @@ import java.util.concurrent.Future;
 
 import org.junit.Test;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_LIST_MAX_RESULTS;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
@@ -55,7 +58,10 @@ public class ITestAzureBlobFileSystemListStatus extends
 
   @Test
   public void testListPath() throws Exception {
-    final AzureBlobFileSystem fs = getFileSystem();
+    Configuration config = new Configuration(this.getRawConfiguration());
+    config.set(AZURE_LIST_MAX_RESULTS, "5000");
+    final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem
+        .newInstance(getFileSystem().getUri(), config);
     final List<Future<Void>> tasks = new ArrayList<>();
 
     ExecutorService es = Executors.newFixedThreadPool(10);