[
https://issues.apache.org/jira/browse/HDFS-15683?focusedWorklogId=543817&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-543817
]
ASF GitHub Bot logged work on HDFS-15683:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 28/Jan/21 19:32
Start Date: 28/Jan/21 19:32
Worklog Time Spent: 10m
Work Description: Jing9 commented on a change in pull request #2625:
URL: https://github.com/apache/hadoop/pull/2625#discussion_r566339929
##########
File path:
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
##########
@@ -144,6 +148,35 @@ public static StorageLocation parse(String rawLocation)
return new StorageLocation(storageType, new Path(location).toUri());
}
+ public static Map<URI, Double> parseCapacityRatio(String capacityRatioConf)
Review comment:
It's better to have javadoc for this method as well
##########
File path:
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MountVolumeMap.java
##########
@@ -89,4 +90,18 @@ void removeVolume(FsVolumeImpl target) {
}
}
}
+
+ void setCapacityRatio(FsVolumeImpl target, double capacityRatio)
+ throws IOException {
+ String mount = target.getMount();
+ if (!mount.isEmpty()) {
+ MountVolumeInfo info = mountVolumeMapping.get(mount);
+ if (!info.setCapacityRatio(
+ target.getStorageType(), capacityRatio)) {
+ throw new IOException(
+ "Not enought capacity ratio left on mount: "
Review comment:
typo: enough. Also we may want to make the exception msg more detailed
##########
File path:
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MountVolumeInfo.java
##########
@@ -102,9 +116,28 @@ boolean addVolume(FsVolumeImpl volume) {
return true;
}
-
void removeVolume(FsVolumeImpl target) {
storageTypeVolumeMap.remove(target.getStorageType());
+ capacityRatioMap.remove(target.getStorageType());
+ }
+
+ /**
+ * Set customize capacity ratio for a storage type.
+ * Return false if the value is too big.
+ */
+ boolean setCapacityRatio(StorageType storageType,
+ double capacityRatio) {
+ double leftover = 1;
+ for (Map.Entry<StorageType, Double> e : capacityRatioMap.entrySet()) {
+ if (e.getKey() != storageType) {
+ leftover -= e.getValue();
+ }
+ }
+ if (leftover < capacityRatio) {
+ return false;
+ }
+ capacityRatioMap.put(storageType, capacityRatio);
Review comment:
Do we allow overwrite an existing capacity ratio of the storage type?
##########
File path:
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MountVolumeInfo.java
##########
@@ -35,10 +36,13 @@
class MountVolumeInfo {
private final ConcurrentMap<StorageType, FsVolumeImpl>
storageTypeVolumeMap;
+ private final ConcurrentMap<StorageType, Double>
Review comment:
shall we use EnumMap for both storageTypeVolumeMap and capacityRatioMap?
##########
File path:
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
##########
@@ -1557,6 +1557,12 @@
public static final double
DFS_DATANODE_RESERVE_FOR_ARCHIVE_DEFAULT_PERCENTAGE_DEFAULT = 0.0;
+ public static final String
Review comment:
Shall we include "disk-tiering" (or some other words) in the names for
these several configurations related to the disk-tiering feature?
##########
File path:
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
##########
@@ -59,6 +61,8 @@
* e.g. [Disk]/storages/storage1/
*/
private static final Pattern regex = Pattern.compile("^\\[(\\w*)\\](.+)$");
+ private static final Pattern CAPACITY_RATIO_REGEX =
Review comment:
Similar with regex, let's add a java doc for CAPACITY_RATIO_REGEX. We
can also consider rename "regex" to some more meaningful name.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 543817)
Time Spent: 1h (was: 50m)
> Allow configuring DISK/ARCHIVE capacity for individual volumes
> --------------------------------------------------------------
>
> Key: HDFS-15683
> URL: https://issues.apache.org/jira/browse/HDFS-15683
> Project: Hadoop HDFS
> Issue Type: Sub-task
> Components: datanode
> Reporter: Leon Gao
> Assignee: Leon Gao
> Priority: Major
> Labels: pull-request-available
> Time Spent: 1h
> Remaining Estimate: 0h
>
> This is a follow-up task for https://issues.apache.org/jira/browse/HDFS-15548
> In case that the datanode disks are not unified, we should allow admins to
> configure capacity for individual volumes on top of the default one.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]