Browse Source

HDFS-6677. Change INodeFile and FSImage to support storage policy ID.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-6584@1610525 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 10 years ago
parent
commit
6f41baa623

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -1,5 +1,10 @@
 Hadoop HDFS Change Log
 
+HDFS-6584: Archival Storage
+
+    HDFS-6677. Change INodeFile and FSImage to support storage policy ID.
+    (szetszwo)
+
 Trunk (Unreleased)
 
   INCOMPATIBLE CHANGES

+ 12 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -249,6 +249,12 @@ public class FSDirectory implements Closeable {
     skipQuotaCheck = true;
   }
 
+  private static INodeFile newINodeFile(long id, PermissionStatus permissions,
+      long mtime, long atime, short replication, long preferredBlockSize) {
+    return new INodeFile(id, null, permissions, mtime, atime,
+        BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize, (byte)0);
+  }
+
   /**
    * Add the given filename to the fs.
    * @throws FileAlreadyExistsException
@@ -263,9 +269,8 @@ public class FSDirectory implements Closeable {
       UnresolvedLinkException, SnapshotAccessControlException, AclException {
 
     long modTime = now();
-    INodeFile newNode = new INodeFile(namesystem.allocateNewInodeId(), null,
-        permissions, modTime, modTime, BlockInfo.EMPTY_ARRAY, replication,
-        preferredBlockSize);
+    INodeFile newNode = newINodeFile(namesystem.allocateNewInodeId(),
+        permissions, modTime, modTime, replication, preferredBlockSize);
     newNode.toUnderConstruction(clientName, clientMachine);
 
     boolean added = false;
@@ -301,14 +306,13 @@ public class FSDirectory implements Closeable {
     final INodeFile newNode;
     assert hasWriteLock();
     if (underConstruction) {
-      newNode = new INodeFile(id, null, permissions, modificationTime,
-          modificationTime, BlockInfo.EMPTY_ARRAY, replication,
-          preferredBlockSize);
+      newNode = newINodeFile(id, permissions, modificationTime,
+          modificationTime, replication, preferredBlockSize);
       newNode.toUnderConstruction(clientName, clientMachine);
 
     } else {
-      newNode = new INodeFile(id, null, permissions, modificationTime, atime,
-          BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize);
+      newNode = newINodeFile(id, permissions, modificationTime, atime,
+          replication, preferredBlockSize);
     }
 
     try {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -774,7 +774,7 @@ public class FSImageFormat {
         counter.increment();
       }
       final INodeFile file = new INodeFile(inodeId, localName, permissions,
-          modificationTime, atime, blocks, replication, blockSize);
+          modificationTime, atime, blocks, replication, blockSize, (byte)0);
       if (underConstruction) {
         file.toUnderConstruction(clientName, clientMachine);
       }
@@ -875,7 +875,7 @@ public class FSImageFormat {
       final long preferredBlockSize = in.readLong();
 
       return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
-          accessTime, replication, preferredBlockSize, null);
+          accessTime, replication, preferredBlockSize, (byte)0, null);
     }
 
     public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -283,7 +283,8 @@ public final class FSImageFormatPBINode {
 
       final INodeFile file = new INodeFile(n.getId(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
-          f.getAccessTime(), blocks, replication, f.getPreferredBlockSize());
+          f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
+          (byte)f.getStoragePolicyID());
 
       if (f.hasAcl()) {
         file.addAclFeature(new AclFeature(loadAclEntries(f.getAcl(),
@@ -391,7 +392,8 @@ public final class FSImageFormatPBINode {
           .setModificationTime(file.getModificationTime())
           .setPermission(buildPermissionStatus(file, state.getStringMap()))
           .setPreferredBlockSize(file.getPreferredBlockSize())
-          .setReplication(file.getFileReplication());
+          .setReplication(file.getFileReplication())
+          .setStoragePolicyID(file.getStoragePolicyID());
 
       AclFeature f = file.getAclFeature();
       if (f != null) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -147,7 +147,7 @@ public class FSImageSerialization {
     assert numLocs == 0 : "Unexpected block locations";
 
     INodeFile file = new INodeFile(inodeId, name, perm, modificationTime,
-        modificationTime, blocks, blockReplication, preferredBlockSize);
+        modificationTime, blocks, blockReplication, preferredBlockSize, (byte)0);
     file.toUnderConstruction(clientName, clientMachine);
     return file;
   }

+ 21 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -71,10 +71,14 @@ public class INodeFile extends INodeWithAdditionalFields
     return inode.asFile();
   }
 
-  /** Format: [16 bits for replication][48 bits for PreferredBlockSize] */
+  /** 
+   * Bit format:
+   * [4-bit storagePolicyID][12-bit replication][48-bit preferredBlockSize]
+   */
   static enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
-    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 16, 1);
+    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 1),
+    STORAGE_POLICY_ID(REPLICATION.BITS, 4, 0);
 
     private final LongBitFormat BITS;
 
@@ -90,10 +94,16 @@ public class INodeFile extends INodeWithAdditionalFields
       return PREFERRED_BLOCK_SIZE.BITS.retrieve(header);
     }
 
-    static long toLong(long preferredBlockSize, short replication) {
+    static byte getStoragePolicyID(long header) {
+      return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
+    }
+
+    static long toLong(long preferredBlockSize, short replication,
+        byte storagePolicyID) {
       long h = 0;
       h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h);
       h = REPLICATION.BITS.combine(replication, h);
+      h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h);
       return h;
     }
   }
@@ -104,9 +114,10 @@ public class INodeFile extends INodeWithAdditionalFields
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
       long atime, BlockInfo[] blklist, short replication,
-      long preferredBlockSize) {
+      long preferredBlockSize, byte storagePolicyID) {
     super(id, name, permissions, mtime, atime);
-    header = HeaderFormat.toLong(preferredBlockSize, replication);
+    header = HeaderFormat.toLong(preferredBlockSize, replication,
+        storagePolicyID);
     this.blocks = blklist;
   }
   
@@ -357,6 +368,11 @@ public class INodeFile extends INodeWithAdditionalFields
     return HeaderFormat.getPreferredBlockSize(header);
   }
 
+  @Override
+  public byte getStoragePolicyID() {
+    return HeaderFormat.getStoragePolicyID(header);
+  }
+
   @Override
   public long getHeaderLong() {
     return header;

+ 11 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java

@@ -33,6 +33,9 @@ public interface INodeFileAttributes extends INodeAttributes {
   /** @return preferred block size in bytes */
   public long getPreferredBlockSize();
   
+  /** @return the storage policy ID. */
+  public byte getStoragePolicyID();
+
   /** @return the header as a long. */
   public long getHeaderLong();
 
@@ -45,10 +48,11 @@ public interface INodeFileAttributes extends INodeAttributes {
 
     public SnapshotCopy(byte[] name, PermissionStatus permissions,
         AclFeature aclFeature, long modificationTime, long accessTime,
-        short replication, long preferredBlockSize, XAttrFeature xAttrsFeature) {
+        short replication, long preferredBlockSize, byte storagePolicyID,
+        XAttrFeature xAttrsFeature) {
       super(name, permissions, aclFeature, modificationTime, accessTime, 
           xAttrsFeature);
-      header = HeaderFormat.toLong(preferredBlockSize, replication);
+      header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID);
     }
 
     public SnapshotCopy(INodeFile file) {
@@ -66,6 +70,11 @@ public interface INodeFileAttributes extends INodeAttributes {
       return HeaderFormat.getPreferredBlockSize(header);
     }
 
+    @Override
+    public byte getStoragePolicyID() {
+      return HeaderFormat.getStoragePolicyID(header);
+    }
+
     @Override
     public long getHeaderLong() {
       return header;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -220,7 +220,8 @@ public class FSImageFormatPBSnapshot {
           copy = new INodeFileAttributes.SnapshotCopy(pbf.getName()
               .toByteArray(), permission, acl, fileInPb.getModificationTime(),
               fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
-              fileInPb.getPreferredBlockSize(), xAttrs);
+              fileInPb.getPreferredBlockSize(),
+              (byte)fileInPb.getStoragePolicyID(), xAttrs);
         }
 
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto

@@ -134,6 +134,7 @@ message INodeSection {
     optional FileUnderConstructionFeature fileUC = 7;
     optional AclFeatureProto acl = 8;
     optional XAttrFeatureProto xAttrs = 9;
+    optional uint32 storagePolicyID = 10;
   }
 
   message INodeDirectory {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java

@@ -82,7 +82,7 @@ public class CreateEditsLog {
       }
 
       final INodeFile inode = new INodeFile(inodeId.nextValue(), null,
-          p, 0L, 0L, blocks, replication, blockSize);
+          p, 0L, 0L, blocks, replication, blockSize, (byte)0);
       inode.toUnderConstruction("", "");
 
      // Append path to filename with information about blockIDs 
@@ -97,7 +97,7 @@ public class CreateEditsLog {
         editLog.logMkDir(currentDir, dirInode);
       }
       INodeFile fileUc = new INodeFile(inodeId.nextValue(), null,
-          p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
+          p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize, (byte)0);
       fileUc.toUnderConstruction("", "");
       editLog.logOpenFile(filePath, fileUc, false);
       editLog.logCloseFile(filePath, inode);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -194,7 +194,7 @@ public class TestEditLog {
 
       for (int i = 0; i < numTransactions; i++) {
         INodeFile inode = new INodeFile(namesystem.allocateNewInodeId(), null,
-            p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
+            p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize, (byte)0);
         inode.toUnderConstruction("", "");
 
         editLog.logOpenFile("/filename" + (startIndex + i), inode, false);

+ 22 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java

@@ -17,32 +17,41 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.fs.permission.AclEntryScope.*;
-import static org.apache.hadoop.fs.permission.AclEntryType.*;
-import static org.apache.hadoop.fs.permission.FsAction.*;
-import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.*;
-import static org.junit.Assert.*;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
+import static org.apache.hadoop.fs.permission.AclEntryType.GROUP;
+import static org.apache.hadoop.fs.permission.AclEntryType.MASK;
+import static org.apache.hadoop.fs.permission.AclEntryType.OTHER;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+import static org.apache.hadoop.fs.permission.FsAction.EXECUTE;
+import static org.apache.hadoop.fs.permission.FsAction.NONE;
+import static org.apache.hadoop.fs.permission.FsAction.READ;
+import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE;
+import static org.apache.hadoop.fs.permission.FsAction.READ_WRITE;
+import static org.apache.hadoop.fs.permission.FsAction.WRITE;
+import static org.apache.hadoop.fs.permission.FsAction.WRITE_EXECUTE;
+import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
-import org.junit.Before;
-import org.junit.Test;
-
 import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclEntryScope;
-import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Before;
+import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
-
-import static org.mockito.Mockito.*;
 /**
  * Unit tests covering FSPermissionChecker.  All tests in this suite have been
  * cross-validated against Linux setfacl/getfacl to check for consistency of the
@@ -423,7 +432,7 @@ public class TestFSPermissionChecker {
       FsPermission.createImmutable(perm));
     INodeFile inodeFile = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
       name.getBytes("UTF-8"), permStatus, 0L, 0L, null, REPLICATION,
-      PREFERRED_BLOCK_SIZE);
+      PREFERRED_BLOCK_SIZE, (byte)0);
     parent.addChild(inodeFile);
     return inodeFile;
   }

+ 30 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -74,15 +74,39 @@ public class TestINodeFile {
   static final short BLOCKBITS = 48;
   static final long BLKSIZE_MAXVALUE = ~(0xffffL << BLOCKBITS);
 
-  private final PermissionStatus perm = new PermissionStatus(
+  private static final PermissionStatus perm = new PermissionStatus(
       "userName", null, FsPermission.getDefault());
   private short replication;
   private long preferredBlockSize = 1024;
 
   INodeFile createINodeFile(short replication, long preferredBlockSize) {
     return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-        null, replication, preferredBlockSize);
+        null, replication, preferredBlockSize, (byte)0);
   }
+
+  private static INodeFile createINodeFile(byte storagePolicyID) {
+    return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+        null, (short)3, 1024L, storagePolicyID);
+  }
+
+  @Test
+  public void testStoragePolicyID () {
+    for(byte i = 0; i < 16; i++) {
+      final INodeFile f = createINodeFile(i);
+      assertEquals(i, f.getStoragePolicyID());
+    }
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testStoragePolicyIdBelowLowerBound () throws IllegalArgumentException {
+    createINodeFile((byte)-1);
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testStoragePolicyIdAboveUpperBound () throws IllegalArgumentException {
+    createINodeFile((byte)16);
+  }
+
   /**
    * Test for the Replication value. Sets a value and checks if it was set
    * correct.
@@ -258,7 +282,7 @@ public class TestINodeFile {
     INodeFile[] iNodes = new INodeFile[nCount];
     for (int i = 0; i < nCount; i++) {
       iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication,
-          preferredBlockSize);
+          preferredBlockSize, (byte)0);
       iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i));
       BlockInfo newblock = new BlockInfo(replication);
       iNodes[i].addBlock(newblock);
@@ -315,7 +339,8 @@ public class TestINodeFile {
 
     {//cast from INodeFileUnderConstruction
       final INode from = new INodeFile(
-          INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication, 1024L);
+          INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication,
+          1024L, (byte)0);
       from.asFile().toUnderConstruction("client", "machine");
     
       //cast to INodeFile, should success
@@ -1063,7 +1088,7 @@ public class TestINodeFile {
   public void testFileUnderConstruction() {
     replication = 3;
     final INodeFile file = new INodeFile(INodeId.GRANDFATHER_INODE_ID, null,
-        perm, 0L, 0L, null, replication, 1024L);
+        perm, 0L, 0L, null, replication, 1024L, (byte)0);
     assertFalse(file.isUnderConstruction());
 
     final String clientName = "client";