|
@@ -39,11 +39,14 @@ import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FsShell;
|
|
import org.apache.hadoop.fs.FsShell;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.hdfs.AppendTestUtil;
|
|
import org.apache.hadoop.hdfs.AppendTestUtil;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
|
+import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
@@ -124,23 +127,85 @@ public class TestFileTruncate {
|
|
|
|
|
|
int newLength = fileLength - toTruncate;
|
|
int newLength = fileLength - toTruncate;
|
|
boolean isReady = fs.truncate(p, newLength);
|
|
boolean isReady = fs.truncate(p, newLength);
|
|
|
|
+ LOG.info("fileLength=" + fileLength + ", newLength=" + newLength
|
|
|
|
+ + ", toTruncate=" + toTruncate + ", isReady=" + isReady);
|
|
|
|
|
|
- if(!isReady)
|
|
|
|
|
|
+ assertEquals("File must be closed for zero truncate"
|
|
|
|
+ + " or truncating at the block boundary",
|
|
|
|
+ isReady, toTruncate == 0 || newLength % BLOCK_SIZE == 0);
|
|
|
|
+ if (!isReady) {
|
|
checkBlockRecovery(p);
|
|
checkBlockRecovery(p);
|
|
-
|
|
|
|
- FileStatus fileStatus = fs.getFileStatus(p);
|
|
|
|
- assertThat(fileStatus.getLen(), is((long) newLength));
|
|
|
|
|
|
+ }
|
|
|
|
|
|
ContentSummary cs = fs.getContentSummary(parent);
|
|
ContentSummary cs = fs.getContentSummary(parent);
|
|
assertEquals("Bad disk space usage",
|
|
assertEquals("Bad disk space usage",
|
|
cs.getSpaceConsumed(), newLength * REPLICATION);
|
|
cs.getSpaceConsumed(), newLength * REPLICATION);
|
|
// validate the file content
|
|
// validate the file content
|
|
- AppendTestUtil.checkFullFile(fs, p, newLength, contents, p.toString());
|
|
|
|
|
|
+ checkFullFile(p, newLength, contents);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
fs.delete(parent, true);
|
|
fs.delete(parent, true);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /** Truncate the same file multiple times until its size is zero. */
|
|
|
|
+ @Test
|
|
|
|
+ public void testMultipleTruncate() throws IOException {
|
|
|
|
+ Path dir = new Path("/testMultipleTruncate");
|
|
|
|
+ fs.mkdirs(dir);
|
|
|
|
+ final Path p = new Path(dir, "file");
|
|
|
|
+ final byte[] data = new byte[100 * BLOCK_SIZE];
|
|
|
|
+ DFSUtil.getRandom().nextBytes(data);
|
|
|
|
+ writeContents(data, data.length, p);
|
|
|
|
+
|
|
|
|
+ for(int n = data.length; n > 0; ) {
|
|
|
|
+ final int newLength = DFSUtil.getRandom().nextInt(n);
|
|
|
|
+ final boolean isReady = fs.truncate(p, newLength);
|
|
|
|
+ LOG.info("newLength=" + newLength + ", isReady=" + isReady);
|
|
|
|
+ assertEquals("File must be closed for truncating at the block boundary",
|
|
|
|
+ isReady, newLength % BLOCK_SIZE == 0);
|
|
|
|
+ if (!isReady) {
|
|
|
|
+ checkBlockRecovery(p);
|
|
|
|
+ }
|
|
|
|
+ checkFullFile(p, newLength, data);
|
|
|
|
+ n = newLength;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ fs.delete(dir, true);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Truncate files and then run other operations such as
|
|
|
|
+ * rename, set replication, set permission, etc.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testTruncateWithOtherOperations() throws IOException {
|
|
|
|
+ Path dir = new Path("/testTruncateOtherOperations");
|
|
|
|
+ fs.mkdirs(dir);
|
|
|
|
+ final Path p = new Path(dir, "file");
|
|
|
|
+ final byte[] data = new byte[2 * BLOCK_SIZE];
|
|
|
|
+
|
|
|
|
+ DFSUtil.getRandom().nextBytes(data);
|
|
|
|
+ writeContents(data, data.length, p);
|
|
|
|
+
|
|
|
|
+ final int newLength = data.length - 1;
|
|
|
|
+ boolean isReady = fs.truncate(p, newLength);
|
|
|
|
+ assertFalse(isReady);
|
|
|
|
+
|
|
|
|
+ fs.setReplication(p, (short)(REPLICATION - 1));
|
|
|
|
+ fs.setPermission(p, FsPermission.createImmutable((short)0444));
|
|
|
|
+
|
|
|
|
+ final Path q = new Path(dir, "newFile");
|
|
|
|
+ fs.rename(p, q);
|
|
|
|
+
|
|
|
|
+ checkBlockRecovery(q);
|
|
|
|
+ checkFullFile(q, newLength, data);
|
|
|
|
+
|
|
|
|
+ cluster.restartNameNode();
|
|
|
|
+ checkFullFile(q, newLength, data);
|
|
|
|
+
|
|
|
|
+ fs.delete(dir, true);
|
|
|
|
+ }
|
|
|
|
+
|
|
@Test
|
|
@Test
|
|
public void testSnapshotWithAppendTruncate() throws IOException {
|
|
public void testSnapshotWithAppendTruncate() throws IOException {
|
|
testSnapshotWithAppendTruncate(0, 1, 2);
|
|
testSnapshotWithAppendTruncate(0, 1, 2);
|
|
@@ -434,15 +499,35 @@ public class TestFileTruncate {
|
|
int toTruncate = 1;
|
|
int toTruncate = 1;
|
|
|
|
|
|
byte[] contents = AppendTestUtil.initBuffer(startingFileSize);
|
|
byte[] contents = AppendTestUtil.initBuffer(startingFileSize);
|
|
- final Path p = new Path("/testTruncateFailure");
|
|
|
|
- FSDataOutputStream out = fs.create(p, false, BLOCK_SIZE, REPLICATION,
|
|
|
|
- BLOCK_SIZE);
|
|
|
|
- out.write(contents, 0, startingFileSize);
|
|
|
|
- try {
|
|
|
|
- fs.truncate(p, 0);
|
|
|
|
- fail("Truncate must fail on open file.");
|
|
|
|
- } catch(IOException expected) {}
|
|
|
|
- out.close();
|
|
|
|
|
|
+ final Path dir = new Path("/dir");
|
|
|
|
+ final Path p = new Path(dir, "testTruncateFailure");
|
|
|
|
+ {
|
|
|
|
+ FSDataOutputStream out = fs.create(p, false, BLOCK_SIZE, REPLICATION,
|
|
|
|
+ BLOCK_SIZE);
|
|
|
|
+ out.write(contents, 0, startingFileSize);
|
|
|
|
+ try {
|
|
|
|
+ fs.truncate(p, 0);
|
|
|
|
+ fail("Truncate must fail on open file.");
|
|
|
|
+ } catch (IOException expected) {
|
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
|
+ "Failed to TRUNCATE_FILE", expected);
|
|
|
|
+ } finally {
|
|
|
|
+ out.close();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ {
|
|
|
|
+ FSDataOutputStream out = fs.append(p);
|
|
|
|
+ try {
|
|
|
|
+ fs.truncate(p, 0);
|
|
|
|
+ fail("Truncate must fail for append.");
|
|
|
|
+ } catch (IOException expected) {
|
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
|
+ "Failed to TRUNCATE_FILE", expected);
|
|
|
|
+ } finally {
|
|
|
|
+ out.close();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
try {
|
|
try {
|
|
fs.truncate(p, -1);
|
|
fs.truncate(p, -1);
|
|
@@ -452,6 +537,45 @@ public class TestFileTruncate {
|
|
"Cannot truncate to a negative file size", expected);
|
|
"Cannot truncate to a negative file size", expected);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ try {
|
|
|
|
+ fs.truncate(p, startingFileSize + 1);
|
|
|
|
+ fail("Truncate must fail for a larger new length.");
|
|
|
|
+ } catch (Exception expected) {
|
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
|
+ "Cannot truncate to a larger file size", expected);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ fs.truncate(dir, 0);
|
|
|
|
+ fail("Truncate must fail for a directory.");
|
|
|
|
+ } catch (Exception expected) {
|
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
|
+ "Path is not a file", expected);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ fs.truncate(new Path(dir, "non-existing"), 0);
|
|
|
|
+ fail("Truncate must fail for a non-existing file.");
|
|
|
|
+ } catch (Exception expected) {
|
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
|
+ "File does not exist", expected);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ fs.setPermission(p, FsPermission.createImmutable((short)0664));
|
|
|
|
+ {
|
|
|
|
+ final UserGroupInformation fooUgi =
|
|
|
|
+ UserGroupInformation.createUserForTesting("foo", new String[]{"foo"});
|
|
|
|
+ try {
|
|
|
|
+ final FileSystem foofs = DFSTestUtil.getFileSystemAs(fooUgi, conf);
|
|
|
|
+ foofs.truncate(p, 0);
|
|
|
|
+ fail("Truncate must fail for no WRITE permission.");
|
|
|
|
+ } catch (Exception expected) {
|
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
|
+ "Permission denied", expected);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
cluster.shutdownDataNodes();
|
|
cluster.shutdownDataNodes();
|
|
NameNodeAdapter.getLeaseManager(cluster.getNamesystem())
|
|
NameNodeAdapter.getLeaseManager(cluster.getNamesystem())
|
|
.setLeasePeriod(LOW_SOFTLIMIT, LOW_HARDLIMIT);
|
|
.setLeasePeriod(LOW_SOFTLIMIT, LOW_HARDLIMIT);
|
|
@@ -461,6 +585,16 @@ public class TestFileTruncate {
|
|
assertThat("truncate should have triggered block recovery.",
|
|
assertThat("truncate should have triggered block recovery.",
|
|
isReady, is(false));
|
|
isReady, is(false));
|
|
|
|
|
|
|
|
+ {
|
|
|
|
+ try {
|
|
|
|
+ fs.truncate(p, 0);
|
|
|
|
+ fail("Truncate must fail since a trancate is already in pregress.");
|
|
|
|
+ } catch (IOException expected) {
|
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
|
+ "Failed to TRUNCATE_FILE", expected);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
boolean recoveryTriggered = false;
|
|
boolean recoveryTriggered = false;
|
|
for(int i = 0; i < RECOVERY_ATTEMPTS; i++) {
|
|
for(int i = 0; i < RECOVERY_ATTEMPTS; i++) {
|
|
String leaseHolder =
|
|
String leaseHolder =
|
|
@@ -484,9 +618,6 @@ public class TestFileTruncate {
|
|
.setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
|
|
.setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
|
|
HdfsConstants.LEASE_HARDLIMIT_PERIOD);
|
|
HdfsConstants.LEASE_HARDLIMIT_PERIOD);
|
|
|
|
|
|
- FileStatus fileStatus = fs.getFileStatus(p);
|
|
|
|
- assertThat(fileStatus.getLen(), is((long) newLength));
|
|
|
|
-
|
|
|
|
checkFullFile(p, newLength, contents);
|
|
checkFullFile(p, newLength, contents);
|
|
fs.delete(p, false);
|
|
fs.delete(p, false);
|
|
}
|
|
}
|
|
@@ -519,10 +650,6 @@ public class TestFileTruncate {
|
|
cluster.getNamesystem().recoverLease(s, holder, "");
|
|
cluster.getNamesystem().recoverLease(s, holder, "");
|
|
|
|
|
|
checkBlockRecovery(p);
|
|
checkBlockRecovery(p);
|
|
-
|
|
|
|
- FileStatus fileStatus = fs.getFileStatus(p);
|
|
|
|
- assertThat(fileStatus.getLen(), is((long) newLength));
|
|
|
|
-
|
|
|
|
checkFullFile(p, newLength, contents);
|
|
checkFullFile(p, newLength, contents);
|
|
fs.delete(p, false);
|
|
fs.delete(p, false);
|
|
}
|
|
}
|
|
@@ -798,9 +925,14 @@ public class TestFileTruncate {
|
|
}
|
|
}
|
|
|
|
|
|
static void checkBlockRecovery(Path p) throws IOException {
|
|
static void checkBlockRecovery(Path p) throws IOException {
|
|
|
|
+ checkBlockRecovery(p, fs);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static void checkBlockRecovery(Path p, DistributedFileSystem dfs)
|
|
|
|
+ throws IOException {
|
|
boolean success = false;
|
|
boolean success = false;
|
|
for(int i = 0; i < SUCCESS_ATTEMPTS; i++) {
|
|
for(int i = 0; i < SUCCESS_ATTEMPTS; i++) {
|
|
- LocatedBlocks blocks = getLocatedBlocks(p);
|
|
|
|
|
|
+ LocatedBlocks blocks = getLocatedBlocks(p, dfs);
|
|
boolean noLastBlock = blocks.getLastLocatedBlock() == null;
|
|
boolean noLastBlock = blocks.getLastLocatedBlock() == null;
|
|
if(!blocks.isUnderConstruction() &&
|
|
if(!blocks.isUnderConstruction() &&
|
|
(noLastBlock || blocks.isLastBlockComplete())) {
|
|
(noLastBlock || blocks.isLastBlockComplete())) {
|
|
@@ -814,7 +946,12 @@ public class TestFileTruncate {
|
|
}
|
|
}
|
|
|
|
|
|
static LocatedBlocks getLocatedBlocks(Path src) throws IOException {
|
|
static LocatedBlocks getLocatedBlocks(Path src) throws IOException {
|
|
- return fs.getClient().getLocatedBlocks(src.toString(), 0, Long.MAX_VALUE);
|
|
|
|
|
|
+ return getLocatedBlocks(src, fs);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ static LocatedBlocks getLocatedBlocks(Path src, DistributedFileSystem dfs)
|
|
|
|
+ throws IOException {
|
|
|
|
+ return dfs.getClient().getLocatedBlocks(src.toString(), 0, Long.MAX_VALUE);
|
|
}
|
|
}
|
|
|
|
|
|
static void assertBlockExists(Block blk) {
|
|
static void assertBlockExists(Block blk) {
|