Sfoglia il codice sorgente

HDFS-6389. Rename restrictions for encryption zones. (clamb)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1606253 13f79535-47bb-0310-9956-ffa450edef68
Charles Lamb 11 anni fa
parent
commit
31617733ac

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

@@ -26,6 +26,8 @@ fs-encryption (Unreleased)
     HDFS-6391. Get the Key/IV from the NameNode for encrypted files in
     DFSClient. (Charles Lamb and wang)
 
+    HDFS-6389. Rename restrictions for encryption zones. (clamb)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 34 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
 import org.apache.hadoop.hdfs.protocol.FsAclPermission;
@@ -544,6 +545,7 @@ public class FSDirectory implements Closeable {
       return false;
     }
     
+    checkEncryptionZoneMoveValidity(src, dst);
     // Ensure dst has quota to accommodate rename
     verifyFsLimitsForRename(srcIIP, dstIIP);
     verifyQuotaForRename(srcIIP.getINodes(), dstIIP.getINodes());
@@ -748,6 +750,7 @@ public class FSDirectory implements Closeable {
       throw new IOException(error);
     }
 
+    checkEncryptionZoneMoveValidity(src, dst);
     final INode dstInode = dstIIP.getLastINode();
     List<INodeDirectorySnapshottable> snapshottableDirs = 
         new ArrayList<INodeDirectorySnapshottable>();
@@ -971,6 +974,37 @@ public class FSDirectory implements Closeable {
     throw new IOException("rename from " + src + " to " + dst + " failed.");
   }
   
+  private void checkEncryptionZoneMoveValidity(String src, String dst)
+    throws IOException {
+    final EncryptionZone srcEZ =
+      getFSNamesystem().getEncryptionZoneForPath(src);
+    final EncryptionZone dstEZ =
+      getFSNamesystem().getEncryptionZoneForPath(dst);
+    final boolean srcInEZ = srcEZ != null;
+    final boolean dstInEZ = dstEZ != null;
+    if (srcInEZ) {
+      if (!dstInEZ) {
+        throw new IOException(src + " can't be moved from an encryption zone.");
+      }
+    } else {
+      if (dstInEZ) {
+        throw new IOException(src + " can't be moved into an encryption zone.");
+      }
+    }
+
+    if (srcInEZ || dstInEZ) {
+      if (!srcEZ.getPath().equals(dstEZ.getPath())) {
+        final StringBuilder sb = new StringBuilder(src);
+        sb.append(" can't be moved from encryption zone ");
+        sb.append(srcEZ.getPath());
+        sb.append(" to encryption zone ");
+        sb.append(dstEZ.getPath());
+        sb.append(".");
+        throw new IOException(sb.toString());
+      }
+    }
+  }
+
   /**
    * Set file replication
    * 

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

@@ -8395,7 +8395,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   /** Lookup the encryption zone of a path. */
-  private EncryptionZone getEncryptionZoneForPath(String src) {
+  EncryptionZone getEncryptionZoneForPath(String src) {
     final String[] components = INode.getPathNames(src);
     for (int i = components.length; i > 0; i--) {
       final List<String> l = Arrays.asList(Arrays.copyOfRange(components, 0, i));

+ 24 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesAPI.java

@@ -65,7 +65,11 @@ public class TestEncryptionZonesAPI {
     conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
         JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks");
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-    fs = cluster.getFileSystem();
+    fs = createFileSystem(conf);
+  }
+
+  protected FileSystem createFileSystem(Configuration conf) throws IOException {
+    return cluster.getFileSystem();
   }
 
   @After
@@ -401,4 +405,23 @@ public class TestEncryptionZonesAPI {
       }
     });
   }
+
+  /** Test success of Rename EZ on a directory which is already an EZ. */
+  @Test(timeout = 30000)
+  public void testRenameEncryptionZone()
+          throws Exception {
+    final HdfsAdmin dfsAdmin =
+            new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+    FileSystem.mkdirs(fs, TEST_PATH_WITH_CHILD,
+      new FsPermission((short) 0777));
+    dfsAdmin.createEncryptionZone(TEST_PATH_WITH_CHILD, null);
+    FileSystem.mkdirs(fs, TEST_PATH_WITH_MULTIPLE_CHILDREN,
+       new FsPermission((short) 0777));
+    try {
+      fs.rename(TEST_PATH_WITH_MULTIPLE_CHILDREN, TEST_PATH);
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(
+              "/test/foo/baz can't be moved from an encryption zone.", e);
+    }
+  }
 }

+ 70 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextEncryptionZones.java

@@ -0,0 +1,70 @@
+/**
+ * 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 java.io.IOException;
+import java.net.URI;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.TestEncryptionZonesAPI;
+import org.junit.BeforeClass;
+
+/**
+ * Tests of encryption zone operations using FileContext APIs.
+ */
+public class TestFileContextEncryptionZones extends TestEncryptionZonesAPI  {
+
+  @Override
+  protected FileSystem createFileSystem(Configuration conf) throws IOException {
+    FileContextFS fcFs = new FileContextFS();
+    fcFs.initialize(FileSystem.getDefaultUri(conf), conf);
+    return fcFs;
+  }
+
+  /**
+   * This reuses FSXAttrBaseTest's testcases by creating a filesystem
+   * implementation which uses FileContext by only overriding the xattr related
+   * methods. Other operations will use the normal filesystem.
+   */
+  public static class FileContextFS extends DistributedFileSystem {
+
+    private FileContext fc;
+
+    @Override
+    public void initialize(URI uri, Configuration conf) throws IOException {
+      super.initialize(uri, conf);
+      fc = FileContext.getFileContext(conf);
+    }
+
+    @Override
+    public boolean rename(Path src, Path dst) throws IOException {
+      fc.rename(src, dst);
+      return true;
+    }
+  }
+}

+ 94 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCryptoConf.xml

@@ -262,5 +262,99 @@
         </comparator>
       </comparators>
     </test>
+
+    <test>
+      <description>Test failure of renaming file cross EZ's</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /src</command>
+        <command>-fs NAMENODE -mkdir /dst</command>
+        <command>-fs NAMENODE -ls /</command>-
+        <crypto-admin-command>-createZone -path /src</crypto-admin-command>
+        <crypto-admin-command>-createZone -path /dst</crypto-admin-command>
+        <command>-fs NAMENODE -mkdir /src/subdir</command>
+        <command>-fs NAMENODE -mv /src/subdir /dst</command>-
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /src/subdir</command>
+        <crypto-admin-command>-deleteZone -path /src</crypto-admin-command>
+        <crypto-admin-command>-deleteZone -path /dst</crypto-admin-command>
+        <command>-fs NAMENODE -rmdir /src</command>
+        <command>-fs NAMENODE -rmdir /dst</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>/src/subdir can't be moved from encryption zone /src to encryption zone /dst.</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>Test failure of renaming a non-EZ file into an EZ</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /src</command>
+        <command>-fs NAMENODE -mkdir /dst</command>
+        <command>-fs NAMENODE -ls /</command>-
+        <crypto-admin-command>-createZone -path /dst</crypto-admin-command>
+        <command>-fs NAMENODE -mv /src /dst</command>-
+      </test-commands>
+      <cleanup-commands>
+        <crypto-admin-command>-deleteZone -path /dst</crypto-admin-command>
+        <command>-fs NAMENODE -rmdir /src</command>
+        <command>-fs NAMENODE -rmdir /dst</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>/src can't be moved into an encryption zone</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>Test failure of renaming a non-EZ file from an EZ</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /src</command>
+        <command>-fs NAMENODE -mkdir /dst</command>
+        <command>-fs NAMENODE -ls /</command>-
+        <crypto-admin-command>-createZone -path /src</crypto-admin-command>
+        <command>-fs NAMENODE -mv /src /dst</command>-
+      </test-commands>
+      <cleanup-commands>
+        <crypto-admin-command>-deleteZone -path /src</crypto-admin-command>
+        <command>-fs NAMENODE -rmdir /src</command>
+        <command>-fs NAMENODE -rmdir /dst</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>/src can't be moved from an encryption zone</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>Test success of renaming file intra-EZ</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /src</command>
+        <crypto-admin-command>-createZone -path /src</crypto-admin-command>
+        <command>-fs NAMENODE -mkdir /src/subdir1</command>
+        <command>-fs NAMENODE -mkdir /src/subdir2</command>
+        <command>-fs NAMENODE -mv /src/subdir1 /src/subdir2</command>-
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /src/subdir2/subdir1</command>
+        <command>-fs NAMENODE -rmdir /src/subdir2</command>
+        <crypto-admin-command>-deleteZone -path /src</crypto-admin-command>
+        <command>-fs NAMENODE -rmdir /src</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output></expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
   </tests>
 </configuration>