|
@@ -17,8 +17,14 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
+import java.io.ByteArrayOutputStream;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.io.PrintStream;
|
|
|
+import java.io.PrintWriter;
|
|
|
+import java.io.RandomAccessFile;
|
|
|
+import java.io.StringReader;
|
|
|
+import java.io.StringWriter;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.List;
|
|
@@ -52,10 +58,15 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.EncryptionFaultInjector;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.EncryptionZoneManager;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
|
|
|
+import org.apache.hadoop.hdfs.tools.DFSck;
|
|
|
+import org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter;
|
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
+import org.apache.hadoop.util.ToolRunner;
|
|
|
import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension.DelegationTokenExtension;
|
|
|
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
|
|
|
import org.apache.hadoop.io.Text;
|
|
@@ -78,6 +89,11 @@ import static org.junit.Assert.assertNotEquals;
|
|
|
import static org.junit.Assert.assertNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
+import org.xml.sax.InputSource;
|
|
|
+import org.xml.sax.helpers.DefaultHandler;
|
|
|
+
|
|
|
+import javax.xml.parsers.SAXParser;
|
|
|
+import javax.xml.parsers.SAXParserFactory;
|
|
|
|
|
|
public class TestEncryptionZones {
|
|
|
|
|
@@ -374,6 +390,7 @@ public class TestEncryptionZones {
|
|
|
final Path allPathFile = new Path(allPath, "file1");
|
|
|
final Path nonEZDir = new Path(testRoot, "nonEZDir");
|
|
|
final Path nonEZFile = new Path(nonEZDir, "file1");
|
|
|
+ final Path nonexistent = new Path("/nonexistent");
|
|
|
final int len = 8192;
|
|
|
|
|
|
fsWrapper.mkdir(testRoot, new FsPermission((short) 0777), true);
|
|
@@ -421,6 +438,9 @@ public class TestEncryptionZones {
|
|
|
assertExceptionContains("Permission denied:", e);
|
|
|
}
|
|
|
|
|
|
+ assertNull("expected null for nonexistent path",
|
|
|
+ userAdmin.getEncryptionZoneForPath(nonexistent));
|
|
|
+
|
|
|
// Check operation with non-ez paths
|
|
|
assertNull("expected null for non-ez path",
|
|
|
userAdmin.getEncryptionZoneForPath(nonEZDir));
|
|
@@ -463,9 +483,15 @@ public class TestEncryptionZones {
|
|
|
final Path testRoot = new Path("/tmp/TestEncryptionZones");
|
|
|
final Path pathFoo = new Path(testRoot, "foo");
|
|
|
final Path pathFooBaz = new Path(pathFoo, "baz");
|
|
|
+ final Path pathFooBazFile = new Path(pathFooBaz, "file");
|
|
|
+ final Path pathFooBar = new Path(pathFoo, "bar");
|
|
|
+ final Path pathFooBarFile = new Path(pathFooBar, "file");
|
|
|
+ final int len = 8192;
|
|
|
wrapper.mkdir(pathFoo, FsPermission.getDirDefault(), true);
|
|
|
dfsAdmin.createEncryptionZone(pathFoo, TEST_KEY);
|
|
|
wrapper.mkdir(pathFooBaz, FsPermission.getDirDefault(), true);
|
|
|
+ DFSTestUtil.createFile(fs, pathFooBazFile, len, (short) 1, 0xFEED);
|
|
|
+ String contents = DFSTestUtil.readFile(fs, pathFooBazFile);
|
|
|
try {
|
|
|
wrapper.rename(pathFooBaz, testRoot);
|
|
|
} catch (IOException e) {
|
|
@@ -473,6 +499,13 @@ public class TestEncryptionZones {
|
|
|
" an encryption zone.", e
|
|
|
);
|
|
|
}
|
|
|
+
|
|
|
+ // Verify that we can rename dir and files within an encryption zone.
|
|
|
+ assertTrue(fs.rename(pathFooBaz, pathFooBar));
|
|
|
+ assertTrue("Rename of dir and file within ez failed",
|
|
|
+ !wrapper.exists(pathFooBaz) && wrapper.exists(pathFooBar));
|
|
|
+ assertEquals("Renamed file contents not the same",
|
|
|
+ contents, DFSTestUtil.readFile(fs, pathFooBarFile));
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 60000)
|
|
@@ -806,4 +839,158 @@ public class TestEncryptionZones {
|
|
|
Assert.assertEquals(tokens[1], testToken);
|
|
|
Assert.assertEquals(1, creds.numberOfTokens());
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test running fsck on a system with encryption zones.
|
|
|
+ */
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testFsckOnEncryptionZones() throws Exception {
|
|
|
+ final int len = 8196;
|
|
|
+ final Path zoneParent = new Path("/zones");
|
|
|
+ final Path zone1 = new Path(zoneParent, "zone1");
|
|
|
+ final Path zone1File = new Path(zone1, "file");
|
|
|
+ fsWrapper.mkdir(zone1, FsPermission.getDirDefault(), true);
|
|
|
+ dfsAdmin.createEncryptionZone(zone1, TEST_KEY);
|
|
|
+ DFSTestUtil.createFile(fs, zone1File, len, (short) 1, 0xFEED);
|
|
|
+ ByteArrayOutputStream bStream = new ByteArrayOutputStream();
|
|
|
+ PrintStream out = new PrintStream(bStream, true);
|
|
|
+ int errCode = ToolRunner.run(new DFSck(conf, out),
|
|
|
+ new String[]{ "/" });
|
|
|
+ assertEquals("Fsck ran with non-zero error code", 0, errCode);
|
|
|
+ String result = bStream.toString();
|
|
|
+ assertTrue("Fsck did not return HEALTHY status",
|
|
|
+ result.contains(NamenodeFsck.HEALTHY_STATUS));
|
|
|
+
|
|
|
+ // Run fsck directly on the encryption zone instead of root
|
|
|
+ errCode = ToolRunner.run(new DFSck(conf, out),
|
|
|
+ new String[]{ zoneParent.toString() });
|
|
|
+ assertEquals("Fsck ran with non-zero error code", 0, errCode);
|
|
|
+ result = bStream.toString();
|
|
|
+ assertTrue("Fsck did not return HEALTHY status",
|
|
|
+ result.contains(NamenodeFsck.HEALTHY_STATUS));
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test correctness of successive snapshot creation and deletion
|
|
|
+ * on a system with encryption zones.
|
|
|
+ */
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testSnapshotsOnEncryptionZones() throws Exception {
|
|
|
+ final int len = 8196;
|
|
|
+ final Path zoneParent = new Path("/zones");
|
|
|
+ final Path zone = new Path(zoneParent, "zone");
|
|
|
+ final Path zoneFile = new Path(zone, "zoneFile");
|
|
|
+ fsWrapper.mkdir(zone, FsPermission.getDirDefault(), true);
|
|
|
+ dfsAdmin.allowSnapshot(zoneParent);
|
|
|
+ dfsAdmin.createEncryptionZone(zone, TEST_KEY);
|
|
|
+ DFSTestUtil.createFile(fs, zoneFile, len, (short) 1, 0xFEED);
|
|
|
+ String contents = DFSTestUtil.readFile(fs, zoneFile);
|
|
|
+ final Path snap1 = fs.createSnapshot(zoneParent);
|
|
|
+ final Path snap1Zone = new Path(snap1, zone.getName());
|
|
|
+ assertEquals("Got unexpected ez path", zone.toString(),
|
|
|
+ dfsAdmin.getEncryptionZoneForPath(snap1Zone).getPath().toString());
|
|
|
+
|
|
|
+ // Now delete the encryption zone, recreate the dir, and take another snapshot
|
|
|
+ fsWrapper.delete(zone, true);
|
|
|
+ fsWrapper.mkdir(zone, FsPermission.getDirDefault(), true);
|
|
|
+ final Path snap2 = fs.createSnapshot(zoneParent);
|
|
|
+ final Path snap2Zone = new Path(snap2, zone.getName());
|
|
|
+ assertNull("Expected null ez path",
|
|
|
+ dfsAdmin.getEncryptionZoneForPath(snap2Zone));
|
|
|
+
|
|
|
+ // Create the encryption zone again
|
|
|
+ dfsAdmin.createEncryptionZone(zone, TEST_KEY);
|
|
|
+ final Path snap3 = fs.createSnapshot(zoneParent);
|
|
|
+ final Path snap3Zone = new Path(snap3, zone.getName());
|
|
|
+ assertEquals("Got unexpected ez path", zone.toString(),
|
|
|
+ dfsAdmin.getEncryptionZoneForPath(snap3Zone).getPath().toString());
|
|
|
+
|
|
|
+ // Verify contents of the snapshotted file
|
|
|
+ final Path snapshottedZoneFile = new Path(
|
|
|
+ snap1.toString() + "/" + zone.getName() + "/" + zoneFile.getName());
|
|
|
+ assertEquals("Contents of snapshotted file have changed unexpectedly",
|
|
|
+ contents, DFSTestUtil.readFile(fs, snapshottedZoneFile));
|
|
|
+
|
|
|
+ // Now delete the snapshots out of order and verify the zones are still correct
|
|
|
+ fs.deleteSnapshot(zoneParent, snap2.getName());
|
|
|
+ assertEquals("Got unexpected ez path", zone.toString(),
|
|
|
+ dfsAdmin.getEncryptionZoneForPath(snap1Zone).getPath().toString());
|
|
|
+ assertEquals("Got unexpected ez path", zone.toString(),
|
|
|
+ dfsAdmin.getEncryptionZoneForPath(snap3Zone).getPath().toString());
|
|
|
+ fs.deleteSnapshot(zoneParent, snap1.getName());
|
|
|
+ assertEquals("Got unexpected ez path", zone.toString(),
|
|
|
+ dfsAdmin.getEncryptionZoneForPath(snap3Zone).getPath().toString());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Verify symlinks can be created in encryption zones and that
|
|
|
+ * they function properly when the target is in the same
|
|
|
+ * or different ez.
|
|
|
+ */
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testEncryptionZonesWithSymlinks() throws Exception {
|
|
|
+ // Verify we can create an encryption zone over both link and target
|
|
|
+ final int len = 8192;
|
|
|
+ final Path parent = new Path("/parent");
|
|
|
+ final Path linkParent = new Path(parent, "symdir1");
|
|
|
+ final Path targetParent = new Path(parent, "symdir2");
|
|
|
+ final Path link = new Path(linkParent, "link");
|
|
|
+ final Path target = new Path(targetParent, "target");
|
|
|
+ fs.mkdirs(parent);
|
|
|
+ dfsAdmin.createEncryptionZone(parent, TEST_KEY);
|
|
|
+ fs.mkdirs(linkParent);
|
|
|
+ fs.mkdirs(targetParent);
|
|
|
+ DFSTestUtil.createFile(fs, target, len, (short)1, 0xFEED);
|
|
|
+ String content = DFSTestUtil.readFile(fs, target);
|
|
|
+ fs.createSymlink(target, link, false);
|
|
|
+ assertEquals("Contents read from link are not the same as target",
|
|
|
+ content, DFSTestUtil.readFile(fs, link));
|
|
|
+ fs.delete(parent, true);
|
|
|
+
|
|
|
+ // Now let's test when the symlink and target are in different
|
|
|
+ // encryption zones
|
|
|
+ fs.mkdirs(linkParent);
|
|
|
+ fs.mkdirs(targetParent);
|
|
|
+ dfsAdmin.createEncryptionZone(linkParent, TEST_KEY);
|
|
|
+ dfsAdmin.createEncryptionZone(targetParent, TEST_KEY);
|
|
|
+ DFSTestUtil.createFile(fs, target, len, (short)1, 0xFEED);
|
|
|
+ content = DFSTestUtil.readFile(fs, target);
|
|
|
+ fs.createSymlink(target, link, false);
|
|
|
+ assertEquals("Contents read from link are not the same as target",
|
|
|
+ content, DFSTestUtil.readFile(fs, link));
|
|
|
+ fs.delete(link, true);
|
|
|
+ fs.delete(target, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test running the OfflineImageViewer on a system with encryption zones.
|
|
|
+ */
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testOfflineImageViewerOnEncryptionZones() throws Exception {
|
|
|
+ final int len = 8196;
|
|
|
+ final Path zoneParent = new Path("/zones");
|
|
|
+ final Path zone1 = new Path(zoneParent, "zone1");
|
|
|
+ final Path zone1File = new Path(zone1, "file");
|
|
|
+ fsWrapper.mkdir(zone1, FsPermission.getDirDefault(), true);
|
|
|
+ dfsAdmin.createEncryptionZone(zone1, TEST_KEY);
|
|
|
+ DFSTestUtil.createFile(fs, zone1File, len, (short) 1, 0xFEED);
|
|
|
+ fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
|
|
|
+ fs.saveNamespace();
|
|
|
+
|
|
|
+ File originalFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil
|
|
|
+ .getFSImage(cluster.getNameNode()).getStorage().getStorageDir(0));
|
|
|
+ if (originalFsimage == null) {
|
|
|
+ throw new RuntimeException("Didn't generate or can't find fsimage");
|
|
|
+ }
|
|
|
+
|
|
|
+ // Run the XML OIV processor
|
|
|
+ StringWriter output = new StringWriter();
|
|
|
+ PrintWriter pw = new PrintWriter(output);
|
|
|
+ PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), pw);
|
|
|
+ v.visit(new RandomAccessFile(originalFsimage, "r"));
|
|
|
+ final String xml = output.getBuffer().toString();
|
|
|
+ SAXParser parser = SAXParserFactory.newInstance().newSAXParser();
|
|
|
+ parser.parse(new InputSource(new StringReader(xml)), new DefaultHandler());
|
|
|
+ }
|
|
|
+
|
|
|
}
|