-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HDFS-15548. Allow configuring DISK/ARCHIVE storage types on same device mount #2288
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
Hexiaoqiao
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @LeonGao91 for your works, some comment inline.
I wonder that if someone could config more than one archive path at one device (for some reason or mis-config), then it may not work correct, right? Which works fine for logic disk in my opinion although it is not recommended. Thanks.
...p-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
Outdated
Show resolved
Hide resolved
| private final DataNodeVolumeMetrics metrics; | ||
| private URI baseURI; | ||
| private boolean enableSameDiskArchival; | ||
| private final String device; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what about using storageID replace device? IMO both of them are in order to index single volume, right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The "device" here is the string value of the filesystem mount point. I wanted to use it to keep track of which two volumes are on the same mount (thus the same disk). Datanode can use the existing DF#getMount() to detect it automatically.
I can probably change the name to "mount" to make it more clear.
| if (reservedForArchive >= 1) { | ||
| FsDatasetImpl.LOG.warn("Value of reserve-for-archival is >= 100% for " | ||
| + currentDir + ". Setting it to 99%."); | ||
| reservedForArchive = 0.99; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why reservedForArchive has to less than 1 here, IIUC it means that this is ARCHIVE device when reservedForArchive set to 1. Right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah I think you are right, I will update and make this at most 1.
Thanks for the review! @Hexiaoqiao I think this feature is mostly useful if users don't want to setup Linux level partitions to divide DISK/ARCHIVE, in which the size of partitions is difficult to change in production. For the question:
|
Thanks @LeonGao91 , this is indeed my concern, I think only log is not proper way, because the following logic will be not correct, especially the capacity remains if mis-config. IMO, DataNode instance exit probably more graceful. FYI, Thanks. |
That makes sense, I will make the change accordingly. |
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
…created on same mount
|
💔 -1 overall
This message was automatically generated. |
|
@Hexiaoqiao Would you please take a second look? I have added a check as we discussed with UT. |
|
Sorry, I missed this message before. Will have another review later today. Thanks. |
Hexiaoqiao
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @LeonGao91 for your works, some comment inline.
...p-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
Outdated
Show resolved
Hide resolved
| DFSConfigKeys.DFS_DATANODE_ALLOW_SAME_DISK_TIERING_DEFAULT); | ||
| if (enableSameDiskArchival) { | ||
| this.mount = usage.getMount(); | ||
| reservedForArchive = conf.getDouble( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
reservedForArchive try to define reserve for archive percentage. If there are heterogeneous disks located one node, do we need config them separate?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, it's a good point. The reason I put it this way is to make configuration less verbose for normal use cases that datanode only has one type of disk. Otherwise, users will need to tag all the disks which is less readable and easy to make mistakes.
I think we can introduce additional config for the use case you mentioned later, to list out each volume and target ratio.
| capacity = configuredCapacity; | ||
| } | ||
|
|
||
| if (enableSameDiskArchival) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The return value seems not expected as annotation says if enable this feature.
the capacity of the file system excluding space reserved for non-HDFS.
IMO, the part for ARCHIVE should also be calculated. It seems be not differentiated by NameNode for DISK or ARCHIVE per storage of DataNode. Please correct if something wrong.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is actually the important part to enable this feature, to allow users to configure the capacity of a fsVolume.
As we are configuring two fsVolume on the same underlying filesystem, if we do nothing the capacity will be calculated twice thus all the stats being reported will be incorrect.
Here is an example:
Let's say we want to configure [DISK]/data01/dfs and [ARCHIVE]/data01/dfs_archive on a 4TB disk mount /data01, and we want to assign 1 TB to [DISK]/data01/dfs and 3 TB for [ARCHIVE]/data01/dfs_archive, we can make reservedForArchive to be 0.75 and put those two dirs in the volume list.
In this case, /data01/dfs will be reported as a 1TB volume and /data01/dfs_archive will be reported as 3TB volume to HDFS. Logically, HDFS will just treat them as two separate volumes.
If we don't make the change here, HDFS will see two volumes and each of them is 4TB, in that case, the 4TB disk will be counted as 4 * 2 = 8TB capacity in namenode and all the related stats will be wrong.
Another change we need to make is the getActualNonDfsUsed() as below. Let's say in the above 4TB disk setup we use 0.1TB as reserved, and [ARCHIVE]/data01/dfs_archive already has 2TB capacity used, in this case when we are calculating the getActualNonDfsUsed() for [DISK]/data01/dfs it will always return 0, which is not correct and it will cause other weird issues. As the two fsVolumes are on the same filesystem, the reserved space should be shared.
According to our analysis and cluster testing result, updating these two functions getCapacity() and getActualNonDfsUsed() is enough to keep stats correct for the two "logical" fsVolumes on same disk.
I can update the java doc to reflect this when the feature is turned on.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @LeonGao91 for your detailed comments. It makes sense for me.
...p-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
Show resolved
Hide resolved
|
@Hexiaoqiao Thanks for the comments! I have replied and please let me know if it makes sense to you. |
|
+1 from my side. Ping @jojochuang @ayushtkn @goiri would you like to have another check? Thanks |
|
Thanks @Jing9 For the review!
|
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
| // Beware that capacity usage might be >100% if there are already | ||
| // data blocks exist and the configured ratio is small, which will | ||
| // prevent the volume from taking new blocks until capacity is balanced out. | ||
| public static final String DFS_DATANODE_RESERVE_FOR_ARCHIVE_PERCENTAGE = |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Take another look at the patch, I think it may be better to have the percentage as a tag added to the configuration "dfs.datanode.data.dir", just following the storage type tag. In this way on the same datanode we can have different percentage settings for different mount points. What do you think?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The intention is to have a configuration as a "default value" for all disks, as in normal cases one datanode server comes with the same type of HDDs. Therefore we can keep the DN configuration less verbose for most of the use cases.
However, you are right that we should allow users to configure different values, and it is a good idea to put it under "dfs.datanode.data.dir".
I will create a follow-up JIRA to address it, so we can keep this PR from being too big, as that could involve quite some change.
| if (volume.getStorageType() == StorageType.ARCHIVE) { | ||
| volumeInfo.setCapacityRatio(reservedForArchive); | ||
| } else if (volume.getStorageType() == StorageType.DISK) { | ||
| volumeInfo.setCapacityRatio(1 - reservedForArchive); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what if we have a mount with one single volume? Following the current implementation we may assign an unnecessary capacity ratio to it. We only need to calculate and assign the ratio for volumes sharing the same mount with others.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thats a good point. I will make the change to ignore the capacity ratio of the volume if there is only one on the mount.
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
Jing9
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for updating the patch, @LeonGao91 ! The current patch looks good to me except some minor issues. +1 after addressing the comments.
...dfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MountVolumeInfo.java
Show resolved
Hide resolved
...dfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MountVolumeInfo.java
Show resolved
Hide resolved
| */ | ||
| @InterfaceAudience.Private | ||
| class MountVolumeInfo { | ||
| private ConcurrentMap<StorageType, FsVolumeImpl> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This field can be declared as "final"
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
will do
...hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MountVolumeMap.java
Show resolved
Hide resolved
| this.enableSameDiskTiering = | ||
| conf.getBoolean(DFSConfigKeys.DFS_DATANODE_ALLOW_SAME_DISK_TIERING, | ||
| DFSConfigKeys.DFS_DATANODE_ALLOW_SAME_DISK_TIERING_DEFAULT); | ||
| if (enableSameDiskTiering) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
let's also add a check " && usage != null"
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Will do
| private final BlockScanner blockScanner; | ||
|
|
||
| private boolean enableSameDiskTiering; | ||
| private MountVolumeMap mountVolumeMap; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These two fields can be declared as "final"
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1, will fix
| void addVolume(FsVolumeReference ref) { | ||
| FsVolumeImpl volume = (FsVolumeImpl) ref.getVolume(); | ||
| volumes.add(volume); | ||
| if (enableSameDiskTiering && |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
maybe consider putting this check into a method
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good idea, will do that
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
|
Hi @Hexiaoqiao I have updated the PR with comments from Jing, please feel free to take a second look if you get a chance, thanks! |
|
Thanks @LeonGao91 and @Jing9 for your works, will confirm today later. Thanks. |
Hexiaoqiao
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @LeonGao91 and @Jing9 for your works here. The patch looks good to me in general. Just leave minor comments. +1 after addressing. Thanks.
|
|
||
| MountVolumeInfo(Configuration conf) { | ||
| storageTypeVolumeMap = new ConcurrentHashMap<>(); | ||
| reservedForArchiveDefault = conf.getDouble( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
check if it set a negative value?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1
| } | ||
| } | ||
|
|
||
|
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
codestyle: redundant empty line.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch, wonder if checksyle should catch it tho.
|
💔 -1 overall
This message was automatically generated. |
NOTICE
Please create an issue in ASF JIRA before opening a pull request,
and you need to set the title of the pull request which starts with
the corresponding JIRA issue number. (e.g. HADOOP-XXXXX. Fix a typo in YYY.)
For more details, please see https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute