Skip to content

Commit 2ddba7d

Browse files
committed
HADOOP-19654. create session now always seems to get called
...so adds cost to the assertion
1 parent e1b5c19 commit 2ddba7d

File tree

1 file changed

+15
-3
lines changed

1 file changed

+15
-3
lines changed

hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -22,10 +22,16 @@
2222

2323
import org.apache.hadoop.conf.Configuration;
2424
import org.apache.hadoop.fs.Path;
25+
import org.apache.hadoop.fs.s3a.S3AFileSystem;
26+
import org.apache.hadoop.fs.s3a.S3ATestUtils;
2527
import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
2628

2729
import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE;
30+
import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_PERFORMANCE_FLAGS;
31+
import static org.apache.hadoop.fs.s3a.Constants.S3EXPRESS_CREATE_SESSION;
32+
import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPerformanceFlags;
2833
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST;
34+
import static org.apache.hadoop.fs.s3a.impl.S3ExpressStorage.STORE_CAPABILITY_S3_EXPRESS_STORAGE;
2935

3036
/**
3137
* Verify that AWS SDK statistics are wired up.
@@ -35,7 +41,10 @@ public class ITestAWSStatisticCollection extends AbstractS3ACostTest {
3541
@Override
3642
public Configuration createConfiguration() {
3743
final Configuration conf = super.createConfiguration();
38-
conf.setBoolean(FS_S3A_CREATE_PERFORMANCE, true);
44+
S3ATestUtils.removeBaseAndBucketOverrides(conf,
45+
S3EXPRESS_CREATE_SESSION);
46+
setPerformanceFlags(conf, "create");
47+
conf.setBoolean(S3EXPRESS_CREATE_SESSION, false);
3948
return conf;
4049
}
4150

@@ -44,8 +53,11 @@ public void testSDKMetricsCostOfGetFileStatusOnFile() throws Throwable {
4453
describe("performing getFileStatus on a file");
4554
Path simpleFile = file(methodPath());
4655
// and repeat on the file looking at AWS wired up stats
47-
verifyMetrics(() -> getFileSystem().getFileStatus(simpleFile),
48-
with(STORE_IO_REQUEST, 1));
56+
final S3AFileSystem fs = getFileSystem();
57+
verifyMetrics(() -> fs.getFileStatus(simpleFile),
58+
with(STORE_IO_REQUEST,
59+
fs.hasPathCapability(new Path("/"), STORE_CAPABILITY_S3_EXPRESS_STORAGE)
60+
? 2 : 1));
4961
}
5062

5163
}

0 commit comments

Comments
 (0)