浏览代码

HDFS-6473. Protocol and API for Encryption Zones (clamb)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1600803 13f79535-47bb-0310-9956-ffa450edef68
Charles Lamb 11 年之前
父节点
当前提交
3f892e9d53

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-fs-encryption.txt

@@ -8,6 +8,8 @@ fs-encryption (Unreleased)
 
   IMPROVEMENTS
 
+    HDFS-6473. Protocol and API for Encryption Zones (clamb)
+
     HDFS-6476. Print out the KeyProvider after finding KP successfully on
     startup. (Juan Yu via wang)
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -294,6 +294,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>datatransfer.proto</include>
                   <include>fsimage.proto</include>
                   <include>hdfs.proto</include>
+                  <include>encryption.proto</include>
                 </includes>
               </source>
               <output>${project.build.directory}/generated-sources/java</output>

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -136,6 +136,7 @@ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -2775,6 +2776,34 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
     }
   }
   
+  public void createEncryptionZone(String src, String keyId)
+    throws IOException {
+    checkOpen();
+    try {
+      namenode.createEncryptionZone(src, keyId);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     SafeModeException.class,
+                                     UnresolvedPathException.class);
+    }
+  }
+
+  public void deleteEncryptionZone(String src) throws IOException {
+    checkOpen();
+    try {
+      namenode.deleteEncryptionZone(src);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     SafeModeException.class,
+                                     UnresolvedPathException.class);
+    }
+  }
+
+  public List<EncryptionZone> listEncryptionZones() throws IOException {
+    checkOpen();
+    return namenode.listEncryptionZones();
+  }
+
   public void setXAttr(String src, String name, byte[] value, 
       EnumSet<XAttrSetFlag> flag) throws IOException {
     checkOpen();

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
@@ -1795,6 +1796,22 @@ public class DistributedFileSystem extends FileSystem {
     }.resolve(this, absF);
   }
   
+  /* HDFS only */
+  public void createEncryptionZone(Path path, String keyId)
+    throws IOException {
+    dfs.createEncryptionZone(getPathName(path), keyId);
+  }
+
+  /* HDFS only */
+  public void deleteEncryptionZone(Path path) throws IOException {
+    dfs.deleteEncryptionZone(getPathName(path));
+  }
+
+  /* HDFS only */
+  public List<EncryptionZone> listEncryptionZones() throws IOException {
+    return dfs.listEncryptionZones();
+  }
+
   @Override
   public void setXAttr(Path path, final String name, final byte[] value, 
       final EnumSet<XAttrSetFlag> flag) throws IOException {

+ 70 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -17,9 +17,11 @@
  */
 package org.apache.hadoop.hdfs.client;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.EnumSet;
+import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -33,7 +35,9 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 
 /**
@@ -225,4 +229,70 @@ public class HdfsAdmin {
   public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
     return dfs.listCachePools();
   }
+
+  /**
+   * Create an encryption zone rooted at path using the optional encryption key
+   * id. An encryption zone is a portion of the HDFS file system hierarchy in
+   * which all files are encrypted with the same key, but possibly different
+   * key versions per file.
+   * <p/>
+   * Path must refer to an empty, existing directory. Otherwise an IOException
+   * will be thrown. keyId specifies the id of an encryption key in the
+   * KeyProvider that the Namenode has been configured to use. If keyId is
+   * null, then a key is generated in the KeyProvider using {@link
+   * java.util.UUID} to generate a key id.
+   *
+   * @param path The path of the root of the encryption zone.
+   *
+   * @param keyId An optional keyId in the KeyProvider. If null, then
+   * a key is generated.
+   *
+   * @throws IOException if there was a general IO exception
+   *
+   * @throws AccessControlException if the caller does not have access to path
+   *
+   * @throws FileNotFoundException if the path does not exist
+   */
+  public void createEncryptionZone(Path path, String keyId)
+    throws IOException, AccessControlException, FileNotFoundException {
+    dfs.createEncryptionZone(path, keyId);
+  }
+
+  /**
+   * Delete the encryption zone rooted at path. Path must refer to an existing,
+   * empty directory. Otherwise, an IOException is thrown. This method removes
+   * those extended attributes on the directory which indicate that it is part
+   * of an encryption zone. Following successful completion of this call, any
+   * new files created in the directory (or it's children) will not be
+   * encrypted. The directory is not removed by this method.
+   *
+   * @param path The path of the root of the encryption zone.
+   *
+   * @throws IOException if there was a general IO exception
+   *
+   * @throws AccessControlException if the caller does not have access to path
+   *
+   * @throws FileNotFoundException if the path does not exist
+   */
+  public void deleteEncryptionZone(Path path)
+    throws IOException, AccessControlException, FileNotFoundException {
+    dfs.deleteEncryptionZone(path);
+  }
+
+  /**
+   * Return a list of all {@EncryptionZone}s in the HDFS hierarchy which are
+   * visible to the caller. If the caller is the HDFS admin, then the returned
+   * EncryptionZone instances will have the key id field filled in. If the
+   * caller is not the HDFS admin, then the EncryptionZone instances will only
+   * have the path field filled in and only those zones that are visible to the
+   * user are returned.
+   *
+   * @throws IOException if there was a general IO exception
+   *
+   * @return List<EncryptionZone> the list of Encryption Zones that the caller has
+   * access to.
+   */
+  public List<EncryptionZone> listEncryptionZones() throws IOException {
+    return dfs.listEncryptionZones();
+  }
 }

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -1257,6 +1257,32 @@ public interface ClientProtocol {
   @Idempotent
   public AclStatus getAclStatus(String src) throws IOException;
   
+  /**
+   * Create an encryption zone
+   */
+  @AtMostOnce
+  public void createEncryptionZone(String src, String keyId)
+    throws IOException;
+
+  /**
+   * Delete an encryption zone
+   */
+  @AtMostOnce
+  public void deleteEncryptionZone(String src)
+    throws IOException;
+
+  /**
+   * Return a list of all {@EncryptionZone}s in the HDFS hierarchy which are
+   * visible to the caller. If the caller is the HDFS admin, then the returned
+   * EncryptionZone instances will have the key id field filled in. If the
+   * caller is not the HDFS admin, then the EncryptionZone instances will only
+   * have the path field filled in and only those zones that are visible to the
+   * user are returned.
+   */
+  @Idempotent
+  public List<EncryptionZone> listEncryptionZones()
+    throws IOException;
+
   /**
    * Set xattr of a file or directory.
    * A regular user only can set xattr of "user" namespace.

+ 79 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZone.java

@@ -0,0 +1,79 @@
+/**
+ * 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.protocol;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A simple class for representing an encryption zone. Presently an encryption
+ * zone only has a path (the root of the encryption zone) and a key id.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class EncryptionZone {
+
+  private final String path;
+  private final String keyId;
+
+  public EncryptionZone(String path, String keyId) {
+    this.path = path;
+    this.keyId = keyId;
+  }
+
+  public String getPath() {
+    return path;
+  }
+
+  public String getKeyId() {
+    return keyId;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(13, 31).
+      append(path).append(keyId).
+      toHashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    }
+    if (obj == this) {
+      return true;
+    }
+    if (obj.getClass() != getClass()) {
+      return false;
+    }
+
+    EncryptionZone rhs = (EncryptionZone) obj;
+    return new EqualsBuilder().
+      append(path, rhs.path).
+      append(keyId, rhs.keyId).
+      isEquals();
+  }
+
+  @Override
+  public String toString() {
+    return "EncryptionZone [path=" + path + ", keyId=" + keyId + "]";
+  }
+}

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -171,6 +171,12 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.DeleteEncryptionZoneResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.DeleteEncryptionZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
@@ -1275,6 +1281,41 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     }
   }
   
+  @Override
+  public CreateEncryptionZoneResponseProto createEncryptionZone(
+    RpcController controller, CreateEncryptionZoneRequestProto req)
+    throws ServiceException {
+    try {
+      server.createEncryptionZone(req.getSrc(), req.getKeyId());
+      return CreateEncryptionZoneResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public DeleteEncryptionZoneResponseProto deleteEncryptionZone(
+    RpcController controller, DeleteEncryptionZoneRequestProto req)
+    throws ServiceException {
+    try {
+      server.deleteEncryptionZone(req.getSrc());
+      return DeleteEncryptionZoneResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ListEncryptionZonesResponseProto listEncryptionZones(
+    RpcController controller, ListEncryptionZonesRequestProto req)
+    throws ServiceException {
+    try {
+      return PBHelper.convertListEZResponse(server.listEncryptionZones());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
   @Override
   public SetXAttrResponseProto setXAttr(RpcController controller,
       SetXAttrRequestProto req) throws ServiceException {

+ 40 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
@@ -143,6 +144,9 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSaf
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.DeleteEncryptionZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
@@ -1273,7 +1277,42 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-  
+
+  @Override
+  public void createEncryptionZone(String src, String keyId)
+    throws IOException {
+    final CreateEncryptionZoneRequestProto req =
+      CreateEncryptionZoneRequestProto.newBuilder().
+      setSrc(src).setKeyId(keyId).build();
+    try {
+      rpcProxy.createEncryptionZone(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void deleteEncryptionZone(String src) throws IOException {
+    final DeleteEncryptionZoneRequestProto req =
+      DeleteEncryptionZoneRequestProto.newBuilder().setSrc(src).build();
+    try {
+      rpcProxy.deleteEncryptionZone(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public List<EncryptionZone> listEncryptionZones() throws IOException {
+    final ListEncryptionZonesRequestProto req =
+      ListEncryptionZonesRequestProto.newBuilder().build();
+    try {
+      return PBHelper.convert(rpcProxy.listEncryptionZones(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   @Override
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FsAclPermission;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -110,6 +111,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.NNHAStatusHe
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -2180,6 +2183,45 @@ public class PBHelper {
     return xAttrs;
   }
   
+  public static List<EncryptionZone> convert(ListEncryptionZonesResponseProto a) {
+    final List<EncryptionZoneProto> ezs = a.getPathsAndKeysList();
+    return convertEZ(ezs);
+  }
+
+  public static ListEncryptionZonesResponseProto convertListEZResponse(
+    List<EncryptionZone> ezs) {
+    final ListEncryptionZonesResponseProto.Builder builder =
+      ListEncryptionZonesResponseProto.newBuilder();
+    builder.addAllPathsAndKeys(convertEZProto(ezs));
+    return builder.build();
+  }
+
+  public static List<EncryptionZoneProto> convertEZProto(
+      List<EncryptionZone> ezs) {
+    final ArrayList<EncryptionZoneProto> ret =
+      Lists.newArrayListWithCapacity(ezs.size());
+    for (EncryptionZone a : ezs) {
+      final EncryptionZoneProto.Builder builder =
+        EncryptionZoneProto.newBuilder();
+      builder.setPath(a.getPath());
+      builder.setKeyId(a.getKeyId());
+      ret.add(builder.build());
+    }
+    return ret;
+  }
+
+  public static List<EncryptionZone> convertEZ(
+    List<EncryptionZoneProto> ezs) {
+    final ArrayList<EncryptionZone> ret =
+      Lists.newArrayListWithCapacity(ezs.size());
+    for (EncryptionZoneProto a : ezs) {
+      final EncryptionZone ez =
+        new EncryptionZone(a.getPath(), a.getKeyId());
+      ret.add(ez);
+    }
+    return ret;
+  }
+
   public static List<XAttr> convert(GetXAttrsResponseProto a) {
     List<XAttrProto> xAttrs = a.getXAttrsList();
     return convertXAttrs(xAttrs);

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -155,6 +155,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -7824,6 +7825,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
   
+  void createEncryptionZone(final String src, final String keyId)
+          throws IOException {
+  }
+
+  void deleteEncryptionZone(final String src) throws IOException {
+  }
+
+  List<EncryptionZone> listEncryptionZones() throws IOException {
+    return null;
+  }
+
   /**
    * Set xattr for a file or directory.
    * 

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -1384,6 +1385,22 @@ class NameNodeRpcServer implements NamenodeProtocols {
     return namesystem.getAclStatus(src);
   }
   
+  @Override
+  public void createEncryptionZone(String src, String keyId)
+    throws IOException {
+    namesystem.createEncryptionZone(src, keyId);
+  }
+
+  @Override
+  public void deleteEncryptionZone(String src) throws IOException {
+    namesystem.deleteEncryptionZone(src);
+  }
+
+  @Override
+  public List<EncryptionZone> listEncryptionZones() throws IOException {
+    return namesystem.listEncryptionZones();
+  }
+
   @Override
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -32,6 +32,7 @@ import "Security.proto";
 import "hdfs.proto";
 import "acl.proto";
 import "xattr.proto";
+import "encryption.proto";
 
 /**
  * The ClientNamenodeProtocol Service defines the interface between a client 
@@ -766,4 +767,10 @@ service ClientNamenodeProtocol {
       returns(GetXAttrsResponseProto);
   rpc removeXAttr(RemoveXAttrRequestProto)
       returns(RemoveXAttrResponseProto);
+  rpc createEncryptionZone(CreateEncryptionZoneRequestProto)
+      returns(CreateEncryptionZoneResponseProto);
+  rpc deleteEncryptionZone(DeleteEncryptionZoneRequestProto)
+      returns(DeleteEncryptionZoneResponseProto);
+  rpc listEncryptionZones(ListEncryptionZonesRequestProto)
+      returns(ListEncryptionZonesResponseProto);
 }

+ 61 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto

@@ -0,0 +1,61 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "EncryptionZonesProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "hdfs.proto";
+
+message CreateEncryptionZoneRequestProto {
+  required string src = 1;
+  optional string keyId = 2;
+}
+
+message CreateEncryptionZoneResponseProto {
+}
+
+message DeleteEncryptionZoneRequestProto {
+  required string src = 1;
+}
+
+message DeleteEncryptionZoneResponseProto {
+}
+
+message ListEncryptionZonesRequestProto {
+}
+
+message EncryptionZoneProto {
+  required string path = 1;
+  required string keyId = 2;
+}
+
+message ListEncryptionZonesResponseProto {
+  repeated EncryptionZoneProto pathsAndKeys = 1;
+}