You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zh...@apache.org on 2015/03/24 19:41:29 UTC

[41/50] [abbrv] hadoop git commit: HDFS-7826. Erasure Coding: Update INodeFile quota computation for striped blocks. Contributed by Kai Sasaki.

HDFS-7826. Erasure Coding: Update INodeFile quota computation for striped blocks. Contributed by Kai Sasaki.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/227cffd9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/227cffd9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/227cffd9

Branch: refs/heads/HDFS-7285
Commit: 227cffd986a3604c7d7948cab8e1e8889646be3b
Parents: 03ebc27
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 16 16:37:08 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Tue Mar 24 11:16:34 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  3 +
 .../blockmanagement/BlockInfoStriped.java       | 12 ++-
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 89 +++++++++++++++++---
 3 files changed, 90 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/227cffd9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index 245b630..07b72e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -186,4 +186,7 @@ public class HdfsConstants {
   public static final byte NUM_PARITY_BLOCKS = 2;
   public static final long BLOCK_GROUP_INDEX_MASK = 15;
   public static final byte MAX_BLOCKS_IN_GROUP = 16;
+
+  // The chunk size for striped block which is used by erasure coding
+  public static final int BLOCK_STRIPED_CHUNK_SIZE = 64 * 1024;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/227cffd9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 84c3be6..cef8318 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
 /**
@@ -34,6 +35,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
  * array to record the block index for each triplet.
  */
 public class BlockInfoStriped extends BlockInfo {
+  private final int   chunkSize = HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
   private final short dataBlockNum;
   private final short parityBlockNum;
   /**
@@ -56,7 +58,7 @@ public class BlockInfoStriped extends BlockInfo {
     this.setBlockCollection(b.getBlockCollection());
   }
 
-  short getTotalBlockNum() {
+  public short getTotalBlockNum() {
     return (short) (dataBlockNum + parityBlockNum);
   }
 
@@ -178,6 +180,14 @@ public class BlockInfoStriped extends BlockInfo {
     }
   }
 
+  public long spaceConsumed() {
+    // In case striped blocks, total usage by this striped blocks should
+    // be the total of data blocks and parity blocks because
+    // `getNumBytes` is the total of actual data block size.
+    return ((getNumBytes() - 1) / (dataBlockNum * chunkSize) + 1)
+        * chunkSize * parityBlockNum + getNumBytes();
+  }
+
   @Override
   public final boolean isStriped() {
     return true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/227cffd9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 8a6bb69..a8ab3ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
@@ -648,6 +649,9 @@ public class INodeFile extends INodeWithAdditionalFields
     long nsDelta = 1;
     final long ssDeltaNoReplication;
     short replication;
+    if (isWithStripedBlocks()) {
+      return computeQuotaUsageWithStriped(bsps, counts);
+    }
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
       FileDiffList fileDiffList = sf.getDiffs();
@@ -685,6 +689,23 @@ public class INodeFile extends INodeWithAdditionalFields
     return counts;
   }
 
+  /**
+   * Compute quota of striped file
+   * @param bsps
+   * @param counts
+   * @param useCache
+   * @param lastSnapshotId
+   * @return quota counts
+   */
+  public final QuotaCounts computeQuotaUsageWithStriped(
+      BlockStoragePolicySuite bsps, QuotaCounts counts) {
+    long nsDelta = 1;
+    final long ssDelta = storagespaceConsumed();
+    counts.addNameSpace(nsDelta);
+    counts.addStorageSpace(ssDelta);
+    return counts;
+  }
+
   @Override
   public final ContentSummaryComputationContext computeContentSummary(
       final ContentSummaryComputationContext summary) {
@@ -748,23 +769,37 @@ public class INodeFile extends INodeWithAdditionalFields
    * @return file size
    */
   public final long computeFileSize(boolean includesLastUcBlock,
-      boolean usePreferredBlockSize4LastUcBlock) {
-    if (blocks == null || blocks.length == 0) {
+                                    boolean usePreferredBlockSize4LastUcBlock) {
+    BlockInfo[] blockInfos = getBlocks();
+    // In case of contiguous blocks
+    if (blockInfos == null || blockInfos.length == 0) {
       return 0;
     }
-    final int last = blocks.length - 1;
+    final int last = blockInfos.length - 1;
     //check if the last block is BlockInfoUnderConstruction
-    long size = blocks[last].getNumBytes();
-    if (blocks[last] instanceof BlockInfoContiguousUnderConstruction) {
-       if (!includesLastUcBlock) {
-         size = 0;
-       } else if (usePreferredBlockSize4LastUcBlock) {
-         size = getPreferredBlockSize();
-       }
+    long size = blockInfos[last].getNumBytes();
+    if (blockInfos[last] instanceof BlockInfoContiguousUnderConstruction) {
+      if (!includesLastUcBlock) {
+        size = 0;
+      } else if (usePreferredBlockSize4LastUcBlock) {
+        size = getPreferredBlockSize();
+      }
+    } else if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
+      if (!includesLastUcBlock) {
+        size = 0;
+      } else if (usePreferredBlockSize4LastUcBlock) {
+        // Striped blocks keeps block group which counts
+        // (data blocks num + parity blocks num). When you
+        // count actual used size by BlockInfoStripedUC must
+        // be multiplied by these blocks number.
+        BlockInfoStripedUnderConstruction blockInfoStripedUC
+            = (BlockInfoStripedUnderConstruction) blockInfos[last];
+        size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum();
+      }
     }
     //sum other blocks
-    for(int i = 0; i < last; i++) {
-      size += blocks[i].getNumBytes();
+    for (int i = 0; i < last; i++) {
+      size += blockInfos[i].getNumBytes();
     }
     return size;
   }
@@ -775,7 +810,35 @@ public class INodeFile extends INodeWithAdditionalFields
    * Use preferred block size for the last block if it is under construction.
    */
   public final long storagespaceConsumed() {
-    return storagespaceConsumedNoReplication() * getBlockReplication();
+    if (isWithStripedBlocks()) {
+      return storagespaceConsumedWithStriped();
+    } else {
+      return storagespaceConsumedNoReplication() * getBlockReplication();
+    }
+  }
+
+  /**
+   * Compute size consumed by striped blocks.
+   * @return long
+   */
+  public final long storagespaceConsumedWithStriped() {
+    BlockInfo[] blockInfos = getBlocks();
+    long size = 0;
+    final int last = blockInfos.length - 1;
+    if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
+      BlockInfoStripedUnderConstruction blockInfoStripedUC
+          =(BlockInfoStripedUnderConstruction)blockInfos[last];
+      size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum();
+    } else {
+      // In case of last block is complete
+      BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[last];
+      size = blockInfoStriped.spaceConsumed();
+    }
+    for (int i = 0; i < last; i++) {
+      BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[i];
+      size += blockInfoStriped.spaceConsumed();
+    }
+    return size;
   }
 
   public final long storagespaceConsumedNoReplication() {