|
@@ -35,7 +35,6 @@ import org.apache.hadoop.fs.Options.Rename;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
-import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
@@ -47,6 +46,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
|
+import org.apache.hadoop.io.EnumSetWritable;
|
|
|
import org.apache.hadoop.ipc.ClientId;
|
|
|
import org.apache.hadoop.ipc.RPC.RpcKind;
|
|
|
import org.apache.hadoop.ipc.RetryCache.CacheEntry;
|
|
@@ -77,9 +78,8 @@ import org.junit.Test;
|
|
|
public class TestNamenodeRetryCache {
|
|
|
private static final byte[] CLIENT_ID = ClientId.getClientId();
|
|
|
private static MiniDFSCluster cluster;
|
|
|
- private static FSNamesystem namesystem;
|
|
|
- private static final PermissionStatus perm = new PermissionStatus(
|
|
|
- "TestNamenodeRetryCache", null, FsPermission.getDefault());
|
|
|
+ private static NamenodeProtocols nnRpc;
|
|
|
+ private static final FsPermission perm = FsPermission.getDefault();
|
|
|
private static DistributedFileSystem filesystem;
|
|
|
private static int callId = 100;
|
|
|
private static Configuration conf;
|
|
@@ -94,7 +94,7 @@ public class TestNamenodeRetryCache {
|
|
|
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
|
|
|
cluster = new MiniDFSCluster.Builder(conf).build();
|
|
|
cluster.waitActive();
|
|
|
- namesystem = cluster.getNamesystem();
|
|
|
+ nnRpc = cluster.getNameNode().getRpcServer();
|
|
|
filesystem = cluster.getFileSystem();
|
|
|
}
|
|
|
|
|
@@ -108,10 +108,6 @@ public class TestNamenodeRetryCache {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
|
|
|
- public static void incrementCallId() {
|
|
|
- callId++;
|
|
|
- }
|
|
|
-
|
|
|
/** Set the current Server RPC call */
|
|
|
public static void newCall() {
|
|
|
Server.Call call = new Server.Call(++callId, 1, null, null,
|
|
@@ -142,15 +138,15 @@ public class TestNamenodeRetryCache {
|
|
|
// Two retried concat calls succeed
|
|
|
concatSetup(file1, file2);
|
|
|
newCall();
|
|
|
- namesystem.concat(file1, new String[]{file2});
|
|
|
- namesystem.concat(file1, new String[]{file2});
|
|
|
- namesystem.concat(file1, new String[]{file2});
|
|
|
+ nnRpc.concat(file1, new String[]{file2});
|
|
|
+ nnRpc.concat(file1, new String[]{file2});
|
|
|
+ nnRpc.concat(file1, new String[]{file2});
|
|
|
|
|
|
// A non-retried concat request fails
|
|
|
newCall();
|
|
|
try {
|
|
|
// Second non-retry call should fail with an exception
|
|
|
- namesystem.concat(file1, new String[]{file2});
|
|
|
+ nnRpc.concat(file1, new String[]{file2});
|
|
|
Assert.fail("testConcat - expected exception is not thrown");
|
|
|
} catch (IOException e) {
|
|
|
// Expected
|
|
@@ -165,15 +161,15 @@ public class TestNamenodeRetryCache {
|
|
|
String dir = "/testNamenodeRetryCache/testDelete";
|
|
|
// Two retried calls to create a non existent file
|
|
|
newCall();
|
|
|
- namesystem.mkdirs(dir, perm, true);
|
|
|
+ nnRpc.mkdirs(dir, perm, true);
|
|
|
newCall();
|
|
|
- Assert.assertTrue(namesystem.delete(dir, false));
|
|
|
- Assert.assertTrue(namesystem.delete(dir, false));
|
|
|
- Assert.assertTrue(namesystem.delete(dir, false));
|
|
|
+ Assert.assertTrue(nnRpc.delete(dir, false));
|
|
|
+ Assert.assertTrue(nnRpc.delete(dir, false));
|
|
|
+ Assert.assertTrue(nnRpc.delete(dir, false));
|
|
|
|
|
|
// non-retried call fails and gets false as return
|
|
|
newCall();
|
|
|
- Assert.assertFalse(namesystem.delete(dir, false));
|
|
|
+ Assert.assertFalse(nnRpc.delete(dir, false));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -185,15 +181,15 @@ public class TestNamenodeRetryCache {
|
|
|
|
|
|
// Two retried symlink calls succeed
|
|
|
newCall();
|
|
|
- namesystem.createSymlink(target, "/a/b", perm, true);
|
|
|
- namesystem.createSymlink(target, "/a/b", perm, true);
|
|
|
- namesystem.createSymlink(target, "/a/b", perm, true);
|
|
|
+ nnRpc.createSymlink(target, "/a/b", perm, true);
|
|
|
+ nnRpc.createSymlink(target, "/a/b", perm, true);
|
|
|
+ nnRpc.createSymlink(target, "/a/b", perm, true);
|
|
|
|
|
|
// non-retried call fails
|
|
|
newCall();
|
|
|
try {
|
|
|
// Second non-retry call should fail with an exception
|
|
|
- namesystem.createSymlink(target, "/a/b", perm, true);
|
|
|
+ nnRpc.createSymlink(target, "/a/b", perm, true);
|
|
|
Assert.fail("testCreateSymlink - expected exception is not thrown");
|
|
|
} catch (IOException e) {
|
|
|
// Expected
|
|
@@ -208,21 +204,16 @@ public class TestNamenodeRetryCache {
|
|
|
String src = "/testNamenodeRetryCache/testCreate/file";
|
|
|
// Two retried calls succeed
|
|
|
newCall();
|
|
|
- HdfsFileStatus status = namesystem.startFile(src, perm, "holder",
|
|
|
- "clientmachine", EnumSet.of(CreateFlag.CREATE), true, (short) 1,
|
|
|
- BlockSize, null);
|
|
|
- Assert.assertEquals(status, namesystem.startFile(src, perm,
|
|
|
- "holder", "clientmachine", EnumSet.of(CreateFlag.CREATE),
|
|
|
- true, (short) 1, BlockSize, null));
|
|
|
- Assert.assertEquals(status, namesystem.startFile(src, perm,
|
|
|
- "holder", "clientmachine", EnumSet.of(CreateFlag.CREATE),
|
|
|
- true, (short) 1, BlockSize, null));
|
|
|
+ HdfsFileStatus status = nnRpc.create(src, perm, "holder",
|
|
|
+ new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true,
|
|
|
+ (short) 1, BlockSize, null);
|
|
|
+ Assert.assertEquals(status, nnRpc.create(src, perm, "holder", new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true, (short) 1, BlockSize, null));
|
|
|
+ Assert.assertEquals(status, nnRpc.create(src, perm, "holder", new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true, (short) 1, BlockSize, null));
|
|
|
|
|
|
// A non-retried call fails
|
|
|
newCall();
|
|
|
try {
|
|
|
- namesystem.startFile(src, perm, "holder", "clientmachine",
|
|
|
- EnumSet.of(CreateFlag.CREATE), true, (short) 1, BlockSize, null);
|
|
|
+ nnRpc.create(src, perm, "holder", new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true, (short) 1, BlockSize, null);
|
|
|
Assert.fail("testCreate - expected exception is not thrown");
|
|
|
} catch (IOException e) {
|
|
|
// expected
|
|
@@ -241,14 +232,14 @@ public class TestNamenodeRetryCache {
|
|
|
|
|
|
// Retried append requests succeed
|
|
|
newCall();
|
|
|
- LocatedBlock b = namesystem.appendFile(src, "holder", "clientMachine");
|
|
|
- Assert.assertEquals(b, namesystem.appendFile(src, "holder", "clientMachine"));
|
|
|
- Assert.assertEquals(b, namesystem.appendFile(src, "holder", "clientMachine"));
|
|
|
+ LocatedBlock b = nnRpc.append(src, "holder");
|
|
|
+ Assert.assertEquals(b, nnRpc.append(src, "holder"));
|
|
|
+ Assert.assertEquals(b, nnRpc.append(src, "holder"));
|
|
|
|
|
|
// non-retried call fails
|
|
|
newCall();
|
|
|
try {
|
|
|
- namesystem.appendFile(src, "holder", "clientMachine");
|
|
|
+ nnRpc.append(src, "holder");
|
|
|
Assert.fail("testAppend - expected exception is not thrown");
|
|
|
} catch (Exception e) {
|
|
|
// Expected
|
|
@@ -264,17 +255,17 @@ public class TestNamenodeRetryCache {
|
|
|
String src = "/testNamenodeRetryCache/testRename1/src";
|
|
|
String target = "/testNamenodeRetryCache/testRename1/target";
|
|
|
resetCall();
|
|
|
- namesystem.mkdirs(src, perm, true);
|
|
|
+ nnRpc.mkdirs(src, perm, true);
|
|
|
|
|
|
// Retried renames succeed
|
|
|
newCall();
|
|
|
- Assert.assertTrue(namesystem.renameTo(src, target));
|
|
|
- Assert.assertTrue(namesystem.renameTo(src, target));
|
|
|
- Assert.assertTrue(namesystem.renameTo(src, target));
|
|
|
+ Assert.assertTrue(nnRpc.rename(src, target));
|
|
|
+ Assert.assertTrue(nnRpc.rename(src, target));
|
|
|
+ Assert.assertTrue(nnRpc.rename(src, target));
|
|
|
|
|
|
// A non-retried request fails
|
|
|
newCall();
|
|
|
- Assert.assertFalse(namesystem.renameTo(src, target));
|
|
|
+ Assert.assertFalse(nnRpc.rename(src, target));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -285,18 +276,18 @@ public class TestNamenodeRetryCache {
|
|
|
String src = "/testNamenodeRetryCache/testRename2/src";
|
|
|
String target = "/testNamenodeRetryCache/testRename2/target";
|
|
|
resetCall();
|
|
|
- namesystem.mkdirs(src, perm, true);
|
|
|
+ nnRpc.mkdirs(src, perm, true);
|
|
|
|
|
|
// Retried renames succeed
|
|
|
newCall();
|
|
|
- namesystem.renameTo(src, target, Rename.NONE);
|
|
|
- namesystem.renameTo(src, target, Rename.NONE);
|
|
|
- namesystem.renameTo(src, target, Rename.NONE);
|
|
|
+ nnRpc.rename2(src, target, Rename.NONE);
|
|
|
+ nnRpc.rename2(src, target, Rename.NONE);
|
|
|
+ nnRpc.rename2(src, target, Rename.NONE);
|
|
|
|
|
|
// A non-retried request fails
|
|
|
newCall();
|
|
|
try {
|
|
|
- namesystem.renameTo(src, target, Rename.NONE);
|
|
|
+ nnRpc.rename2(src, target, Rename.NONE);
|
|
|
Assert.fail("testRename 2 expected exception is not thrown");
|
|
|
} catch (IOException e) {
|
|
|
// expected
|
|
@@ -310,11 +301,12 @@ public class TestNamenodeRetryCache {
|
|
|
@Test(timeout = 60000)
|
|
|
public void testUpdatePipelineWithFailOver() throws Exception {
|
|
|
cluster.shutdown();
|
|
|
- namesystem = null;
|
|
|
+ nnRpc = null;
|
|
|
filesystem = null;
|
|
|
cluster = new MiniDFSCluster.Builder(conf).nnTopology(
|
|
|
MiniDFSNNTopology.simpleHATopology()).numDataNodes(1).build();
|
|
|
- FSNamesystem ns0 = cluster.getNamesystem(0);
|
|
|
+ cluster.waitActive();
|
|
|
+ NamenodeProtocols ns0 = cluster.getNameNodeRpc(0);
|
|
|
ExtendedBlock oldBlock = new ExtendedBlock();
|
|
|
ExtendedBlock newBlock = new ExtendedBlock();
|
|
|
DatanodeID[] newNodes = new DatanodeID[2];
|
|
@@ -345,20 +337,20 @@ public class TestNamenodeRetryCache {
|
|
|
public void testSnapshotMethods() throws Exception {
|
|
|
String dir = "/testNamenodeRetryCache/testCreateSnapshot/src";
|
|
|
resetCall();
|
|
|
- namesystem.mkdirs(dir, perm, true);
|
|
|
- namesystem.allowSnapshot(dir);
|
|
|
+ nnRpc.mkdirs(dir, perm, true);
|
|
|
+ nnRpc.allowSnapshot(dir);
|
|
|
|
|
|
// Test retry of create snapshot
|
|
|
newCall();
|
|
|
- String name = namesystem.createSnapshot(dir, "snap1");
|
|
|
- Assert.assertEquals(name, namesystem.createSnapshot(dir, "snap1"));
|
|
|
- Assert.assertEquals(name, namesystem.createSnapshot(dir, "snap1"));
|
|
|
- Assert.assertEquals(name, namesystem.createSnapshot(dir, "snap1"));
|
|
|
+ String name = nnRpc.createSnapshot(dir, "snap1");
|
|
|
+ Assert.assertEquals(name, nnRpc.createSnapshot(dir, "snap1"));
|
|
|
+ Assert.assertEquals(name, nnRpc.createSnapshot(dir, "snap1"));
|
|
|
+ Assert.assertEquals(name, nnRpc.createSnapshot(dir, "snap1"));
|
|
|
|
|
|
// Non retried calls should fail
|
|
|
newCall();
|
|
|
try {
|
|
|
- namesystem.createSnapshot(dir, "snap1");
|
|
|
+ nnRpc.createSnapshot(dir, "snap1");
|
|
|
Assert.fail("testSnapshotMethods expected exception is not thrown");
|
|
|
} catch (IOException e) {
|
|
|
// exptected
|
|
@@ -366,14 +358,14 @@ public class TestNamenodeRetryCache {
|
|
|
|
|
|
// Test retry of rename snapshot
|
|
|
newCall();
|
|
|
- namesystem.renameSnapshot(dir, "snap1", "snap2");
|
|
|
- namesystem.renameSnapshot(dir, "snap1", "snap2");
|
|
|
- namesystem.renameSnapshot(dir, "snap1", "snap2");
|
|
|
+ nnRpc.renameSnapshot(dir, "snap1", "snap2");
|
|
|
+ nnRpc.renameSnapshot(dir, "snap1", "snap2");
|
|
|
+ nnRpc.renameSnapshot(dir, "snap1", "snap2");
|
|
|
|
|
|
// Non retried calls should fail
|
|
|
newCall();
|
|
|
try {
|
|
|
- namesystem.renameSnapshot(dir, "snap1", "snap2");
|
|
|
+ nnRpc.renameSnapshot(dir, "snap1", "snap2");
|
|
|
Assert.fail("testSnapshotMethods expected exception is not thrown");
|
|
|
} catch (IOException e) {
|
|
|
// expected
|
|
@@ -381,14 +373,14 @@ public class TestNamenodeRetryCache {
|
|
|
|
|
|
// Test retry of delete snapshot
|
|
|
newCall();
|
|
|
- namesystem.deleteSnapshot(dir, "snap2");
|
|
|
- namesystem.deleteSnapshot(dir, "snap2");
|
|
|
- namesystem.deleteSnapshot(dir, "snap2");
|
|
|
+ nnRpc.deleteSnapshot(dir, "snap2");
|
|
|
+ nnRpc.deleteSnapshot(dir, "snap2");
|
|
|
+ nnRpc.deleteSnapshot(dir, "snap2");
|
|
|
|
|
|
// Non retried calls should fail
|
|
|
newCall();
|
|
|
try {
|
|
|
- namesystem.deleteSnapshot(dir, "snap2");
|
|
|
+ nnRpc.deleteSnapshot(dir, "snap2");
|
|
|
Assert.fail("testSnapshotMethods expected exception is not thrown");
|
|
|
} catch (IOException e) {
|
|
|
// expected
|
|
@@ -399,7 +391,7 @@ public class TestNamenodeRetryCache {
|
|
|
public void testRetryCacheConfig() {
|
|
|
// By default retry configuration should be enabled
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
- Assert.assertNotNull(FSNamesystem.initRetryCache(conf));
|
|
|
+ Assert.assertNotNull(FSNamesystem.initRetryCache(conf));
|
|
|
|
|
|
// If retry cache is disabled, it should not be created
|
|
|
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY, false);
|
|
@@ -413,7 +405,8 @@ public class TestNamenodeRetryCache {
|
|
|
@Test
|
|
|
public void testRetryCacheRebuild() throws Exception {
|
|
|
DFSTestUtil.runOperations(cluster, filesystem, conf, BlockSize, 0);
|
|
|
-
|
|
|
+ FSNamesystem namesystem = cluster.getNamesystem();
|
|
|
+
|
|
|
LightWeightCache<CacheEntry, CacheEntry> cacheSet =
|
|
|
(LightWeightCache<CacheEntry, CacheEntry>) namesystem.getRetryCache().getCacheSet();
|
|
|
assertEquals(23, cacheSet.size());
|
|
@@ -429,8 +422,8 @@ public class TestNamenodeRetryCache {
|
|
|
// restart NameNode
|
|
|
cluster.restartNameNode();
|
|
|
cluster.waitActive();
|
|
|
+
|
|
|
namesystem = cluster.getNamesystem();
|
|
|
-
|
|
|
// check retry cache
|
|
|
assertTrue(namesystem.hasRetryCache());
|
|
|
cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) namesystem
|