|
@@ -24,6 +24,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.TestDistributedFileSystem.checkOpStatistics;
|
|
|
+import static org.apache.hadoop.hdfs.TestDistributedFileSystem.checkStatistics;
|
|
|
+import static org.apache.hadoop.hdfs.TestDistributedFileSystem.getOpStatistics;
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
@@ -57,6 +60,7 @@ import java.util.Random;
|
|
|
import com.google.common.collect.ImmutableList;
|
|
|
import org.apache.commons.io.IOUtils;
|
|
|
import org.apache.hadoop.fs.QuotaUsage;
|
|
|
+import org.apache.hadoop.hdfs.DFSOpsCountStatistics;
|
|
|
import org.apache.hadoop.test.LambdaTestUtils;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -2012,6 +2016,62 @@ public class TestWebHDFS {
|
|
|
ecpolicyForECfile, ecPolicyName);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testStatistics() throws Exception {
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+ conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
|
|
|
+ StoragePolicySatisfierMode.EXTERNAL.toString());
|
|
|
+ StoragePolicySatisfier sps = new StoragePolicySatisfier(conf);
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).storageTypes(
|
|
|
+ new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE}})
|
|
|
+ .storagesPerDatanode(2).numDataNodes(1).build();
|
|
|
+ cluster.waitActive();
|
|
|
+ sps.init(new ExternalSPSContext(sps, DFSTestUtil
|
|
|
+ .getNameNodeConnector(conf, HdfsServerConstants.MOVER_ID_PATH, 1,
|
|
|
+ false)));
|
|
|
+ sps.start(StoragePolicySatisfierMode.EXTERNAL);
|
|
|
+ sps.start(StoragePolicySatisfierMode.EXTERNAL);
|
|
|
+ final WebHdfsFileSystem webHdfs = WebHdfsTestUtil
|
|
|
+ .getWebHdfsFileSystem(conf, WebHdfsConstants.WEBHDFS_SCHEME);
|
|
|
+ Path dir = new Path("/test");
|
|
|
+ webHdfs.mkdirs(dir);
|
|
|
+ int readOps = 0;
|
|
|
+ int writeOps = 0;
|
|
|
+ FileSystem.clearStatistics();
|
|
|
+
|
|
|
+ long opCount =
|
|
|
+ getOpStatistics(DFSOpsCountStatistics.OpType.GET_STORAGE_POLICY);
|
|
|
+ webHdfs.getStoragePolicy(dir);
|
|
|
+ checkStatistics(webHdfs, ++readOps, writeOps, 0);
|
|
|
+ checkOpStatistics(DFSOpsCountStatistics.OpType.GET_STORAGE_POLICY,
|
|
|
+ opCount + 1);
|
|
|
+
|
|
|
+ opCount =
|
|
|
+ getOpStatistics(DFSOpsCountStatistics.OpType.GET_STORAGE_POLICIES);
|
|
|
+ webHdfs.getAllStoragePolicies();
|
|
|
+ checkStatistics(webHdfs, ++readOps, writeOps, 0);
|
|
|
+ checkOpStatistics(DFSOpsCountStatistics.OpType.GET_STORAGE_POLICIES,
|
|
|
+ opCount + 1);
|
|
|
+
|
|
|
+ opCount =
|
|
|
+ getOpStatistics(DFSOpsCountStatistics.OpType.SATISFY_STORAGE_POLICY);
|
|
|
+ webHdfs.satisfyStoragePolicy(dir);
|
|
|
+ checkStatistics(webHdfs, readOps, ++writeOps, 0);
|
|
|
+ checkOpStatistics(DFSOpsCountStatistics.OpType.SATISFY_STORAGE_POLICY,
|
|
|
+ opCount + 1);
|
|
|
+
|
|
|
+ opCount = getOpStatistics(
|
|
|
+ DFSOpsCountStatistics.OpType.GET_SNAPSHOTTABLE_DIRECTORY_LIST);
|
|
|
+ webHdfs.getSnapshottableDirectoryList();
|
|
|
+ checkStatistics(webHdfs, ++readOps, writeOps, 0);
|
|
|
+ checkOpStatistics(
|
|
|
+ DFSOpsCountStatistics.OpType.GET_SNAPSHOTTABLE_DIRECTORY_LIST,
|
|
|
+ opCount + 1);
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
/**
|
|
|
* Get FileStatus JSONObject from ListStatus response.
|
|
|
*/
|