[ 
https://issues.apache.org/jira/browse/HDFS-15548?focusedWorklogId=506486&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-506486
 ]

ASF GitHub Bot logged work on HDFS-15548:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 30/Oct/20 01:28
            Start Date: 30/Oct/20 01:28
    Worklog Time Spent: 10m 
      Work Description: Jing9 commented on a change in pull request #2288:
URL: https://github.com/apache/hadoop/pull/2288#discussion_r514666342



##########
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MountVolumeMap.java
##########
@@ -0,0 +1,143 @@
+/**
+ * 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.server.datanode.fsdataset.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
+
+import java.nio.channels.ClosedChannelException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * MountVolumeMap contains information of the relationship
+ * between underlying filesystem mount and datanode volumes.
+ *
+ * This is useful when configuring block tiering on same disk mount 
(HDFS-15548)
+ * For now,
+ * we don't configure multiple volumes with same storage type on a mount.
+ */
+@InterfaceAudience.Private
+class MountVolumeMap {
+  private ConcurrentMap<String, Map<StorageType, VolumeInfo>>
+      mountVolumeMapping;
+  private double reservedForArchive;
+
+  MountVolumeMap(Configuration conf) {
+    mountVolumeMapping = new ConcurrentHashMap<>();
+    reservedForArchive = conf.getDouble(
+        DFSConfigKeys.DFS_DATANODE_RESERVE_FOR_ARCHIVE_PERCENTAGE,
+        DFSConfigKeys.DFS_DATANODE_RESERVE_FOR_ARCHIVE_PERCENTAGE_DEFAULT);
+    if (reservedForArchive > 1) {
+      FsDatasetImpl.LOG.warn("Value of reserve-for-archival is > 100%." +
+          " Setting it to 100%.");
+      reservedForArchive = 1;
+    }
+  }
+
+  FsVolumeReference getVolumeRefByMountAndStorageType(String mount,
+      StorageType storageType) {
+    if (mountVolumeMapping != null
+        && mountVolumeMapping.containsKey(mount)) {
+      try {
+        VolumeInfo volumeInfo = mountVolumeMapping
+            .get(mount).getOrDefault(storageType, null);
+        if (volumeInfo != null) {
+          return volumeInfo.getFsVolume().obtainReference();
+        }
+      } catch (ClosedChannelException e) {
+        FsDatasetImpl.LOG.warn("Volume closed when getting volume" +
+            " by mount and storage type: "
+            + mount + ", " + storageType);
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Return configured capacity ratio. Otherwise return 1 as default
+   */
+  double getCapacityRatioByMountAndStorageType(String mount,
+      StorageType storageType) {
+    if (mountVolumeMapping != null
+        && mountVolumeMapping.containsKey(mount)) {
+      return mountVolumeMapping
+          .get(mount).getOrDefault(storageType, null).getCapacityRatio();
+    }
+    return 1;
+  }
+
+  void addVolume(FsVolumeImpl volume) {
+    String mount = volume.getMount();
+    if (!mount.isEmpty()) {
+      Map<StorageType, VolumeInfo> storageTypeMap =
+          mountVolumeMapping
+              .getOrDefault(mount, new ConcurrentHashMap<>());
+      if (storageTypeMap.containsKey(volume.getStorageType())) {
+        FsDatasetImpl.LOG.error("Found storage type already exist." +
+            " Skipping for now. Please check disk configuration");
+      } else {
+        VolumeInfo volumeInfo = new VolumeInfo(volume, 1);
+        if (volume.getStorageType() == StorageType.ARCHIVE) {
+          volumeInfo.setCapacityRatio(reservedForArchive);
+        } else if (volume.getStorageType() == StorageType.DISK) {
+          volumeInfo.setCapacityRatio(1 - reservedForArchive);

Review comment:
       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.

##########
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
##########
@@ -1503,6 +1503,20 @@
   public static final boolean DFS_PROTECTED_SUBDIRECTORIES_ENABLE_DEFAULT =
       false;
 
+  public static final String DFS_DATANODE_ALLOW_SAME_DISK_TIERING =
+      "dfs.datanode.same-disk-tiering.enabled";
+  public static final boolean DFS_DATANODE_ALLOW_SAME_DISK_TIERING_DEFAULT =
+      false;
+
+  // HDFS-15548 to allow DISK/ARCHIVE configured on the same disk mount.
+  // 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 =

Review comment:
       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?




----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 506486)
    Time Spent: 6h 40m  (was: 6.5h)

> Allow configuring DISK/ARCHIVE storage types on same device mount
> -----------------------------------------------------------------
>
>                 Key: HDFS-15548
>                 URL: https://issues.apache.org/jira/browse/HDFS-15548
>             Project: Hadoop HDFS
>          Issue Type: Sub-task
>          Components: datanode
>            Reporter: Leon Gao
>            Assignee: Leon Gao
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 6h 40m
>  Remaining Estimate: 0h
>
> We can allow configuring DISK/ARCHIVE storage types on the same device mount 
> on two separate directories.
> Users should be able to configure the capacity for each. Also, the datanode 
> usage report should report stats correctly.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to