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/05/13 08:10:41 UTC

hadoop git commit: HDFS-8195. Erasure coding: Fix file quota change when we complete/commit the striped blocks. Contributed by Takuya Fukudome.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 64be3d5ba -> 2b11befc4


HDFS-8195. Erasure coding: Fix file quota change when we complete/commit the striped blocks. Contributed by Takuya Fukudome.


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

Branch: refs/heads/HDFS-7285
Commit: 2b11befc4edc2e64b6429f209449ee09d8485239
Parents: 64be3d5
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue May 12 23:10:25 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Tue May 12 23:10:25 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../hdfs/server/namenode/FSDirectory.java       |  29 +++--
 .../hdfs/server/namenode/FSNamesystem.java      |  25 +++-
 .../namenode/TestQuotaWithStripedBlocks.java    | 125 +++++++++++++++++++
 4 files changed, 167 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b11befc/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 0a2bb9e..0945d72 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -206,3 +206,6 @@
     handled properly (Rakesh R via zhz)
 
     HDFS-8363. Erasure Coding: DFSStripedInputStream#seekToNewSource. (yliu)
+
+    HDFS-8195. Erasure coding: Fix file quota change when we complete/commit 
+    the striped blocks. (Takuya Fukudome via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b11befc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 5cf103c..30ccd26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -503,23 +503,28 @@ public class FSDirectory implements Closeable {
     writeLock();
     try {
       final INodeFile fileINode = inodesInPath.getLastINode().asFile();
-      short numLocations = isStriped ?
-          HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS :
-          fileINode.getFileReplication();
       Preconditions.checkState(fileINode.isUnderConstruction());
 
-      // check quota limits and updated space consumed
-      // TODO add quota usage for EC files
-      updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
-          fileINode.getBlockReplication(), true);
-
       // associate new last block for the file
       final BlockInfo blockInfo;
       if (isStriped) {
-        blockInfo = new BlockInfoStripedUnderConstruction(block,
-            HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS,
-            BlockUCState.UNDER_CONSTRUCTION, targets);
+        ECSchema ecSchema = getECSchema(inodesInPath);
+        short numDataUnits = (short) ecSchema.getNumDataUnits();
+        short numParityUnits = (short) ecSchema.getNumParityUnits();
+        short numLocations = (short) (numDataUnits + numParityUnits);
+
+        // check quota limits and updated space consumed
+        updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
+            numLocations, true);
+
+        blockInfo = new BlockInfoStripedUnderConstruction(block, numDataUnits,
+            numParityUnits, BlockUCState.UNDER_CONSTRUCTION, targets);
       } else {
+        // check quota limits and updated space consumed
+        updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
+            fileINode.getBlockReplication(), true);
+
+        short numLocations = fileINode.getFileReplication();
         blockInfo = new BlockInfoContiguousUnderConstruction(block,
             numLocations, BlockUCState.UNDER_CONSTRUCTION, targets);
       }
@@ -687,7 +692,7 @@ public class FSDirectory implements Closeable {
     final INodeFile fileINode = iip.getLastINode().asFile();
     EnumCounters<StorageType> typeSpaceDeltas =
       getStorageTypeDeltas(fileINode.getStoragePolicyID(), ssDelta,
-          replication, replication);;
+          replication, replication);
     updateCount(iip, iip.length() - 1,
       new QuotaCounts.Builder().nameSpace(nsDelta).storageSpace(ssDelta * replication).
           typeSpaces(typeSpaceDeltas).build(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b11befc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index fb67263..191b566 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -4220,11 +4220,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // Adjust disk space consumption if required
-    // TODO: support EC files
-    final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();    
+    final long diff;
+    final short replicationFactor;
+    if (fileINode.isStriped()) {
+      final ECSchema ecSchema = dir.getECSchema(iip);
+      final short numDataUnits = (short) ecSchema.getNumDataUnits();
+      final short numParityUnits = (short) ecSchema.getNumParityUnits();
+
+      final long numBlocks = numDataUnits + numParityUnits;
+      final long fullBlockGroupSize =
+          fileINode.getPreferredBlockSize() * numBlocks;
+
+      final BlockInfoStriped striped = new BlockInfoStriped(commitBlock,
+          numDataUnits, numParityUnits);
+      final long actualBlockGroupSize = striped.spaceConsumed();
+
+      diff = fullBlockGroupSize - actualBlockGroupSize;
+      replicationFactor = (short) 1;
+    } else {
+      diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();
+      replicationFactor = fileINode.getFileReplication();
+    }
     if (diff > 0) {
       try {
-        dir.updateSpaceConsumed(iip, 0, -diff, fileINode.getFileReplication());
+        dir.updateSpaceConsumed(iip, 0, -diff, replicationFactor);
       } catch (IOException e) {
         LOG.warn("Unexpected exception while updating disk space.", e);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b11befc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
new file mode 100644
index 0000000..86fcb88
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
@@ -0,0 +1,125 @@
+/**
+ * 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.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Make sure we correctly update the quota usage with the striped blocks.
+ */
+public class TestQuotaWithStripedBlocks {
+  private static final int BLOCK_SIZE = 1024 * 1024;
+  private static final long DISK_QUOTA = BLOCK_SIZE * 10;
+  private static final ECSchema ecSchema =
+      ErasureCodingSchemaManager.getSystemDefaultSchema();
+  private static final int NUM_DATA_BLOCKS = ecSchema.getNumDataUnits();
+  private static final int NUM_PARITY_BLOCKS = ecSchema.getNumParityUnits();
+  private static final int GROUP_SIZE = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
+  private static final Path ecDir = new Path("/ec");
+
+  private MiniDFSCluster cluster;
+  private FSDirectory dir;
+  private DistributedFileSystem dfs;
+
+  @Before
+  public void setUp() throws IOException {
+    final Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE).build();
+    cluster.waitActive();
+
+    dir = cluster.getNamesystem().getFSDirectory();
+    dfs = cluster.getFileSystem();
+
+    dfs.mkdirs(ecDir);
+    dfs.getClient().createErasureCodingZone(ecDir.toString(), ecSchema);
+    dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA);
+    dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA);
+    dfs.setStoragePolicy(ecDir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testUpdatingQuotaCount() throws Exception {
+    final Path file = new Path(ecDir, "file");
+    FSDataOutputStream out = null;
+
+    try {
+      out = dfs.create(file, (short) 1);
+
+      INodeFile fileNode = dir.getINode4Write(file.toString()).asFile();
+      ExtendedBlock previous = null;
+      // Create striped blocks which have a cell in each block.
+      Block newBlock = DFSTestUtil.addStripedBlockToFile(cluster.getDataNodes(),
+          dfs, cluster.getNamesystem(), file.toString(), fileNode,
+          dfs.getClient().getClientName(), previous, 1);
+      previous = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(),
+          newBlock);
+
+      final INodeDirectory dirNode = dir.getINode4Write(ecDir.toString())
+          .asDirectory();
+      final long spaceUsed = dirNode.getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().getStorageSpace();
+      final long diskUsed = dirNode.getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
+      // When we add a new block we update the quota using the full block size.
+      Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, spaceUsed);
+      Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, diskUsed);
+
+      dfs.getClient().getNamenode().complete(file.toString(),
+          dfs.getClient().getClientName(), previous, fileNode.getId());
+
+      final long actualSpaceUsed = dirNode.getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().getStorageSpace();
+      final long actualDiskUsed = dirNode.getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
+      // In this case the file's real size is cell size * block group size.
+      Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
+          actualSpaceUsed);
+      Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
+          actualDiskUsed);
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+  }
+}