HADOOP-18372. ILoadTestS3ABulkDeleteThrottling failing. (#4642)

Contributed by Ahmar Suhail
This commit is contained in:
ahmarsuhail 2022-07-27 17:19:57 +01:00 committed by GitHub
parent 4c8cd61961
commit c92ff0b4f1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 14 additions and 14 deletions

View File

@ -45,6 +45,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.auth.delegation.Csvout;
@ -144,6 +145,18 @@ public ILoadTestS3ABulkDeleteThrottling(
@Override
protected Configuration createScaleConfiguration() {
Configuration conf = super.createScaleConfiguration();
S3ATestUtils.removeBaseAndBucketOverrides(conf,
EXPERIMENTAL_AWS_INTERNAL_THROTTLING,
BULK_DELETE_PAGE_SIZE,
USER_AGENT_PREFIX,
ENABLE_MULTI_DELETE);
conf.setBoolean(EXPERIMENTAL_AWS_INTERNAL_THROTTLING, throttle);
conf.setInt(BULK_DELETE_PAGE_SIZE, pageSize);
conf.set(USER_AGENT_PREFIX,
String.format("ILoadTestS3ABulkDeleteThrottling-%s-%04d",
throttle, pageSize));
S3ATestUtils.disableFilesystemCaching(conf);
return conf;
}
@ -151,19 +164,6 @@ protected Configuration createScaleConfiguration() {
@Override
public void setup() throws Exception {
final Configuration conf = getConf();
S3ATestUtils.removeBaseAndBucketOverrides(conf,
EXPERIMENTAL_AWS_INTERNAL_THROTTLING,
BULK_DELETE_PAGE_SIZE,
USER_AGENT_PREFIX);
conf.setBoolean(EXPERIMENTAL_AWS_INTERNAL_THROTTLING, throttle);
Assertions.assertThat(pageSize)
.describedAs("page size")
.isGreaterThan(0);
conf.setInt(BULK_DELETE_PAGE_SIZE, pageSize);
conf.set(USER_AGENT_PREFIX,
String.format("ILoadTestS3ABulkDeleteThrottling-%s-%04d",
throttle, pageSize));
super.setup();
Assume.assumeTrue("multipart delete disabled",
conf.getBoolean(ENABLE_MULTI_DELETE, true));
@ -246,7 +246,7 @@ private File deleteFiles(final int requestCount,
final ContractTestUtils.NanoTimer timer =
new ContractTestUtils.NanoTimer();
Exception ex = null;
try {
try (AuditSpan span = span()) {
fs.removeKeys(fileList, false);
} catch (IOException e) {
ex = e;