From f09d6be8b952f60229a5a2155a5521448eb5f62e Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 8 Aug 2019 00:54:36 -0700 Subject: [PATCH 1/7] HDFS-8631: WebHDFS : Support setQuota --- .../java/org/apache/hadoop/fs/FileSystem.java | 28 ++++++ .../hadoop/hdfs/DistributedFileSystem.java | 2 + .../hadoop/hdfs/web/WebHdfsFileSystem.java | 43 ++++++++ .../web/resources/NameSpaceQuotaParam.java | 45 +++++++++ .../hadoop/hdfs/web/resources/PutOpParam.java | 3 + .../web/resources/StorageSpaceQuotaParam.java | 45 +++++++++ .../hdfs/web/resources/StorageTypeParam.java | 37 +++++++ .../hadoop/fs/http/server/FSOperations.java | 41 ++++++++ .../web/resources/NamenodeWebHdfsMethods.java | 44 +++++++-- .../hadoop-hdfs/src/site/markdown/WebHDFS.md | 64 ++++++++++++ .../apache/hadoop/hdfs/web/TestWebHDFS.java | 79 +++++++++++++++ .../hadoop/hdfs/web/resources/TestParam.java | 99 ++++++++++++------- 12 files changed, 488 insertions(+), 42 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/NameSpaceQuotaParam.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StorageSpaceQuotaParam.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StorageTypeParam.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index 3cbab1afb1ba1..09c9e94c9dacb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -1788,6 +1788,34 @@ public QuotaUsage getQuotaUsage(Path f) throws IOException { return getContentSummary(f); } + /** + * Set quota for the given {@link Path}. + * + * @param src the target path to set quota for + * @param namespaceQuota the namespace quota (i.e., # of files/directories) to set + * @param storagespaceQuota the storage space quota to set + * @throws IOException IO failure + */ + public void setQuota(Path src, final long namespaceQuota, + final long storagespaceQuota) throws IOException { + throw new UnsupportedOperationException(getClass().getCanonicalName() + + "does not support setQuota"); + } + + /** + * Set per storage type quota for the given {@link Path}. + * + * @param src the target path to set storage type quota for + * @param type the storage type to set + * @param quota the quota to set for the given storage type + * @throws IOException IO failure + */ + public void setQuotaByStorageType(Path src, final StorageType type, + final long quota) throws IOException { + throw new UnsupportedOperationException(getClass().getCanonicalName() + + "does not support setQuotaByStorageType"); + } + /** * The default filter accepts all paths. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 111b23b0ef1c0..fa75dbebd6ec3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -1000,6 +1000,7 @@ public QuotaUsage next(final FileSystem fs, final Path p) * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, * long, long, StorageType) */ + @Override public void setQuota(Path src, final long namespaceQuota, final long storagespaceQuota) throws IOException { statistics.incrementWriteOps(1); @@ -1029,6 +1030,7 @@ public Void next(final FileSystem fs, final Path p) * @param quota value of the specific storage type quota to be modified. * Maybe {@link HdfsConstants#QUOTA_RESET} to clear quota by storage type. */ + @Override public void setQuotaByStorageType(Path src, final StorageType type, final long quota) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java index 699eb4dfb40ab..01b1219606f52 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java @@ -77,6 +77,7 @@ import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider; import org.apache.hadoop.fs.QuotaUsage; import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.permission.FsCreateModes; import org.apache.hadoop.hdfs.DFSOpsCountStatistics; import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType; @@ -1913,6 +1914,48 @@ QuotaUsage decodeResponse(Map json) { }.run(); } + @Override + public void setQuota(Path p, final long namespaceQuota, + final long storagespaceQuota) throws IOException { + // sanity check + if ((namespaceQuota <= 0 && + namespaceQuota != HdfsConstants.QUOTA_RESET) || + (storagespaceQuota < 0 && + storagespaceQuota != HdfsConstants.QUOTA_RESET)) { + throw new IllegalArgumentException("Invalid values for quota : " + + namespaceQuota + " and " + storagespaceQuota); + } + + statistics.incrementWriteOps(1); + storageStatistics.incrementOpCounter(OpType.SET_QUOTA_USAGE); + + final HttpOpParam.Op op = PutOpParam.Op.SETQUOTA; + new FsPathRunner(op, p, new NameSpaceQuotaParam(namespaceQuota), + new StorageSpaceQuotaParam(storagespaceQuota)).run(); + } + + @Override + public void setQuotaByStorageType(Path path, StorageType type, long quota) + throws IOException { + if (quota <= 0 && quota != HdfsConstants.QUOTA_RESET) { + throw new IllegalArgumentException("Invalid values for quota :" + quota); + } + if (type == null) { + throw new IllegalArgumentException("Invalid storage type (null)"); + } + if (!type.supportTypeQuota()) { + throw new IllegalArgumentException( + "Quota for storage type '" + type.toString() + "' is not supported"); + } + + statistics.incrementWriteOps(1); + storageStatistics.incrementOpCounter(OpType.SET_QUOTA_BYTSTORAGEYPE); + + final HttpOpParam.Op op = PutOpParam.Op.SETQUOTABYSTORAGETYPE; + new FsPathRunner(op, path, new StorageTypeParam(type.name()), + new StorageSpaceQuotaParam(quota)).run(); + } + @Override public MD5MD5CRC32FileChecksum getFileChecksum(final Path p ) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/NameSpaceQuotaParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/NameSpaceQuotaParam.java new file mode 100644 index 0000000000000..0886c53e77ea0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/NameSpaceQuotaParam.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.web.resources; + +import org.apache.hadoop.hdfs.protocol.HdfsConstants; + +/** The name space quota parameter for directory. */ +public class NameSpaceQuotaParam extends LongParam { + /** Parameter name. */ + public static final String NAME = "namespacequota"; + /** Default parameter value ({@link Long#MAX_VALUE}). */ + // public static final String DEFAULT = String.valueOf(Long.MAX_VALUE); + public static final String DEFAULT = "9223372036854775807"; + + private static final Domain DOMAIN = new Domain(NAME); + + public NameSpaceQuotaParam(final Long value) { + super(DOMAIN, value, HdfsConstants.QUOTA_RESET, + HdfsConstants.QUOTA_DONT_SET); + } + + public NameSpaceQuotaParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java index e3d9294156e56..4fc00011ca1e6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java @@ -57,6 +57,9 @@ public enum Op implements HttpOpParam.Op { RENAMESNAPSHOT(false, HttpURLConnection.HTTP_OK), SETSTORAGEPOLICY(false, HttpURLConnection.HTTP_OK), + SETQUOTA(false, HttpURLConnection.HTTP_OK), + SETQUOTABYSTORAGETYPE(false, HttpURLConnection.HTTP_OK), + NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED); final boolean doOutputAndRedirect; diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StorageSpaceQuotaParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StorageSpaceQuotaParam.java new file mode 100644 index 0000000000000..65b18bdb90bff --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StorageSpaceQuotaParam.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.web.resources; + +import org.apache.hadoop.hdfs.protocol.HdfsConstants; + +/** The storage space quota parameter for directory. */ +public class StorageSpaceQuotaParam extends LongParam { + /** Parameter name. */ + public static final String NAME = "storagespacequota"; + /** Default parameter value ({@link Long#MAX_VALUE}). */ + public static final String DEFAULT = "9223372036854775807"; + + private static final Domain DOMAIN = new Domain(NAME); + + public StorageSpaceQuotaParam(final Long value) { + super(DOMAIN, value, HdfsConstants.QUOTA_RESET, + HdfsConstants.QUOTA_DONT_SET); + } + + public StorageSpaceQuotaParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} + diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StorageTypeParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StorageTypeParam.java new file mode 100644 index 0000000000000..42bf6559403cf --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StorageTypeParam.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.web.resources; + +/** storage type parameter. */ +public class StorageTypeParam extends StringParam { + /** Parameter name. */ + public static final String NAME = "storagetype"; + /** Default parameter value. */ + public static final String DEFAULT = ""; + + private static final Domain DOMAIN = new Domain(NAME, null); + + public StorageTypeParam(final String str) { + super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str); + } + + @Override + public String getName() { + return NAME; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java index 7ce8a2395cfe7..3869558477e03 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java @@ -547,6 +547,47 @@ public Map execute(FileSystem fs) throws IOException { } } + @InterfaceAudience.Private + public static class FSSetQuota + implements FileSystemAccess.FileSystemExecutor { + private final Path path; + private final long namespaceQuota; + private final long storagespaceQuota; + + public FSSetQuota(String path, long namespaceQuota, + long storagespaceQuota) { + this.path = new Path(path); + this.namespaceQuota = namespaceQuota; + this.storagespaceQuota = storagespaceQuota; + } + + @Override + public Void execute(FileSystem fs) throws IOException { + fs.setQuota(path, namespaceQuota, storagespaceQuota); + return null; + } + } + + @InterfaceAudience.Private + public static class FSSetQuotaByStorageType + implements FileSystemAccess.FileSystemExecutor { + private final Path path; + private final StorageType type; + private final long quota; + + public FSSetQuotaByStorageType(String path, StorageType type, long quota) { + this.path = new Path(path); + this.type = type; + this.quota = quota; + } + + @Override + public Void execute(FileSystem fs) throws IOException { + fs.setQuotaByStorageType(path, type, quota); + return null; + } + } + /** * Executor that performs a create FileSystemAccess files system operation. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java index b14c3a6fc5789..294ba28d3ee24 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java @@ -57,6 +57,8 @@ import javax.ws.rs.core.Response.Status; import org.apache.hadoop.fs.QuotaUsage; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -510,14 +512,21 @@ public Response putRoot( @QueryParam(StoragePolicyParam.NAME) @DefaultValue(StoragePolicyParam .DEFAULT) final StoragePolicyParam policyName, @QueryParam(ECPolicyParam.NAME) @DefaultValue(ECPolicyParam - .DEFAULT) final ECPolicyParam ecpolicy + .DEFAULT) final ECPolicyParam ecpolicy, + @QueryParam(NameSpaceQuotaParam.NAME) @DefaultValue(NameSpaceQuotaParam.DEFAULT) + final NameSpaceQuotaParam namespaceQuota, + @QueryParam(StorageSpaceQuotaParam.NAME) @DefaultValue(StorageSpaceQuotaParam.DEFAULT) + final StorageSpaceQuotaParam storagespaceQuota, + @QueryParam(StorageTypeParam.NAME) @DefaultValue(StorageTypeParam.DEFAULT) + final StorageTypeParam storageType ) throws IOException, InterruptedException { return put(ugi, delegation, username, doAsUser, ROOT, op, destination, owner, group, permission, unmaskedPermission, overwrite, bufferSize, replication, blockSize, modificationTime, accessTime, renameOptions, createParent, delegationTokenArgument, aclPermission, xattrName, xattrValue, xattrSetFlag, snapshotName, oldSnapshotName, - excludeDatanodes, createFlagParam, noredirect, policyName, ecpolicy); + excludeDatanodes, createFlagParam, noredirect, policyName, ecpolicy, + namespaceQuota, storagespaceQuota, storageType); } /** Validate all required params. */ @@ -599,15 +608,21 @@ public Response put( @QueryParam(StoragePolicyParam.NAME) @DefaultValue(StoragePolicyParam .DEFAULT) final StoragePolicyParam policyName, @QueryParam(ECPolicyParam.NAME) @DefaultValue(ECPolicyParam.DEFAULT) - final ECPolicyParam ecpolicy + final ECPolicyParam ecpolicy, + @QueryParam(NameSpaceQuotaParam.NAME) @DefaultValue(NameSpaceQuotaParam.DEFAULT) + final NameSpaceQuotaParam namespaceQuota, + @QueryParam(StorageSpaceQuotaParam.NAME) @DefaultValue(StorageSpaceQuotaParam.DEFAULT) + final StorageSpaceQuotaParam storagespaceQuota, + @QueryParam(StorageTypeParam.NAME) @DefaultValue(StorageTypeParam.DEFAULT) + final StorageTypeParam storageType ) throws IOException, InterruptedException { - init(ugi, delegation, username, doAsUser, path, op, destination, owner, group, permission, unmaskedPermission, overwrite, bufferSize, replication, blockSize, modificationTime, accessTime, renameOptions, delegationTokenArgument, aclPermission, xattrName, xattrValue, xattrSetFlag, snapshotName, oldSnapshotName, excludeDatanodes, - createFlagParam, noredirect, policyName); + createFlagParam, noredirect, policyName, ecpolicy, + namespaceQuota, storagespaceQuota, storageType); return doAs(ugi, new PrivilegedExceptionAction() { @Override @@ -619,7 +634,8 @@ public Response run() throws IOException, URISyntaxException { renameOptions, createParent, delegationTokenArgument, aclPermission, xattrName, xattrValue, xattrSetFlag, snapshotName, oldSnapshotName, excludeDatanodes, - createFlagParam, noredirect, policyName, ecpolicy); + createFlagParam, noredirect, policyName, ecpolicy, + namespaceQuota, storagespaceQuota, storageType); } }); } @@ -655,7 +671,10 @@ protected Response put( final CreateFlagParam createFlagParam, final NoRedirectParam noredirectParam, final StoragePolicyParam policyName, - final ECPolicyParam ecpolicy + final ECPolicyParam ecpolicy, + final NameSpaceQuotaParam namespaceQuota, + final StorageSpaceQuotaParam storagespaceQuota, + final StorageTypeParam storageType ) throws IOException, URISyntaxException { final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF); final ClientProtocol cp = getRpcClientProtocol(); @@ -832,6 +851,17 @@ protected Response put( validateOpParams(op, ecpolicy); cp.setErasureCodingPolicy(fullpath, ecpolicy.getValue()); return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build(); + case SETQUOTA: + validateOpParams(op, namespaceQuota, storagespaceQuota); + cp.setQuota(fullpath, namespaceQuota.getValue(), + storagespaceQuota.getValue(), null); + return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build(); + case SETQUOTABYSTORAGETYPE: + validateOpParams(op, storagespaceQuota, storageType); + cp.setQuota(fullpath, HdfsConstants.QUOTA_DONT_SET, + storagespaceQuota.getValue(), + StorageType.parseStorageType(storageType.getValue())); + return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build(); default: throw new UnsupportedOperationException(op + " is not supported"); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md index 8afb7af933aa7..9e1b160d6f91f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md @@ -832,6 +832,34 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getConten See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getQuotaUsage +### Set Quota + +* Submit a HTTP PUT request. + + curl -i -X PUT "http://:/webhdfs/v1/?op=SETQUOTA + &namespacequota=[&storagespacequota=]" + + The client receives a response with zero content length: + + HTTP/1.1 200 OK + Content-Length: 0 + +See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).setQuota + +### Set Quota By Storage Type + +* Submit a HTTP PUT request. + + curl -i -X PUT "http://:/webhdfs/v1/?op=SETQUOTABYSTORAGETYPE + &storagetype=&storagespacequota=" + + The client receives a response with zero content length: + + HTTP/1.1 200 OK + Content-Length: 0 + +See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).setQuotaByStorageType + ### Get File Checksum * Submit a HTTP GET request. @@ -3179,6 +3207,42 @@ See also: [Authentication](#Authentication) See also: [Create and Write to a File](#Create_and_Write_to_a_File) +### Namespace Quota + +| Name | `namespacequota` | +|:---- |:---- | +| Description | Limit on the namespace usage, i.e., number of files/directories, under a directory. | +| Type | String | +| Default Value | Long.MAX_VALUE | +| Valid Values | \> 0. | +| Syntax | Any integer. | + +See also: [`SETQUOTA`](#Set_Quota) + +### Storage Space Quota + +| Name | `storagespacequota` | +|:---- |:---- | +| Description | Limit on storage space usage (in bytes, including replication) under a directory. | +| Type | String | +| Default Value | Long.MAX_VALUE | +| Valid Values | \> 0. | +| Syntax | Any integer. | + +See also: [`SETQUOTA`](#Set_Quota), [`SETQUOTABYSTORAGETYPE`](#Set_Quota_By_Storage_Type) + +### Storage Type + +| Name | `storagetype` | +|:---- |:---- | +| Description | Storage type of the specific storage type quota to be modified. | +| Type | String | +| Default Value | \ | +| Valid Values | Any valid storage type. | +| Syntax | Any string. | + +See also: [`SETQUOTABYSTORAGETYPE`](#Set_Quota_By_Storage_Type) + ### Storage Policy | Name | `storagepolicy` | diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java index 9d9c905043e75..4458217fe86fb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java @@ -1129,9 +1129,88 @@ public void testQuotaUsage() throws Exception { cluster.shutdown(); } } + } + + @Test + public void testSetQuota() throws Exception { + MiniDFSCluster cluster = null; + final Configuration conf = WebHdfsTestUtil.createConf(); + final Path path = new Path("/TestDir"); + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); + final WebHdfsFileSystem webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem( + conf, WebHdfsConstants.WEBHDFS_SCHEME); + final DistributedFileSystem dfs = cluster.getFileSystem(); + final long nsQuota = 100; + final long spaceQuota = 1024; + + webHdfs.mkdirs(path); + + webHdfs.setQuota(path, nsQuota, spaceQuota); + QuotaUsage quotaUsage = dfs.getQuotaUsage(path); + assertEquals(nsQuota, quotaUsage.getQuota()); + assertEquals(spaceQuota, quotaUsage.getSpaceQuota()); + + webHdfs.setQuota(path, + HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET); + quotaUsage = dfs.getQuotaUsage(path); + assertEquals(HdfsConstants.QUOTA_RESET, quotaUsage.getQuota()); + assertEquals(HdfsConstants.QUOTA_RESET, quotaUsage.getSpaceQuota()); + + webHdfs.setQuotaByStorageType(path, StorageType.DISK, spaceQuota); + webHdfs.setQuotaByStorageType(path, StorageType.ARCHIVE, spaceQuota); + webHdfs.setQuotaByStorageType(path, StorageType.SSD, spaceQuota); + quotaUsage = dfs.getQuotaUsage(path); + assertEquals(spaceQuota, quotaUsage.getTypeQuota(StorageType.DISK)); + assertEquals(spaceQuota, quotaUsage.getTypeQuota(StorageType.ARCHIVE)); + assertEquals(spaceQuota, quotaUsage.getTypeQuota(StorageType.SSD)); + + // Test invalid parameters + + try { + webHdfs.setQuota(path, -100, 100); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid values for quota")); + } + + try { + webHdfs.setQuota(path, 100, -100); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid values for quota")); + } + + try { + webHdfs.setQuotaByStorageType(path, StorageType.SSD, -100); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid values for quota")); + } + + try { + webHdfs.setQuotaByStorageType(path, null, 100); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid storage type")); + } + + try { + webHdfs.setQuotaByStorageType(path, StorageType.RAM_DISK, 100); + fail("Should have thrown exception"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("is not supported")); + } + + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } } + @Test public void testWebHdfsPread() throws Exception { final Configuration conf = WebHdfsTestUtil.createConf(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java index c9247dfa7ddc8..48e99eada3ae6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.web.resources; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -25,6 +26,7 @@ import java.util.EnumSet; import java.util.List; +import org.apache.hadoop.fs.StorageType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -49,7 +51,7 @@ public class TestParam { @Test public void testAccessTimeParam() { final AccessTimeParam p = new AccessTimeParam(AccessTimeParam.DEFAULT); - Assert.assertEquals(-1L, p.getValue().longValue()); + assertEquals(-1L, p.getValue().longValue()); new AccessTimeParam(-1L); @@ -64,8 +66,8 @@ public void testAccessTimeParam() { @Test public void testBlockSizeParam() { final BlockSizeParam p = new BlockSizeParam(BlockSizeParam.DEFAULT); - Assert.assertEquals(null, p.getValue()); - Assert.assertEquals( + assertEquals(null, p.getValue()); + assertEquals( conf.getLongBytes(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT), p.getValue(conf)); @@ -83,8 +85,8 @@ public void testBlockSizeParam() { @Test public void testBufferSizeParam() { final BufferSizeParam p = new BufferSizeParam(BufferSizeParam.DEFAULT); - Assert.assertEquals(null, p.getValue()); - Assert.assertEquals( + assertEquals(null, p.getValue()); + assertEquals( conf.getInt(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY, CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT), p.getValue(conf)); @@ -102,13 +104,13 @@ public void testBufferSizeParam() { @Test public void testDelegationParam() { final DelegationParam p = new DelegationParam(DelegationParam.DEFAULT); - Assert.assertEquals(null, p.getValue()); + assertEquals(null, p.getValue()); } @Test public void testDestinationParam() { final DestinationParam p = new DestinationParam(DestinationParam.DEFAULT); - Assert.assertEquals(null, p.getValue()); + assertEquals(null, p.getValue()); new DestinationParam("/abc"); @@ -123,13 +125,13 @@ public void testDestinationParam() { @Test public void testGroupParam() { final GroupParam p = new GroupParam(GroupParam.DEFAULT); - Assert.assertEquals(null, p.getValue()); + assertEquals(null, p.getValue()); } @Test public void testModificationTimeParam() { final ModificationTimeParam p = new ModificationTimeParam(ModificationTimeParam.DEFAULT); - Assert.assertEquals(-1L, p.getValue().longValue()); + assertEquals(-1L, p.getValue().longValue()); new ModificationTimeParam(-1L); @@ -144,7 +146,7 @@ public void testModificationTimeParam() { @Test public void testOverwriteParam() { final OverwriteParam p = new OverwriteParam(OverwriteParam.DEFAULT); - Assert.assertEquals(false, p.getValue()); + assertEquals(false, p.getValue()); new OverwriteParam("trUe"); @@ -159,14 +161,14 @@ public void testOverwriteParam() { @Test public void testOwnerParam() { final OwnerParam p = new OwnerParam(OwnerParam.DEFAULT); - Assert.assertEquals(null, p.getValue()); + assertEquals(null, p.getValue()); } @Test public void testPermissionParam() { final PermissionParam p = new PermissionParam(PermissionParam.DEFAULT); - Assert.assertEquals(new FsPermission((short)0755), p.getDirFsPermission()); - Assert.assertEquals(new FsPermission((short)0644), p.getFileFsPermission()); + assertEquals(new FsPermission((short)0755), p.getDirFsPermission()); + assertEquals(new FsPermission((short)0644), p.getFileFsPermission()); new PermissionParam("0"); @@ -204,7 +206,7 @@ public void testPermissionParam() { @Test public void testRecursiveParam() { final RecursiveParam p = new RecursiveParam(RecursiveParam.DEFAULT); - Assert.assertEquals(false, p.getValue()); + assertEquals(false, p.getValue()); new RecursiveParam("falSe"); @@ -219,14 +221,14 @@ public void testRecursiveParam() { @Test public void testRenewerParam() { final RenewerParam p = new RenewerParam(RenewerParam.DEFAULT); - Assert.assertEquals(null, p.getValue()); + assertEquals(null, p.getValue()); } @Test public void testReplicationParam() { final ReplicationParam p = new ReplicationParam(ReplicationParam.DEFAULT); - Assert.assertEquals(null, p.getValue()); - Assert.assertEquals( + assertEquals(null, p.getValue()); + assertEquals( (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, DFSConfigKeys.DFS_REPLICATION_DEFAULT), p.getValue(conf)); @@ -248,7 +250,7 @@ public void testToSortedStringEscapesURICharacters() { Param equalParam = new RenewerParam("renewer=equal"); final String expected = "&renewer=renewer%3Dequal&token=token%26ampersand"; final String actual = Param.toSortedString(sep, equalParam, ampParam); - Assert.assertEquals(expected, actual); + assertEquals(expected, actual); } @Test @@ -291,7 +293,7 @@ public void testConcatSourcesParam() { final String expected = StringUtils.join(",", Arrays.asList(sub)); final ConcatSourcesParam computed = new ConcatSourcesParam(paths); - Assert.assertEquals(expected, computed.getValue()); + assertEquals(expected, computed.getValue()); } } @@ -317,7 +319,7 @@ public void testAclPermissionParam() { List setAclList = AclEntry.parseAclSpec("user::rwx,group::r--,other::rwx,user:user1:rwx", true); - Assert.assertEquals(setAclList.toString(), p.getAclPermission(true) + assertEquals(setAclList.toString(), p.getAclPermission(true) .toString()); new AclPermissionParam("user::rw-,group::rwx,other::rw-,user:user1:rwx"); @@ -373,12 +375,12 @@ public void testUserGroupOkAfterAlteringAclPattern() { String numericUserSpec = "user:110201:rwx"; AclPermissionParam aclNumericUserParam = new AclPermissionParam(numericUserSpec); - Assert.assertEquals(numericUserSpec, aclNumericUserParam.getValue()); + assertEquals(numericUserSpec, aclNumericUserParam.getValue()); String oddGroupSpec = "group:foo@bar:rwx"; AclPermissionParam aclGroupWithDomainParam = new AclPermissionParam(oddGroupSpec); - Assert.assertEquals(oddGroupSpec, aclGroupWithDomainParam.getValue()); + assertEquals(oddGroupSpec, aclGroupWithDomainParam.getValue()); } finally { // Revert back to the default rules for remainder of tests @@ -390,7 +392,7 @@ public void testUserGroupOkAfterAlteringAclPattern() { @Test public void testXAttrNameParam() { final XAttrNameParam p = new XAttrNameParam("user.a1"); - Assert.assertEquals(p.getXAttrName(), "user.a1"); + assertEquals(p.getXAttrName(), "user.a1"); } @Test @@ -403,9 +405,9 @@ public void testXAttrValueParam() throws IOException { @Test public void testXAttrEncodingParam() { final XAttrEncodingParam p = new XAttrEncodingParam(XAttrCodec.BASE64); - Assert.assertEquals(p.getEncoding(), XAttrCodec.BASE64); + assertEquals(p.getEncoding(), XAttrCodec.BASE64); final XAttrEncodingParam p1 = new XAttrEncodingParam(p.getValueString()); - Assert.assertEquals(p1.getEncoding(), XAttrCodec.BASE64); + assertEquals(p1.getEncoding(), XAttrCodec.BASE64); } @Test @@ -413,9 +415,9 @@ public void testXAttrSetFlagParam() { EnumSet flag = EnumSet.of( XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE); final XAttrSetFlagParam p = new XAttrSetFlagParam(flag); - Assert.assertEquals(p.getFlag(), flag); + assertEquals(p.getFlag(), flag); final XAttrSetFlagParam p1 = new XAttrSetFlagParam(p.getValueString()); - Assert.assertEquals(p1.getFlag(), flag); + assertEquals(p1.getFlag(), flag); } @Test @@ -424,7 +426,7 @@ public void testRenameOptionSetParam() { Options.Rename.OVERWRITE, Options.Rename.NONE); final RenameOptionSetParam p1 = new RenameOptionSetParam( p.getValueString()); - Assert.assertEquals(p1.getValue(), EnumSet.of( + assertEquals(p1.getValue(), EnumSet.of( Options.Rename.OVERWRITE, Options.Rename.NONE)); } @@ -432,8 +434,8 @@ public void testRenameOptionSetParam() { public void testSnapshotNameParam() { final OldSnapshotNameParam s1 = new OldSnapshotNameParam("s1"); final SnapshotNameParam s2 = new SnapshotNameParam("s2"); - Assert.assertEquals("s1", s1.getValue()); - Assert.assertEquals("s2", s2.getValue()); + assertEquals("s1", s1.getValue()); + assertEquals("s2", s2.getValue()); } @Test @@ -494,23 +496,50 @@ public void testFsActionParam() { public void testStartAfterParam() throws Exception { String s = "/helloWorld"; StartAfterParam param = new StartAfterParam(s); - Assert.assertEquals(s, param.getValue()); + assertEquals(s, param.getValue()); } @Test public void testStoragePolicyParam() { StoragePolicyParam p = new StoragePolicyParam(StoragePolicyParam.DEFAULT); - Assert.assertEquals(null, p.getValue()); + assertEquals(null, p.getValue()); p = new StoragePolicyParam("COLD"); - Assert.assertEquals("COLD", p.getValue()); + assertEquals("COLD", p.getValue()); + } + + @Test + public void testNamespaceQuotaParam() { + NameSpaceQuotaParam p = + new NameSpaceQuotaParam(NameSpaceQuotaParam.DEFAULT); + assertEquals(Long.valueOf(NameSpaceQuotaParam.DEFAULT), p.getValue()); + p = new NameSpaceQuotaParam(100L); + assertEquals(new Long(100), p.getValue()); + } + + @Test + public void testStorageSpaceQuotaParam() { + StorageSpaceQuotaParam sp = new StorageSpaceQuotaParam( + StorageSpaceQuotaParam.DEFAULT); + assertEquals(Long.valueOf(StorageSpaceQuotaParam.DEFAULT), + sp.getValue()); + sp = new StorageSpaceQuotaParam(100L); + assertEquals(new Long(100), sp.getValue()); + } + + @Test + public void testStorageTypeParam() { + StorageTypeParam p = new StorageTypeParam(StorageTypeParam.DEFAULT); + assertNull(p.getValue()); + p = new StorageTypeParam(StorageType.DISK.name()); + assertEquals(StorageType.DISK.name(), p.getValue()); } @Test public void testECPolicyParam() { ECPolicyParam p = new ECPolicyParam(ECPolicyParam.DEFAULT); - Assert.assertEquals(null, p.getValue()); + assertEquals(null, p.getValue()); p = new ECPolicyParam("RS-6-3-1024k"); - Assert.assertEquals("RS-6-3-1024k", p.getValue()); + assertEquals("RS-6-3-1024k", p.getValue()); } @Test From cbf9f7ad906d91fc4bcf1ad4c9b89d7e22cd47ee Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 8 Aug 2019 22:28:09 -0700 Subject: [PATCH 2/7] Address CR & Fix RouterWebHdfsMethods --- .../web/resources/NameSpaceQuotaParam.java | 1 - .../router/RouterWebHdfsMethods.java | 10 +++- .../apache/hadoop/hdfs/web/TestWebHDFS.java | 47 +++++-------------- .../hadoop/hdfs/web/resources/TestParam.java | 2 +- 4 files changed, 22 insertions(+), 38 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/NameSpaceQuotaParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/NameSpaceQuotaParam.java index 0886c53e77ea0..d85022194b84c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/NameSpaceQuotaParam.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/NameSpaceQuotaParam.java @@ -24,7 +24,6 @@ public class NameSpaceQuotaParam extends LongParam { /** Parameter name. */ public static final String NAME = "namespacequota"; /** Default parameter value ({@link Long#MAX_VALUE}). */ - // public static final String DEFAULT = String.valueOf(Long.MAX_VALUE); public static final String DEFAULT = "9223372036854775807"; private static final Domain DOMAIN = new Domain(NAME); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java index 6bc6bcc17dcbc..9f0d06d7695cd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java @@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.web.resources.HttpOpParam; import org.apache.hadoop.hdfs.web.resources.LengthParam; import org.apache.hadoop.hdfs.web.resources.ModificationTimeParam; +import org.apache.hadoop.hdfs.web.resources.NameSpaceQuotaParam; import org.apache.hadoop.hdfs.web.resources.NewLengthParam; import org.apache.hadoop.hdfs.web.resources.NoRedirectParam; import org.apache.hadoop.hdfs.web.resources.OffsetParam; @@ -73,6 +74,8 @@ import org.apache.hadoop.hdfs.web.resources.SnapshotNameParam; import org.apache.hadoop.hdfs.web.resources.StartAfterParam; import org.apache.hadoop.hdfs.web.resources.StoragePolicyParam; +import org.apache.hadoop.hdfs.web.resources.StorageSpaceQuotaParam; +import org.apache.hadoop.hdfs.web.resources.StorageTypeParam; import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam; import org.apache.hadoop.hdfs.web.resources.TokenKindParam; import org.apache.hadoop.hdfs.web.resources.TokenServiceParam; @@ -209,7 +212,10 @@ protected Response put( final CreateFlagParam createFlagParam, final NoRedirectParam noredirectParam, final StoragePolicyParam policyName, - final ECPolicyParam ecpolicy + final ECPolicyParam ecpolicy, + final NameSpaceQuotaParam namespaceQuota, + final StorageSpaceQuotaParam storagespaceQuota, + final StorageTypeParam storageType ) throws IOException, URISyntaxException { switch(op.getValue()) { @@ -261,7 +267,7 @@ protected Response put( accessTime, renameOptions, createParent, delegationTokenArgument, aclPermission, xattrName, xattrValue, xattrSetFlag, snapshotName, oldSnapshotName, exclDatanodes, createFlagParam, noredirectParam, - policyName, ecpolicy); + policyName, ecpolicy, namespaceQuota, storagespaceQuota, storageType); } default: throw new UnsupportedOperationException(op + " is not supported"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java index 4458217fe86fb..b77c8cbc6d513 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java @@ -55,6 +55,7 @@ import com.google.common.collect.ImmutableList; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.QuotaUsage; +import org.apache.hadoop.test.LambdaTestUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -1168,40 +1169,18 @@ public void testSetQuota() throws Exception { // Test invalid parameters - try { - webHdfs.setQuota(path, -100, 100); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid values for quota")); - } - - try { - webHdfs.setQuota(path, 100, -100); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid values for quota")); - } - - try { - webHdfs.setQuotaByStorageType(path, StorageType.SSD, -100); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid values for quota")); - } - - try { - webHdfs.setQuotaByStorageType(path, null, 100); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid storage type")); - } - - try { - webHdfs.setQuotaByStorageType(path, StorageType.RAM_DISK, 100); - fail("Should have thrown exception"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("is not supported")); - } + LambdaTestUtils.intercept(IllegalArgumentException.class, + () -> webHdfs.setQuota(path, -100, 100)); + LambdaTestUtils.intercept(IllegalArgumentException.class, + () -> webHdfs.setQuota(path, 100, -100)); + LambdaTestUtils.intercept(IllegalArgumentException.class, + () -> webHdfs.setQuotaByStorageType(path, StorageType.SSD, -100)); + LambdaTestUtils.intercept(IllegalArgumentException.class, + () -> webHdfs.setQuotaByStorageType(path, null, 100)); + LambdaTestUtils.intercept(IllegalArgumentException.class, + () -> webHdfs.setQuotaByStorageType(path, StorageType.SSD, -100)); + LambdaTestUtils.intercept(IllegalArgumentException.class, + () -> webHdfs.setQuotaByStorageType(path, StorageType.RAM_DISK, 100)); } finally { if (cluster != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java index 48e99eada3ae6..8875ea55061df 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java @@ -523,7 +523,7 @@ public void testStorageSpaceQuotaParam() { assertEquals(Long.valueOf(StorageSpaceQuotaParam.DEFAULT), sp.getValue()); sp = new StorageSpaceQuotaParam(100L); - assertEquals(new Long(100), sp.getValue()); + assertEquals(100L, sp.getValue().longValue()); } @Test From 0b26ca4204be22d20345dba41887cb3fe1aad753 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 8 Aug 2019 22:32:24 -0700 Subject: [PATCH 3/7] Fix TestFilterFileSystem --- .../test/java/org/apache/hadoop/fs/TestFilterFileSystem.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java index e7f42ff2b5603..917fa590871cd 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java @@ -135,6 +135,8 @@ public Token[] addDelegationTokens(String renewer, Credentials creds) public Path fixRelativePart(Path p); public ContentSummary getContentSummary(Path f); public QuotaUsage getQuotaUsage(Path f); + public void setQuota(Path f, long namespaceQuota, long storagespaceQuota); + public void setQuotaByStorageType(Path f, StorageType type, long quota); StorageStatistics getStorageStatistics(); } From 7acc30f315f605e3b02dd349dc7662f7e134dca7 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 8 Aug 2019 22:33:39 -0700 Subject: [PATCH 4/7] Fix TestHarFileSystem --- .../src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java index 57798c2c8b982..b442ca61e103e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java @@ -118,6 +118,8 @@ public FSDataOutputStream create(Path f, FsPermission permission, public void processDeleteOnExit(); public ContentSummary getContentSummary(Path f); public QuotaUsage getQuotaUsage(Path f); + public void setQuota(Path f, long namespaceQuota, long storagespaceQuota); + public void setQuotaByStorageType(Path f, StorageType type, long quota); public FsStatus getStatus(); public FileStatus[] listStatus(Path f, PathFilter filter); public FileStatus[] listStatusBatch(Path f, byte[] token); From adcd2638e4842e9ccc176ee0ef8bc00014974cd9 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Fri, 9 Aug 2019 10:06:43 -0700 Subject: [PATCH 5/7] Use 100L instead of new Long(100) --- .../java/org/apache/hadoop/hdfs/web/resources/TestParam.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java index 8875ea55061df..963d7e1668227 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java @@ -513,7 +513,7 @@ public void testNamespaceQuotaParam() { new NameSpaceQuotaParam(NameSpaceQuotaParam.DEFAULT); assertEquals(Long.valueOf(NameSpaceQuotaParam.DEFAULT), p.getValue()); p = new NameSpaceQuotaParam(100L); - assertEquals(new Long(100), p.getValue()); + assertEquals(100L, p.getValue().longValue()); } @Test From 661f56c99e26198bc279f3eac4053e6a86b5ef7a Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Tue, 13 Aug 2019 10:23:21 -0700 Subject: [PATCH 6/7] Address comments - Added a helper method for unsupported methods in FileSystem - Revert unrelated changes in TestParams --- .../java/org/apache/hadoop/fs/FileSystem.java | 22 ++++-- .../hadoop/hdfs/web/resources/TestParam.java | 70 +++++++++---------- 2 files changed, 53 insertions(+), 39 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index 09c9e94c9dacb..0a721a8c0dc60 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -1798,8 +1798,7 @@ public QuotaUsage getQuotaUsage(Path f) throws IOException { */ public void setQuota(Path src, final long namespaceQuota, final long storagespaceQuota) throws IOException { - throw new UnsupportedOperationException(getClass().getCanonicalName() + - "does not support setQuota"); + methodNotSupported(); } /** @@ -1812,8 +1811,7 @@ public void setQuota(Path src, final long namespaceQuota, */ public void setQuotaByStorageType(Path src, final StorageType type, final long quota) throws IOException { - throw new UnsupportedOperationException(getClass().getCanonicalName() + - "does not support setQuotaByStorageType"); + methodNotSupported(); } /** @@ -4483,6 +4481,22 @@ protected CompletableFuture openFileWithOptions( return result; } + /** + * Helper method that throws an {@link UnsupportedOperationException} for the + * current {@link FileSystem} method being called. + */ + protected void methodNotSupported() { + // The order of the stacktrace elements look like this (from top to bottom): + // - java.lang.Thread.getStackTrace + // - org.apache.hadoop.fs.FileSystem.methodNotSupported + // - + // therefore, to find out the current method name, we use the element at + // index 2. + String name = Thread.currentThread().getStackTrace()[2].getMethodName(); + throw new UnsupportedOperationException(getClass().getCanonicalName() + + " does not support method " + name); + } + /** * Create instance of the standard {@link FSDataInputStreamBuilder} for the * given filesystem and path. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java index 963d7e1668227..95078a5beee3c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java @@ -51,7 +51,7 @@ public class TestParam { @Test public void testAccessTimeParam() { final AccessTimeParam p = new AccessTimeParam(AccessTimeParam.DEFAULT); - assertEquals(-1L, p.getValue().longValue()); + Assert.assertEquals(-1L, p.getValue().longValue()); new AccessTimeParam(-1L); @@ -66,8 +66,8 @@ public void testAccessTimeParam() { @Test public void testBlockSizeParam() { final BlockSizeParam p = new BlockSizeParam(BlockSizeParam.DEFAULT); - assertEquals(null, p.getValue()); - assertEquals( + Assert.assertEquals(null, p.getValue()); + Assert.assertEquals( conf.getLongBytes(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT), p.getValue(conf)); @@ -85,8 +85,8 @@ public void testBlockSizeParam() { @Test public void testBufferSizeParam() { final BufferSizeParam p = new BufferSizeParam(BufferSizeParam.DEFAULT); - assertEquals(null, p.getValue()); - assertEquals( + Assert.assertEquals(null, p.getValue()); + Assert.assertEquals( conf.getInt(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY, CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT), p.getValue(conf)); @@ -104,13 +104,13 @@ public void testBufferSizeParam() { @Test public void testDelegationParam() { final DelegationParam p = new DelegationParam(DelegationParam.DEFAULT); - assertEquals(null, p.getValue()); + Assert.assertEquals(null, p.getValue()); } @Test public void testDestinationParam() { final DestinationParam p = new DestinationParam(DestinationParam.DEFAULT); - assertEquals(null, p.getValue()); + Assert.assertEquals(null, p.getValue()); new DestinationParam("/abc"); @@ -125,13 +125,13 @@ public void testDestinationParam() { @Test public void testGroupParam() { final GroupParam p = new GroupParam(GroupParam.DEFAULT); - assertEquals(null, p.getValue()); + Assert.assertEquals(null, p.getValue()); } @Test public void testModificationTimeParam() { final ModificationTimeParam p = new ModificationTimeParam(ModificationTimeParam.DEFAULT); - assertEquals(-1L, p.getValue().longValue()); + Assert.assertEquals(-1L, p.getValue().longValue()); new ModificationTimeParam(-1L); @@ -146,7 +146,7 @@ public void testModificationTimeParam() { @Test public void testOverwriteParam() { final OverwriteParam p = new OverwriteParam(OverwriteParam.DEFAULT); - assertEquals(false, p.getValue()); + Assert.assertEquals(false, p.getValue()); new OverwriteParam("trUe"); @@ -161,14 +161,14 @@ public void testOverwriteParam() { @Test public void testOwnerParam() { final OwnerParam p = new OwnerParam(OwnerParam.DEFAULT); - assertEquals(null, p.getValue()); + Assert.assertEquals(null, p.getValue()); } @Test public void testPermissionParam() { final PermissionParam p = new PermissionParam(PermissionParam.DEFAULT); - assertEquals(new FsPermission((short)0755), p.getDirFsPermission()); - assertEquals(new FsPermission((short)0644), p.getFileFsPermission()); + Assert.assertEquals(new FsPermission((short)0755), p.getDirFsPermission()); + Assert.assertEquals(new FsPermission((short)0644), p.getFileFsPermission()); new PermissionParam("0"); @@ -206,7 +206,7 @@ public void testPermissionParam() { @Test public void testRecursiveParam() { final RecursiveParam p = new RecursiveParam(RecursiveParam.DEFAULT); - assertEquals(false, p.getValue()); + Assert.assertEquals(false, p.getValue()); new RecursiveParam("falSe"); @@ -221,14 +221,14 @@ public void testRecursiveParam() { @Test public void testRenewerParam() { final RenewerParam p = new RenewerParam(RenewerParam.DEFAULT); - assertEquals(null, p.getValue()); + Assert.assertEquals(null, p.getValue()); } @Test public void testReplicationParam() { final ReplicationParam p = new ReplicationParam(ReplicationParam.DEFAULT); - assertEquals(null, p.getValue()); - assertEquals( + Assert.assertEquals(null, p.getValue()); + Assert.assertEquals( (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, DFSConfigKeys.DFS_REPLICATION_DEFAULT), p.getValue(conf)); @@ -250,7 +250,7 @@ public void testToSortedStringEscapesURICharacters() { Param equalParam = new RenewerParam("renewer=equal"); final String expected = "&renewer=renewer%3Dequal&token=token%26ampersand"; final String actual = Param.toSortedString(sep, equalParam, ampParam); - assertEquals(expected, actual); + Assert.assertEquals(expected, actual); } @Test @@ -293,7 +293,7 @@ public void testConcatSourcesParam() { final String expected = StringUtils.join(",", Arrays.asList(sub)); final ConcatSourcesParam computed = new ConcatSourcesParam(paths); - assertEquals(expected, computed.getValue()); + Assert.assertEquals(expected, computed.getValue()); } } @@ -319,7 +319,7 @@ public void testAclPermissionParam() { List setAclList = AclEntry.parseAclSpec("user::rwx,group::r--,other::rwx,user:user1:rwx", true); - assertEquals(setAclList.toString(), p.getAclPermission(true) + Assert.assertEquals(setAclList.toString(), p.getAclPermission(true) .toString()); new AclPermissionParam("user::rw-,group::rwx,other::rw-,user:user1:rwx"); @@ -375,12 +375,12 @@ public void testUserGroupOkAfterAlteringAclPattern() { String numericUserSpec = "user:110201:rwx"; AclPermissionParam aclNumericUserParam = new AclPermissionParam(numericUserSpec); - assertEquals(numericUserSpec, aclNumericUserParam.getValue()); + Assert.assertEquals(numericUserSpec, aclNumericUserParam.getValue()); String oddGroupSpec = "group:foo@bar:rwx"; AclPermissionParam aclGroupWithDomainParam = new AclPermissionParam(oddGroupSpec); - assertEquals(oddGroupSpec, aclGroupWithDomainParam.getValue()); + Assert.assertEquals(oddGroupSpec, aclGroupWithDomainParam.getValue()); } finally { // Revert back to the default rules for remainder of tests @@ -392,7 +392,7 @@ public void testUserGroupOkAfterAlteringAclPattern() { @Test public void testXAttrNameParam() { final XAttrNameParam p = new XAttrNameParam("user.a1"); - assertEquals(p.getXAttrName(), "user.a1"); + Assert.assertEquals(p.getXAttrName(), "user.a1"); } @Test @@ -405,9 +405,9 @@ public void testXAttrValueParam() throws IOException { @Test public void testXAttrEncodingParam() { final XAttrEncodingParam p = new XAttrEncodingParam(XAttrCodec.BASE64); - assertEquals(p.getEncoding(), XAttrCodec.BASE64); + Assert.assertEquals(p.getEncoding(), XAttrCodec.BASE64); final XAttrEncodingParam p1 = new XAttrEncodingParam(p.getValueString()); - assertEquals(p1.getEncoding(), XAttrCodec.BASE64); + Assert.assertEquals(p1.getEncoding(), XAttrCodec.BASE64); } @Test @@ -415,9 +415,9 @@ public void testXAttrSetFlagParam() { EnumSet flag = EnumSet.of( XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE); final XAttrSetFlagParam p = new XAttrSetFlagParam(flag); - assertEquals(p.getFlag(), flag); + Assert.assertEquals(p.getFlag(), flag); final XAttrSetFlagParam p1 = new XAttrSetFlagParam(p.getValueString()); - assertEquals(p1.getFlag(), flag); + Assert.assertEquals(p1.getFlag(), flag); } @Test @@ -426,7 +426,7 @@ public void testRenameOptionSetParam() { Options.Rename.OVERWRITE, Options.Rename.NONE); final RenameOptionSetParam p1 = new RenameOptionSetParam( p.getValueString()); - assertEquals(p1.getValue(), EnumSet.of( + Assert.assertEquals(p1.getValue(), EnumSet.of( Options.Rename.OVERWRITE, Options.Rename.NONE)); } @@ -434,8 +434,8 @@ public void testRenameOptionSetParam() { public void testSnapshotNameParam() { final OldSnapshotNameParam s1 = new OldSnapshotNameParam("s1"); final SnapshotNameParam s2 = new SnapshotNameParam("s2"); - assertEquals("s1", s1.getValue()); - assertEquals("s2", s2.getValue()); + Assert.assertEquals("s1", s1.getValue()); + Assert.assertEquals("s2", s2.getValue()); } @Test @@ -496,15 +496,15 @@ public void testFsActionParam() { public void testStartAfterParam() throws Exception { String s = "/helloWorld"; StartAfterParam param = new StartAfterParam(s); - assertEquals(s, param.getValue()); + Assert.assertEquals(s, param.getValue()); } @Test public void testStoragePolicyParam() { StoragePolicyParam p = new StoragePolicyParam(StoragePolicyParam.DEFAULT); - assertEquals(null, p.getValue()); + Assert.assertEquals(null, p.getValue()); p = new StoragePolicyParam("COLD"); - assertEquals("COLD", p.getValue()); + Assert.assertEquals("COLD", p.getValue()); } @Test @@ -537,9 +537,9 @@ public void testStorageTypeParam() { @Test public void testECPolicyParam() { ECPolicyParam p = new ECPolicyParam(ECPolicyParam.DEFAULT); - assertEquals(null, p.getValue()); + Assert.assertEquals(null, p.getValue()); p = new ECPolicyParam("RS-6-3-1024k"); - assertEquals("RS-6-3-1024k", p.getValue()); + Assert.assertEquals("RS-6-3-1024k", p.getValue()); } @Test From dc8a24732853cc6959b9a9d77c1abbaf0dcb868e Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Tue, 13 Aug 2019 23:39:19 -0700 Subject: [PATCH 7/7] Dummy change to trigger Yetus again --- .../src/main/java/org/apache/hadoop/fs/FileSystem.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index 0a721a8c0dc60..eda8726a2c8a0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -4486,7 +4486,7 @@ protected CompletableFuture openFileWithOptions( * current {@link FileSystem} method being called. */ protected void methodNotSupported() { - // The order of the stacktrace elements look like this (from top to bottom): + // The order of the stacktrace elements looks like this (from top to bottom): // - java.lang.Thread.getStackTrace // - org.apache.hadoop.fs.FileSystem.methodNotSupported // -