|
@@ -26,13 +26,18 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSOutputStream;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
+import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.junit.After;
|
|
|
+import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
@@ -148,12 +153,11 @@ public class TestDiskspaceQuotaUpdate {
|
|
|
final Path foo = new Path("/foo");
|
|
|
final Path bar = new Path(foo, "bar");
|
|
|
DFSTestUtil.createFile(dfs, bar, BLOCKSIZE, REPLICATION, 0L);
|
|
|
- dfs.setQuota(foo, Long.MAX_VALUE-1, Long.MAX_VALUE-1);
|
|
|
+ dfs.setQuota(foo, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
|
|
|
|
|
|
FSDataOutputStream out = dfs.append(bar);
|
|
|
out.write(new byte[BLOCKSIZE / 4]);
|
|
|
- ((DFSOutputStream) out.getWrappedStream()).hsync(EnumSet
|
|
|
- .of(HdfsDataOutputStream.SyncFlag.UPDATE_LENGTH));
|
|
|
+ ((DFSOutputStream) out.getWrappedStream()).hsync(EnumSet.of(HdfsDataOutputStream.SyncFlag.UPDATE_LENGTH));
|
|
|
|
|
|
INodeDirectory fooNode = fsdir.getINode4Write(foo.toString()).asDirectory();
|
|
|
QuotaCounts quota = fooNode.getDirectoryWithQuotaFeature()
|
|
@@ -182,4 +186,87 @@ public class TestDiskspaceQuotaUpdate {
|
|
|
assertEquals(2, ns); // foo and bar
|
|
|
assertEquals((BLOCKSIZE * 2 + BLOCKSIZE / 2) * REPLICATION, ds);
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test append over storage quota does not mark file as UC or create lease
|
|
|
+ */
|
|
|
+ @Test (timeout=60000)
|
|
|
+ public void testAppendOverStorageQuota() throws Exception {
|
|
|
+ final Path dir = new Path("/TestAppendOverQuota");
|
|
|
+ final Path file = new Path(dir, "file");
|
|
|
+
|
|
|
+ // create partial block file
|
|
|
+ dfs.mkdirs(dir);
|
|
|
+ DFSTestUtil.createFile(dfs, file, BLOCKSIZE/2, REPLICATION, seed);
|
|
|
+
|
|
|
+ // lower quota to cause exception when appending to partial block
|
|
|
+ dfs.setQuota(dir, Long.MAX_VALUE - 1, 1);
|
|
|
+ final INodeDirectory dirNode = fsdir.getINode4Write(dir.toString())
|
|
|
+ .asDirectory();
|
|
|
+ final long spaceUsed = dirNode.getDirectoryWithQuotaFeature()
|
|
|
+ .getSpaceConsumed().getStorageSpace();
|
|
|
+ try {
|
|
|
+ DFSTestUtil.appendFile(dfs, file, BLOCKSIZE);
|
|
|
+ Assert.fail("append didn't fail");
|
|
|
+ } catch (DSQuotaExceededException e) {
|
|
|
+ // ignore
|
|
|
+ }
|
|
|
+
|
|
|
+ // check that the file exists, isn't UC, and has no dangling lease
|
|
|
+ INodeFile inode = fsdir.getINode(file.toString()).asFile();
|
|
|
+ Assert.assertNotNull(inode);
|
|
|
+ Assert.assertFalse("should not be UC", inode.isUnderConstruction());
|
|
|
+ Assert.assertNull("should not have a lease", cluster.getNamesystem()
|
|
|
+ .getLeaseManager().getLeaseByPath(file.toString()));
|
|
|
+ // make sure the quota usage is unchanged
|
|
|
+ final long newSpaceUsed = dirNode.getDirectoryWithQuotaFeature()
|
|
|
+ .getSpaceConsumed().getStorageSpace();
|
|
|
+ assertEquals(spaceUsed, newSpaceUsed);
|
|
|
+ // make sure edits aren't corrupted
|
|
|
+ dfs.recoverLease(file);
|
|
|
+ cluster.restartNameNodes();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test append over a specific type of storage quota does not mark file as
|
|
|
+ * UC or create a lease
|
|
|
+ */
|
|
|
+ @Test (timeout=60000)
|
|
|
+ public void testAppendOverTypeQuota() throws Exception {
|
|
|
+ final Path dir = new Path("/TestAppendOverTypeQuota");
|
|
|
+ final Path file = new Path(dir, "file");
|
|
|
+
|
|
|
+ // create partial block file
|
|
|
+ dfs.mkdirs(dir);
|
|
|
+ // set the storage policy on dir
|
|
|
+ dfs.setStoragePolicy(dir, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
|
|
|
+ DFSTestUtil.createFile(dfs, file, BLOCKSIZE/2, REPLICATION, seed);
|
|
|
+
|
|
|
+ // set quota of SSD to 1L
|
|
|
+ dfs.setQuotaByStorageType(dir, StorageType.SSD, 1L);
|
|
|
+ final INodeDirectory dirNode = fsdir.getINode4Write(dir.toString())
|
|
|
+ .asDirectory();
|
|
|
+ final long spaceUsed = dirNode.getDirectoryWithQuotaFeature()
|
|
|
+ .getSpaceConsumed().getStorageSpace();
|
|
|
+ try {
|
|
|
+ DFSTestUtil.appendFile(dfs, file, BLOCKSIZE);
|
|
|
+ Assert.fail("append didn't fail");
|
|
|
+ } catch (RemoteException e) {
|
|
|
+ assertTrue(e.getClassName().contains("QuotaByStorageTypeExceededException"));
|
|
|
+ }
|
|
|
+
|
|
|
+ // check that the file exists, isn't UC, and has no dangling lease
|
|
|
+ INodeFile inode = fsdir.getINode(file.toString()).asFile();
|
|
|
+ Assert.assertNotNull(inode);
|
|
|
+ Assert.assertFalse("should not be UC", inode.isUnderConstruction());
|
|
|
+ Assert.assertNull("should not have a lease", cluster.getNamesystem()
|
|
|
+ .getLeaseManager().getLeaseByPath(file.toString()));
|
|
|
+ // make sure the quota usage is unchanged
|
|
|
+ final long newSpaceUsed = dirNode.getDirectoryWithQuotaFeature()
|
|
|
+ .getSpaceConsumed().getStorageSpace();
|
|
|
+ assertEquals(spaceUsed, newSpaceUsed);
|
|
|
+ // make sure edits aren't corrupted
|
|
|
+ dfs.recoverLease(file);
|
|
|
+ cluster.restartNameNodes();
|
|
|
+ }
|
|
|
}
|