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:09 UTC

[21/50] [abbrv] hadoop git commit: HDFS-7347. Configurable erasure coding policy for individual files and directories ( Contributed by Zhe Zhang )

HDFS-7347. Configurable erasure coding policy for individual files and directories ( Contributed by Zhe Zhang )


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

Branch: refs/heads/HDFS-7285
Commit: 5b810d03a6d67f1409b4f946884cb413ffee32d8
Parents: a16bfff
Author: Vinayakumar B <vi...@apache.org>
Authored: Thu Nov 6 10:03:26 2014 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Tue Mar 24 11:16:31 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  4 ++
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  2 +
 .../BlockStoragePolicySuite.java                |  5 ++
 .../hadoop/hdfs/TestBlockStoragePolicy.java     | 12 +++-
 .../TestBlockInitialEncoding.java               | 75 ++++++++++++++++++++
 5 files changed, 95 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b810d03/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
new file mode 100644
index 0000000..2ef8527
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -0,0 +1,4 @@
+  BREAKDOWN OF HDFS-7285 SUBTASKS AND RELATED JIRAS
+
+    HDFS-7347. Configurable erasure coding policy for individual files and
+    directories ( Zhe Zhang via vinayakumarb )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b810d03/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 7cf8a47..54c650b 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
@@ -171,6 +171,7 @@ public class HdfsConstants {
   public static final String ONESSD_STORAGE_POLICY_NAME = "ONE_SSD";
   public static final String HOT_STORAGE_POLICY_NAME = "HOT";
   public static final String WARM_STORAGE_POLICY_NAME = "WARM";
+  public static final String EC_STORAGE_POLICY_NAME = "EC";
   public static final String COLD_STORAGE_POLICY_NAME = "COLD";
 
   public static final byte MEMORY_STORAGE_POLICY_ID = 15;
@@ -178,5 +179,6 @@ public class HdfsConstants {
   public static final byte ONESSD_STORAGE_POLICY_ID = 10;
   public static final byte HOT_STORAGE_POLICY_ID = 7;
   public static final byte WARM_STORAGE_POLICY_ID = 5;
+  public static final byte EC_STORAGE_POLICY_ID = 4;
   public static final byte COLD_STORAGE_POLICY_ID = 2;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b810d03/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
index 020cb5f..3d121cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
@@ -78,6 +78,11 @@ public class BlockStoragePolicySuite {
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE});
+    final byte ecId = HdfsConstants.EC_STORAGE_POLICY_ID;
+    policies[ecId] = new BlockStoragePolicy(ecId,
+        HdfsConstants.EC_STORAGE_POLICY_NAME,
+        new StorageType[]{StorageType.DISK}, StorageType.EMPTY_ARRAY,
+        new StorageType[]{StorageType.ARCHIVE});
     final byte coldId = HdfsConstants.COLD_STORAGE_POLICY_ID;
     policies[coldId] = new BlockStoragePolicy(coldId,
         HdfsConstants.COLD_STORAGE_POLICY_NAME,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b810d03/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index 8f99a85..606a332 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -68,6 +68,7 @@ public class TestBlockStoragePolicy {
   static final short REPLICATION = 3;
 
   static final byte COLD = HdfsConstants.COLD_STORAGE_POLICY_ID;
+  static final byte EC = HdfsConstants.EC_STORAGE_POLICY_ID;
   static final byte WARM = HdfsConstants.WARM_STORAGE_POLICY_ID;
   static final byte HOT  = HdfsConstants.HOT_STORAGE_POLICY_ID;
   static final byte ONESSD  = HdfsConstants.ONESSD_STORAGE_POLICY_ID;
@@ -115,6 +116,9 @@ public class TestBlockStoragePolicy {
     expectedPolicyStrings.put(COLD,
         "BlockStoragePolicy{COLD:" + COLD + ", storageTypes=[ARCHIVE], " +
             "creationFallbacks=[], replicationFallbacks=[]}");
+    expectedPolicyStrings.put(EC,
+        "BlockStoragePolicy{EC:" + EC + ", storageTypes=[DISK], " +
+            "creationFallbacks=[], replicationFallbacks=[ARCHIVE]}");
     expectedPolicyStrings.put(WARM,
         "BlockStoragePolicy{WARM:" + WARM + ", storageTypes=[DISK, ARCHIVE], " +
             "creationFallbacks=[DISK, ARCHIVE], " +
@@ -1157,13 +1161,15 @@ public class TestBlockStoragePolicy {
     final DistributedFileSystem fs = cluster.getFileSystem();
     try {
       BlockStoragePolicy[] policies = fs.getStoragePolicies();
-      Assert.assertEquals(6, policies.length);
+      Assert.assertEquals(7, policies.length);
       Assert.assertEquals(POLICY_SUITE.getPolicy(COLD).toString(),
           policies[0].toString());
-      Assert.assertEquals(POLICY_SUITE.getPolicy(WARM).toString(),
+      Assert.assertEquals(POLICY_SUITE.getPolicy(EC).toString(),
           policies[1].toString());
-      Assert.assertEquals(POLICY_SUITE.getPolicy(HOT).toString(),
+      Assert.assertEquals(POLICY_SUITE.getPolicy(WARM).toString(),
           policies[2].toString());
+      Assert.assertEquals(POLICY_SUITE.getPolicy(HOT).toString(),
+          policies[3].toString());
     } finally {
       IOUtils.cleanup(null, fs);
       cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b810d03/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
new file mode 100644
index 0000000..a84f67b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
@@ -0,0 +1,75 @@
+/**
+ * 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.blockmanagement;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.*;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_ID;
+import static org.junit.Assert.assertEquals;
+
+public class TestBlockInitialEncoding {
+  private final int NUM_OF_DATANODES = 3;
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private static final int BLOCK_SIZE = 1024;
+  private HdfsAdmin dfsAdmin;
+  private FSNamesystem namesystem;
+
+  @Before
+  public void setupCluster() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    cluster = new MiniDFSCluster.Builder(conf).
+        numDataNodes(NUM_OF_DATANODES).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
+    namesystem = cluster.getNamesystem();
+  }
+
+  @After
+  public void shutdownCluster() throws IOException {
+    cluster.shutdown();
+  }
+
+  @Test
+  public void testBlockInitialEncoding()
+      throws IOException, InterruptedException {
+    final Path testDir = new Path("/test");
+    fs.mkdir(testDir, FsPermission.getDirDefault());
+    dfsAdmin.setStoragePolicy(testDir, EC_STORAGE_POLICY_NAME);
+    final Path ECFilePath = new Path("/test/foo.ec");
+    DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, (short) 3, 0);
+    INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString());
+    assertEquals(EC_STORAGE_POLICY_ID, inode.getStoragePolicyID());
+  }
+
+}
\ No newline at end of file