HDFS-14741. RBF: RecoverLease should be return false when the file is open in multiple destination. Contributed by xuzq

This commit is contained in:
Ayush Saxena 2019-08-22 08:57:22 +05:30
parent 34dd9ee366
commit 52c77bc160
2 changed files with 26 additions and 1 deletions

View File

@ -363,7 +363,7 @@ public boolean recoverLease(String src, String clientName)
new Class<?>[] {String.class, String.class}, new RemoteParam(),
clientName);
Object result = rpcClient.invokeSequential(
locations, method, Boolean.class, Boolean.TRUE);
locations, method, Boolean.class, null);
return (boolean) result;
}

View File

@ -20,6 +20,7 @@
import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.createFile;
import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyFileExists;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
@ -38,6 +39,7 @@
import java.util.Set;
import java.util.TreeSet;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -55,6 +57,7 @@
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.test.GenericTestUtils;
@ -230,6 +233,28 @@ public void testProxyRenameFiles() throws IOException, InterruptedException {
testRename2(getRouterContext(), filename1, renamedFile, false);
}
/**
* Test recoverLease when the result is false.
*/
@Test
public void testRecoverLease() throws Exception {
Path testPath = new Path("/recovery/test_recovery_lease");
DistributedFileSystem routerFs =
(DistributedFileSystem) getRouterFileSystem();
FSDataOutputStream fsDataOutputStream = null;
try {
fsDataOutputStream = routerFs.create(testPath);
fsDataOutputStream.write("hello world".getBytes());
fsDataOutputStream.hflush();
boolean result = routerFs.recoverLease(testPath);
assertFalse(result);
} finally {
IOUtils.closeStream(fsDataOutputStream);
routerFs.delete(testPath, true);
}
}
@Test
public void testGetContentSummaryEc() throws Exception {
DistributedFileSystem routerDFS =