|
@@ -24,15 +24,18 @@ import java.io.IOException;
|
|
|
import java.util.EnumSet;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
@@ -43,6 +46,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
import org.apache.hadoop.io.EnumSetWritable;
|
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Test;
|
|
|
|
|
@@ -314,4 +318,107 @@ public class TestLeaseRecovery {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * HDFS-14498 - test lease can be recovered for a file where the final
|
|
|
+ * block was never registered with the DNs, and hence the IBRs will never
|
|
|
+ * be received. In this case the final block should be zero bytes and can
|
|
|
+ * be removed.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testLeaseRecoveryEmptyCommittedLastBlock() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ DFSClient client = null;
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
|
+ DistributedFileSystem dfs = cluster.getFileSystem();
|
|
|
+ client =
|
|
|
+ new DFSClient(cluster.getNameNode().getServiceRpcAddress(), conf);
|
|
|
+ String file = "/test/f1";
|
|
|
+ Path filePath = new Path(file);
|
|
|
+
|
|
|
+ createCommittedNotCompleteFile(client, file);
|
|
|
+
|
|
|
+ // Ensure a different client cannot append the file
|
|
|
+ try {
|
|
|
+ dfs.append(filePath);
|
|
|
+ fail("Append to a file(lease is held by another client) should fail");
|
|
|
+ } catch (RemoteException e) {
|
|
|
+ assertTrue(e.getMessage().contains("file lease is currently owned"));
|
|
|
+ }
|
|
|
+
|
|
|
+ // Ensure the lease can be recovered on the first try
|
|
|
+ boolean recovered = client.recoverLease(file);
|
|
|
+ assertEquals(true, recovered);
|
|
|
+
|
|
|
+ // Ensure the recovered file can now be written
|
|
|
+ FSDataOutputStream append = dfs.append(filePath);
|
|
|
+ append.write("test".getBytes());
|
|
|
+ append.close();
|
|
|
+ } finally {
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ cluster = null;
|
|
|
+ }
|
|
|
+ if (client != null) {
|
|
|
+ client.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * HDFS-14498 - similar to testLeaseRecoveryEmptyCommittedLastBlock except
|
|
|
+ * we wait for the lease manager to recover the lease automatically.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testLeaseManagerRecoversEmptyCommittedLastBlock()
|
|
|
+ throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ DFSClient client = null;
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
|
+ client =
|
|
|
+ new DFSClient(cluster.getNameNode().getServiceRpcAddress(), conf);
|
|
|
+ String file = "/test/f1";
|
|
|
+
|
|
|
+ createCommittedNotCompleteFile(client, file);
|
|
|
+ waitLeaseRecovery(cluster);
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ String holder = NameNodeAdapter
|
|
|
+ .getLeaseHolderForPath(cluster.getNameNode(), file);
|
|
|
+ return holder == null;
|
|
|
+ }, 100, 10000);
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ cluster = null;
|
|
|
+ }
|
|
|
+ if (client != null) {
|
|
|
+ client.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void createCommittedNotCompleteFile(DFSClient client, String file)
|
|
|
+ throws IOException {
|
|
|
+ HdfsFileStatus stat = client.getNamenode()
|
|
|
+ .create(file, new FsPermission("777"), "test client",
|
|
|
+ new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)),
|
|
|
+ true, (short) 1, 1024 * 1024 * 128L,
|
|
|
+ new CryptoProtocolVersion[0], null);
|
|
|
+ // Add a block to the file
|
|
|
+ LocatedBlock blk = client.getNamenode()
|
|
|
+ .addBlock(file, "test client", null,
|
|
|
+ new DatanodeInfo[0], stat.getFileId(), new String[0], null);
|
|
|
+ // Without writing anything to the file, or setting up the DN pipeline
|
|
|
+ // attempt to close the file. This will fail (return false) as the NN will
|
|
|
+ // be expecting the registered block to be reported from the DNs via IBR,
|
|
|
+ // but that will never happen, as the pipeline was never established
|
|
|
+ boolean closed = client.getNamenode().complete(
|
|
|
+ file, "test client", blk.getBlock(), stat.getFileId());
|
|
|
+ assertEquals(false, closed);
|
|
|
+ }
|
|
|
+
|
|
|
}
|