|
@@ -18,14 +18,33 @@
|
|
|
|
|
|
package org.apache.hadoop.fs.s3a.scale;
|
|
|
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.FileStatus;
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
+import org.apache.hadoop.fs.LocatedFileStatus;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.RemoteIterator;
|
|
|
+import org.apache.hadoop.fs.s3a.Constants;
|
|
|
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
|
|
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
|
|
|
import org.apache.hadoop.fs.s3a.Statistic;
|
|
|
+
|
|
|
import org.junit.Test;
|
|
|
+import org.assertj.core.api.Assertions;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.io.InputStream;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.List;
|
|
|
+import java.util.concurrent.Callable;
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
+import java.util.concurrent.Executors;
|
|
|
+
|
|
|
+import com.amazonaws.services.s3.model.ObjectMetadata;
|
|
|
+import com.amazonaws.services.s3.model.PutObjectRequest;
|
|
|
+import com.amazonaws.services.s3.model.PutObjectResult;
|
|
|
|
|
|
import static org.apache.hadoop.fs.s3a.Statistic.*;
|
|
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
|
@@ -137,6 +156,93 @@ public class ITestS3ADirectoryPerformance extends S3AScaleTestBase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testMultiPagesListingPerformanceAndCorrectness()
|
|
|
+ throws Throwable {
|
|
|
+ describe("Check performance and correctness for multi page listing " +
|
|
|
+ "using different listing api");
|
|
|
+ final Path dir = methodPath();
|
|
|
+ final int batchSize = 10;
|
|
|
+ final int numOfPutRequests = 1000;
|
|
|
+ final int eachFileProcessingTime = 10;
|
|
|
+ final int numOfPutThreads = 50;
|
|
|
+ final Configuration conf =
|
|
|
+ getConfigurationWithConfiguredBatchSize(batchSize);
|
|
|
+ final InputStream im = new InputStream() {
|
|
|
+ @Override
|
|
|
+ public int read() throws IOException {
|
|
|
+ return -1;
|
|
|
+ }
|
|
|
+ };
|
|
|
+ final List<String> originalListOfFiles = new ArrayList<>();
|
|
|
+ List<Callable<PutObjectResult>> putObjectRequests = new ArrayList<>();
|
|
|
+ ExecutorService executorService = Executors
|
|
|
+ .newFixedThreadPool(numOfPutThreads);
|
|
|
+
|
|
|
+ NanoTimer uploadTimer = new NanoTimer();
|
|
|
+ try(S3AFileSystem fs = (S3AFileSystem) FileSystem.get(dir.toUri(), conf)) {
|
|
|
+ fs.create(dir);
|
|
|
+ assume("Test is only for raw fs", !fs.hasMetadataStore());
|
|
|
+ for (int i=0; i<numOfPutRequests; i++) {
|
|
|
+ Path file = new Path(dir, String.format("file-%03d", i));
|
|
|
+ originalListOfFiles.add(file.toString());
|
|
|
+ ObjectMetadata om = fs.newObjectMetadata(0L);
|
|
|
+ PutObjectRequest put = new PutObjectRequest(fs.getBucket(),
|
|
|
+ fs.pathToKey(file),
|
|
|
+ im,
|
|
|
+ om);
|
|
|
+ putObjectRequests.add(() ->
|
|
|
+ fs.getWriteOperationHelper().putObject(put));
|
|
|
+ }
|
|
|
+ executorService.invokeAll(putObjectRequests);
|
|
|
+ uploadTimer.end("uploading %d files with a parallelism of %d",
|
|
|
+ numOfPutRequests, numOfPutThreads);
|
|
|
+
|
|
|
+ RemoteIterator<LocatedFileStatus> resIterator = fs.listFiles(dir, true);
|
|
|
+ List<String> listUsingListFiles = new ArrayList<>();
|
|
|
+ NanoTimer timeUsingListFiles = new NanoTimer();
|
|
|
+ while(resIterator.hasNext()) {
|
|
|
+ listUsingListFiles.add(resIterator.next().getPath().toString());
|
|
|
+ Thread.sleep(eachFileProcessingTime);
|
|
|
+ }
|
|
|
+ timeUsingListFiles.end("listing %d files using listFiles() api with " +
|
|
|
+ "batch size of %d including %dms of processing time" +
|
|
|
+ " for each file",
|
|
|
+ numOfPutRequests, batchSize, eachFileProcessingTime);
|
|
|
+
|
|
|
+ Assertions.assertThat(listUsingListFiles)
|
|
|
+ .describedAs("Listing results using listFiles() must" +
|
|
|
+ "match with original list of files")
|
|
|
+ .hasSameElementsAs(originalListOfFiles)
|
|
|
+ .hasSize(numOfPutRequests);
|
|
|
+ List<String> listUsingListStatus = new ArrayList<>();
|
|
|
+ NanoTimer timeUsingListStatus = new NanoTimer();
|
|
|
+ FileStatus[] fileStatuses = fs.listStatus(dir);
|
|
|
+ for(FileStatus fileStatus : fileStatuses) {
|
|
|
+ listUsingListStatus.add(fileStatus.getPath().toString());
|
|
|
+ Thread.sleep(eachFileProcessingTime);
|
|
|
+ }
|
|
|
+ timeUsingListStatus.end("listing %d files using listStatus() api with " +
|
|
|
+ "batch size of %d including %dms of processing time" +
|
|
|
+ " for each file",
|
|
|
+ numOfPutRequests, batchSize, eachFileProcessingTime);
|
|
|
+ Assertions.assertThat(listUsingListStatus)
|
|
|
+ .describedAs("Listing results using listStatus() must" +
|
|
|
+ "match with original list of files")
|
|
|
+ .hasSameElementsAs(originalListOfFiles)
|
|
|
+ .hasSize(numOfPutRequests);
|
|
|
+ } finally {
|
|
|
+ executorService.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private Configuration getConfigurationWithConfiguredBatchSize(int batchSize) {
|
|
|
+ Configuration conf = new Configuration(getFileSystem().getConf());
|
|
|
+ S3ATestUtils.disableFilesystemCaching(conf);
|
|
|
+ conf.setInt(Constants.MAX_PAGING_KEYS, batchSize);
|
|
|
+ return conf;
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testTimeToStatEmptyDirectory() throws Throwable {
|
|
|
describe("Time to stat an empty directory");
|
|
@@ -188,5 +294,4 @@ public class ITestS3ADirectoryPerformance extends S3AScaleTestBase {
|
|
|
LOG.info("listObjects: {}", listRequests);
|
|
|
LOG.info("listObjects: per operation {}", listRequests.diff() / attempts);
|
|
|
}
|
|
|
-
|
|
|
}
|