|
@@ -86,6 +86,8 @@ import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.apache.log4j.Level;
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
|
|
@@ -93,6 +95,7 @@ import org.junit.Test;
|
|
|
* This class tests various cases during file creation.
|
|
|
*/
|
|
|
public class TestFileCreation {
|
|
|
+ public static final Log LOG = LogFactory.getLog(TestFileCreation.class);
|
|
|
static final String DIR = "/" + TestFileCreation.class.getSimpleName() + "/";
|
|
|
|
|
|
{
|
|
@@ -108,7 +111,7 @@ public class TestFileCreation {
|
|
|
static final int numBlocks = 2;
|
|
|
static final int fileSize = numBlocks * blockSize + 1;
|
|
|
boolean simulatedStorage = false;
|
|
|
-
|
|
|
+
|
|
|
private static final String[] NON_CANONICAL_PATHS = new String[] {
|
|
|
"//foo",
|
|
|
"///foo2",
|
|
@@ -121,7 +124,7 @@ public class TestFileCreation {
|
|
|
// creates a file but does not close it
|
|
|
public static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
|
|
|
throws IOException {
|
|
|
- System.out.println("createFile: Created " + name + " with " + repl + " replica.");
|
|
|
+ LOG.info("createFile: Created " + name + " with " + repl + " replica.");
|
|
|
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
|
|
|
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
|
|
|
(short) repl, blockSize);
|
|
@@ -194,8 +197,8 @@ public class TestFileCreation {
|
|
|
public void testFileCreationSetLocalInterface() throws IOException {
|
|
|
assumeTrue(System.getProperty("os.name").startsWith("Linux"));
|
|
|
|
|
|
- // The mini cluster listens on the loopback so we can use it here
|
|
|
- checkFileCreation("lo", false);
|
|
|
+ // Use wildcard address to force interface to be used
|
|
|
+ checkFileCreation("0.0.0.0", false);
|
|
|
|
|
|
try {
|
|
|
checkFileCreation("bogus-interface", false);
|
|
@@ -212,6 +215,7 @@ public class TestFileCreation {
|
|
|
*/
|
|
|
public void checkFileCreation(String netIf, boolean useDnHostname)
|
|
|
throws IOException {
|
|
|
+
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
if (netIf != null) {
|
|
|
conf.set(HdfsClientConfigKeys.DFS_CLIENT_LOCAL_INTERFACES, netIf);
|
|
@@ -237,9 +241,9 @@ public class TestFileCreation {
|
|
|
// check that / exists
|
|
|
//
|
|
|
Path path = new Path("/");
|
|
|
- System.out.println("Path : \"" + path.toString() + "\"");
|
|
|
- System.out.println(fs.getFileStatus(path).isDirectory());
|
|
|
- assertTrue("/ should be a directory",
|
|
|
+ LOG.info("Path : \"" + path.toString() + "\"");
|
|
|
+ LOG.info(fs.getFileStatus(path).isDirectory());
|
|
|
+ assertTrue("/ should be a directory",
|
|
|
fs.getFileStatus(path).isDirectory());
|
|
|
|
|
|
//
|
|
@@ -247,7 +251,7 @@ public class TestFileCreation {
|
|
|
//
|
|
|
Path dir1 = new Path("/test_dir");
|
|
|
fs.mkdirs(dir1);
|
|
|
- System.out.println("createFile: Creating " + dir1.getName() +
|
|
|
+ LOG.info("createFile: Creating " + dir1.getName() +
|
|
|
" for overwrite of existing directory.");
|
|
|
try {
|
|
|
fs.create(dir1, true); // Create path, overwrite=true
|
|
@@ -268,9 +272,9 @@ public class TestFileCreation {
|
|
|
FSDataOutputStream stm = createFile(fs, file1, 1);
|
|
|
|
|
|
// verify that file exists in FS namespace
|
|
|
- assertTrue(file1 + " should be a file",
|
|
|
+ assertTrue(file1 + " should be a file",
|
|
|
fs.getFileStatus(file1).isFile());
|
|
|
- System.out.println("Path : \"" + file1 + "\"");
|
|
|
+ LOG.info("Path : \"" + file1 + "\"");
|
|
|
|
|
|
// write to file
|
|
|
writeFile(stm);
|
|
@@ -282,13 +286,13 @@ public class TestFileCreation {
|
|
|
assertTrue(file1 + " should be of size " + fileSize +
|
|
|
" but found to be of size " + len,
|
|
|
len == fileSize);
|
|
|
-
|
|
|
+
|
|
|
// verify the disk space the file occupied
|
|
|
long diskSpace = dfs.getContentSummary(file1.getParent()).getLength();
|
|
|
assertEquals(file1 + " should take " + fileSize + " bytes disk space " +
|
|
|
"but found to take " + diskSpace + " bytes", fileSize, diskSpace);
|
|
|
-
|
|
|
- // Check storage usage
|
|
|
+
|
|
|
+ // Check storage usage
|
|
|
// can't check capacities for real storage since the OS file system may be changing under us.
|
|
|
if (simulatedStorage) {
|
|
|
DataNode dn = cluster.getDataNodes().get(0);
|
|
@@ -325,7 +329,7 @@ public class TestFileCreation {
|
|
|
FSDataOutputStream stm1 = createFile(fs, file1, 1);
|
|
|
FSDataOutputStream stm2 = createFile(fs, file2, 1);
|
|
|
FSDataOutputStream stm3 = createFile(localfs, file3, 1);
|
|
|
- System.out.println("DeleteOnExit: Created files.");
|
|
|
+ LOG.info("DeleteOnExit: Created files.");
|
|
|
|
|
|
// write to files and close. Purposely, do not close file2.
|
|
|
writeFile(stm1);
|
|
@@ -356,7 +360,7 @@ public class TestFileCreation {
|
|
|
!fs.exists(file2));
|
|
|
assertTrue(file3 + " still exists inspite of deletOnExit set.",
|
|
|
!localfs.exists(file3));
|
|
|
- System.out.println("DeleteOnExit successful.");
|
|
|
+ LOG.info("DeleteOnExit successful.");
|
|
|
|
|
|
} finally {
|
|
|
IOUtils.closeStream(fs);
|
|
@@ -453,7 +457,7 @@ public class TestFileCreation {
|
|
|
// verify that file exists in FS namespace
|
|
|
assertTrue(file1 + " should be a file",
|
|
|
fs.getFileStatus(file1).isFile());
|
|
|
- System.out.println("Path : \"" + file1 + "\"");
|
|
|
+ LOG.info("Path : \"" + file1 + "\"");
|
|
|
|
|
|
// kill the datanode
|
|
|
cluster.shutdownDataNodes();
|
|
@@ -465,7 +469,7 @@ public class TestFileCreation {
|
|
|
if (info.length == 0) {
|
|
|
break;
|
|
|
}
|
|
|
- System.out.println("testFileCreationError1: waiting for datanode " +
|
|
|
+ LOG.info("testFileCreationError1: waiting for datanode " +
|
|
|
" to die.");
|
|
|
try {
|
|
|
Thread.sleep(1000);
|
|
@@ -480,14 +484,14 @@ public class TestFileCreation {
|
|
|
stm.write(buffer);
|
|
|
stm.close();
|
|
|
} catch (Exception e) {
|
|
|
- System.out.println("Encountered expected exception");
|
|
|
+ LOG.info("Encountered expected exception");
|
|
|
}
|
|
|
|
|
|
// verify that no blocks are associated with this file
|
|
|
// bad block allocations were cleaned up earlier.
|
|
|
LocatedBlocks locations = client.getNamenode().getBlockLocations(
|
|
|
file1.toString(), 0, Long.MAX_VALUE);
|
|
|
- System.out.println("locations = " + locations.locatedBlockCount());
|
|
|
+ LOG.info("locations = " + locations.locatedBlockCount());
|
|
|
assertTrue("Error blocks were not cleaned up",
|
|
|
locations.locatedBlockCount() == 0);
|
|
|
} finally {
|
|
@@ -503,7 +507,7 @@ public class TestFileCreation {
|
|
|
@Test
|
|
|
public void testFileCreationError2() throws IOException {
|
|
|
long leasePeriod = 1000;
|
|
|
- System.out.println("testFileCreationError2 start");
|
|
|
+ LOG.info("testFileCreationError2 start");
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
|
|
|
conf.setInt(DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
|
@@ -522,24 +526,24 @@ public class TestFileCreation {
|
|
|
//
|
|
|
Path file1 = new Path("/filestatus.dat");
|
|
|
createFile(dfs, file1, 1);
|
|
|
- System.out.println("testFileCreationError2: "
|
|
|
+ LOG.info("testFileCreationError2: "
|
|
|
+ "Created file filestatus.dat with one replicas.");
|
|
|
|
|
|
LocatedBlocks locations = client.getNamenode().getBlockLocations(
|
|
|
file1.toString(), 0, Long.MAX_VALUE);
|
|
|
- System.out.println("testFileCreationError2: "
|
|
|
+ LOG.info("testFileCreationError2: "
|
|
|
+ "The file has " + locations.locatedBlockCount() + " blocks.");
|
|
|
|
|
|
// add one block to the file
|
|
|
LocatedBlock location = client.getNamenode().addBlock(file1.toString(),
|
|
|
client.clientName, null, null, HdfsConstants.GRANDFATHER_INODE_ID, null);
|
|
|
- System.out.println("testFileCreationError2: "
|
|
|
+ LOG.info("testFileCreationError2: "
|
|
|
+ "Added block " + location.getBlock());
|
|
|
|
|
|
locations = client.getNamenode().getBlockLocations(file1.toString(),
|
|
|
0, Long.MAX_VALUE);
|
|
|
int count = locations.locatedBlockCount();
|
|
|
- System.out.println("testFileCreationError2: "
|
|
|
+ LOG.info("testFileCreationError2: "
|
|
|
+ "The file now has " + count + " blocks.");
|
|
|
|
|
|
// set the soft and hard limit to be 1 second so that the
|
|
@@ -555,10 +559,10 @@ public class TestFileCreation {
|
|
|
// verify that the last block was synchronized.
|
|
|
locations = client.getNamenode().getBlockLocations(file1.toString(),
|
|
|
0, Long.MAX_VALUE);
|
|
|
- System.out.println("testFileCreationError2: "
|
|
|
+ LOG.info("testFileCreationError2: "
|
|
|
+ "locations = " + locations.locatedBlockCount());
|
|
|
assertEquals(0, locations.locatedBlockCount());
|
|
|
- System.out.println("testFileCreationError2 successful");
|
|
|
+ LOG.info("testFileCreationError2 successful");
|
|
|
} finally {
|
|
|
IOUtils.closeStream(dfs);
|
|
|
cluster.shutdown();
|
|
@@ -568,7 +572,7 @@ public class TestFileCreation {
|
|
|
/** test addBlock(..) when replication<min and excludeNodes==null. */
|
|
|
@Test
|
|
|
public void testFileCreationError3() throws IOException {
|
|
|
- System.out.println("testFileCreationError3 start");
|
|
|
+ LOG.info("testFileCreationError3 start");
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
// create cluster
|
|
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
|
|
@@ -589,7 +593,7 @@ public class TestFileCreation {
|
|
|
FileSystem.LOG.info("GOOD!", ioe);
|
|
|
}
|
|
|
|
|
|
- System.out.println("testFileCreationError3 successful");
|
|
|
+ LOG.info("testFileCreationError3 successful");
|
|
|
} finally {
|
|
|
IOUtils.closeStream(dfs);
|
|
|
cluster.shutdown();
|
|
@@ -622,7 +626,7 @@ public class TestFileCreation {
|
|
|
// create a new file.
|
|
|
Path file1 = new Path("/filestatus.dat");
|
|
|
HdfsDataOutputStream stm = create(fs, file1, 1);
|
|
|
- System.out.println("testFileCreationNamenodeRestart: "
|
|
|
+ LOG.info("testFileCreationNamenodeRestart: "
|
|
|
+ "Created file " + file1);
|
|
|
assertEquals(file1 + " should be replicated to 1 datanode.", 1,
|
|
|
stm.getCurrentBlockReplication());
|
|
@@ -636,7 +640,7 @@ public class TestFileCreation {
|
|
|
// rename file wile keeping it open.
|
|
|
Path fileRenamed = new Path("/filestatusRenamed.dat");
|
|
|
fs.rename(file1, fileRenamed);
|
|
|
- System.out.println("testFileCreationNamenodeRestart: "
|
|
|
+ LOG.info("testFileCreationNamenodeRestart: "
|
|
|
+ "Renamed file " + file1 + " to " +
|
|
|
fileRenamed);
|
|
|
file1 = fileRenamed;
|
|
@@ -645,7 +649,7 @@ public class TestFileCreation {
|
|
|
//
|
|
|
Path file2 = new Path("/filestatus2.dat");
|
|
|
FSDataOutputStream stm2 = createFile(fs, file2, 1);
|
|
|
- System.out.println("testFileCreationNamenodeRestart: "
|
|
|
+ LOG.info("testFileCreationNamenodeRestart: "
|
|
|
+ "Created file " + file2);
|
|
|
|
|
|
// create yet another new file with full path name.
|
|
@@ -653,21 +657,21 @@ public class TestFileCreation {
|
|
|
//
|
|
|
Path file3 = new Path("/user/home/fullpath.dat");
|
|
|
FSDataOutputStream stm3 = createFile(fs, file3, 1);
|
|
|
- System.out.println("testFileCreationNamenodeRestart: "
|
|
|
+ LOG.info("testFileCreationNamenodeRestart: "
|
|
|
+ "Created file " + file3);
|
|
|
Path file4 = new Path("/user/home/fullpath4.dat");
|
|
|
FSDataOutputStream stm4 = createFile(fs, file4, 1);
|
|
|
- System.out.println("testFileCreationNamenodeRestart: "
|
|
|
+ LOG.info("testFileCreationNamenodeRestart: "
|
|
|
+ "Created file " + file4);
|
|
|
|
|
|
fs.mkdirs(new Path("/bin"));
|
|
|
fs.rename(new Path("/user/home"), new Path("/bin"));
|
|
|
Path file3new = new Path("/bin/home/fullpath.dat");
|
|
|
- System.out.println("testFileCreationNamenodeRestart: "
|
|
|
+ LOG.info("testFileCreationNamenodeRestart: "
|
|
|
+ "Renamed file " + file3 + " to " +
|
|
|
file3new);
|
|
|
Path file4new = new Path("/bin/home/fullpath4.dat");
|
|
|
- System.out.println("testFileCreationNamenodeRestart: "
|
|
|
+ LOG.info("testFileCreationNamenodeRestart: "
|
|
|
+ "Renamed file " + file4 + " to " +
|
|
|
file4new);
|
|
|
|
|
@@ -727,14 +731,14 @@ public class TestFileCreation {
|
|
|
DFSClient client = fs.dfs;
|
|
|
LocatedBlocks locations = client.getNamenode().getBlockLocations(
|
|
|
file1.toString(), 0, Long.MAX_VALUE);
|
|
|
- System.out.println("locations = " + locations.locatedBlockCount());
|
|
|
+ LOG.info("locations = " + locations.locatedBlockCount());
|
|
|
assertTrue("Error blocks were not cleaned up for file " + file1,
|
|
|
locations.locatedBlockCount() == 3);
|
|
|
|
|
|
// verify filestatus2.dat
|
|
|
locations = client.getNamenode().getBlockLocations(
|
|
|
file2.toString(), 0, Long.MAX_VALUE);
|
|
|
- System.out.println("locations = " + locations.locatedBlockCount());
|
|
|
+ LOG.info("locations = " + locations.locatedBlockCount());
|
|
|
assertTrue("Error blocks were not cleaned up for file " + file2,
|
|
|
locations.locatedBlockCount() == 1);
|
|
|
} finally {
|
|
@@ -749,7 +753,7 @@ public class TestFileCreation {
|
|
|
@Test
|
|
|
public void testDFSClientDeath() throws IOException, InterruptedException {
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
- System.out.println("Testing adbornal client death.");
|
|
|
+ LOG.info("Testing adbornal client death.");
|
|
|
if (simulatedStorage) {
|
|
|
SimulatedFSDataset.setFactory(conf);
|
|
|
}
|
|
@@ -763,7 +767,7 @@ public class TestFileCreation {
|
|
|
//
|
|
|
Path file1 = new Path("/clienttest.dat");
|
|
|
FSDataOutputStream stm = createFile(fs, file1, 1);
|
|
|
- System.out.println("Created file clienttest.dat");
|
|
|
+ LOG.info("Created file clienttest.dat");
|
|
|
|
|
|
// write to file
|
|
|
writeFile(stm);
|
|
@@ -779,7 +783,7 @@ public class TestFileCreation {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Test file creation using createNonRecursive().
|
|
|
*/
|
|
@@ -861,7 +865,7 @@ public class TestFileCreation {
|
|
|
static IOException createNonRecursive(FileSystem fs, Path name,
|
|
|
int repl, EnumSet<CreateFlag> flag) throws IOException {
|
|
|
try {
|
|
|
- System.out.println("createNonRecursive: Attempting to create " + name +
|
|
|
+ LOG.info("createNonRecursive: Attempting to create " + name +
|
|
|
" with " + repl + " replica.");
|
|
|
int bufferSize = fs.getConf()
|
|
|
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
|
|
@@ -895,9 +899,9 @@ public class TestFileCreation {
|
|
|
|
|
|
try {
|
|
|
FileSystem fs = cluster.getFileSystem();
|
|
|
-
|
|
|
+
|
|
|
Path[] p = {new Path("/foo"), new Path("/bar")};
|
|
|
-
|
|
|
+
|
|
|
//write 2 files at the same time
|
|
|
FSDataOutputStream[] out = {fs.create(p[0]), fs.create(p[1])};
|
|
|
int i = 0;
|
|
@@ -929,9 +933,9 @@ public class TestFileCreation {
|
|
|
|
|
|
try {
|
|
|
FileSystem fs = cluster.getFileSystem();
|
|
|
-
|
|
|
+
|
|
|
Path[] p = {new Path("/foo"), new Path("/bar")};
|
|
|
-
|
|
|
+
|
|
|
//write 2 files at the same time
|
|
|
FSDataOutputStream[] out = {fs.create(p[0]), fs.create(p[1])};
|
|
|
int i = 0;
|
|
@@ -959,7 +963,7 @@ public class TestFileCreation {
|
|
|
*/
|
|
|
@Test
|
|
|
public void testLeaseExpireHardLimit() throws Exception {
|
|
|
- System.out.println("testLeaseExpireHardLimit start");
|
|
|
+ LOG.info("testLeaseExpireHardLimit start");
|
|
|
final long leasePeriod = 1000;
|
|
|
final int DATANODE_NUM = 3;
|
|
|
|
|
@@ -1004,20 +1008,20 @@ public class TestFileCreation {
|
|
|
successcount++;
|
|
|
}
|
|
|
}
|
|
|
- System.out.println("successcount=" + successcount);
|
|
|
- assertTrue(successcount > 0);
|
|
|
+ LOG.info("successcount=" + successcount);
|
|
|
+ assertTrue(successcount > 0);
|
|
|
} finally {
|
|
|
IOUtils.closeStream(dfs);
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
|
|
|
- System.out.println("testLeaseExpireHardLimit successful");
|
|
|
+ LOG.info("testLeaseExpireHardLimit successful");
|
|
|
}
|
|
|
|
|
|
// test closing file system before all file handles are closed.
|
|
|
@Test
|
|
|
public void testFsClose() throws Exception {
|
|
|
- System.out.println("test file system close start");
|
|
|
+ LOG.info("test file system close start");
|
|
|
final int DATANODE_NUM = 3;
|
|
|
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
@@ -1038,7 +1042,7 @@ public class TestFileCreation {
|
|
|
// close file system without closing file
|
|
|
dfs.close();
|
|
|
} finally {
|
|
|
- System.out.println("testFsClose successful");
|
|
|
+ LOG.info("testFsClose successful");
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
@@ -1046,7 +1050,7 @@ public class TestFileCreation {
|
|
|
// test closing file after cluster is shutdown
|
|
|
@Test
|
|
|
public void testFsCloseAfterClusterShutdown() throws IOException {
|
|
|
- System.out.println("test testFsCloseAfterClusterShutdown start");
|
|
|
+ LOG.info("test testFsCloseAfterClusterShutdown start");
|
|
|
final int DATANODE_NUM = 3;
|
|
|
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
@@ -1077,13 +1081,13 @@ public class TestFileCreation {
|
|
|
boolean hasException = false;
|
|
|
try {
|
|
|
out.close();
|
|
|
- System.out.println("testFsCloseAfterClusterShutdown: Error here");
|
|
|
+ LOG.info("testFsCloseAfterClusterShutdown: Error here");
|
|
|
} catch (IOException e) {
|
|
|
hasException = true;
|
|
|
}
|
|
|
assertTrue("Failed to close file after cluster shutdown", hasException);
|
|
|
} finally {
|
|
|
- System.out.println("testFsCloseAfterClusterShutdown successful");
|
|
|
+ LOG.info("testFsCloseAfterClusterShutdown successful");
|
|
|
if (cluster != null) {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
@@ -1102,7 +1106,7 @@ public class TestFileCreation {
|
|
|
public void testCreateNonCanonicalPathAndRestartRpc() throws Exception {
|
|
|
doCreateTest(CreationMethod.DIRECT_NN_RPC);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Another regression test for HDFS-3626. This one creates files using
|
|
|
* a Path instantiated from a string object.
|
|
@@ -1122,7 +1126,7 @@ public class TestFileCreation {
|
|
|
throws Exception {
|
|
|
doCreateTest(CreationMethod.PATH_FROM_URI);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private static enum CreationMethod {
|
|
|
DIRECT_NN_RPC,
|
|
|
PATH_FROM_URI,
|
|
@@ -1137,7 +1141,7 @@ public class TestFileCreation {
|
|
|
NamenodeProtocols nnrpc = cluster.getNameNodeRpc();
|
|
|
|
|
|
for (String pathStr : NON_CANONICAL_PATHS) {
|
|
|
- System.out.println("Creating " + pathStr + " by " + method);
|
|
|
+ LOG.info("Creating " + pathStr + " by " + method);
|
|
|
switch (method) {
|
|
|
case DIRECT_NN_RPC:
|
|
|
try {
|
|
@@ -1152,7 +1156,7 @@ public class TestFileCreation {
|
|
|
// So, we expect all of them to fail.
|
|
|
}
|
|
|
break;
|
|
|
-
|
|
|
+
|
|
|
case PATH_FROM_URI:
|
|
|
case PATH_FROM_STRING:
|
|
|
// Unlike the above direct-to-NN case, we expect these to succeed,
|
|
@@ -1170,7 +1174,7 @@ public class TestFileCreation {
|
|
|
throw new AssertionError("bad method: " + method);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
cluster.restartNameNode();
|
|
|
|
|
|
} finally {
|
|
@@ -1227,13 +1231,13 @@ public class TestFileCreation {
|
|
|
dfs.mkdirs(new Path("/foo/dir"));
|
|
|
String file = "/foo/dir/file";
|
|
|
Path filePath = new Path(file);
|
|
|
-
|
|
|
+
|
|
|
// Case 1: Create file with overwrite, check the blocks of old file
|
|
|
// are cleaned after creating with overwrite
|
|
|
NameNode nn = cluster.getNameNode();
|
|
|
FSNamesystem fsn = NameNodeAdapter.getNamesystem(nn);
|
|
|
BlockManager bm = fsn.getBlockManager();
|
|
|
-
|
|
|
+
|
|
|
FSDataOutputStream out = dfs.create(filePath);
|
|
|
byte[] oldData = AppendTestUtil.randomBytes(seed, fileSize);
|
|
|
try {
|
|
@@ -1241,11 +1245,11 @@ public class TestFileCreation {
|
|
|
} finally {
|
|
|
out.close();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
LocatedBlocks oldBlocks = NameNodeAdapter.getBlockLocations(
|
|
|
nn, file, 0, fileSize);
|
|
|
assertBlocks(bm, oldBlocks, true);
|
|
|
-
|
|
|
+
|
|
|
out = dfs.create(filePath, true);
|
|
|
byte[] newData = AppendTestUtil.randomBytes(seed, fileSize);
|
|
|
try {
|
|
@@ -1254,12 +1258,12 @@ public class TestFileCreation {
|
|
|
out.close();
|
|
|
}
|
|
|
dfs.deleteOnExit(filePath);
|
|
|
-
|
|
|
+
|
|
|
LocatedBlocks newBlocks = NameNodeAdapter.getBlockLocations(
|
|
|
nn, file, 0, fileSize);
|
|
|
assertBlocks(bm, newBlocks, true);
|
|
|
assertBlocks(bm, oldBlocks, false);
|
|
|
-
|
|
|
+
|
|
|
FSDataInputStream in = dfs.open(filePath);
|
|
|
byte[] result = null;
|
|
|
try {
|
|
@@ -1268,7 +1272,7 @@ public class TestFileCreation {
|
|
|
in.close();
|
|
|
}
|
|
|
Assert.assertArrayEquals(newData, result);
|
|
|
-
|
|
|
+
|
|
|
// Case 2: Restart NN, check the file
|
|
|
cluster.restartNameNode();
|
|
|
nn = cluster.getNameNode();
|
|
@@ -1279,13 +1283,13 @@ public class TestFileCreation {
|
|
|
in.close();
|
|
|
}
|
|
|
Assert.assertArrayEquals(newData, result);
|
|
|
-
|
|
|
+
|
|
|
// Case 3: Save new checkpoint and restart NN, check the file
|
|
|
NameNodeAdapter.enterSafeMode(nn, false);
|
|
|
NameNodeAdapter.saveNamespace(nn);
|
|
|
cluster.restartNameNode();
|
|
|
nn = cluster.getNameNode();
|
|
|
-
|
|
|
+
|
|
|
in = dfs.open(filePath);
|
|
|
try {
|
|
|
result = readAll(in);
|
|
@@ -1302,8 +1306,8 @@ public class TestFileCreation {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- private void assertBlocks(BlockManager bm, LocatedBlocks lbs,
|
|
|
+
|
|
|
+ private void assertBlocks(BlockManager bm, LocatedBlocks lbs,
|
|
|
boolean exist) {
|
|
|
for (LocatedBlock locatedBlock : lbs.getLocatedBlocks()) {
|
|
|
if (exist) {
|
|
@@ -1315,7 +1319,7 @@ public class TestFileCreation {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private byte[] readAll(FSDataInputStream in) throws IOException {
|
|
|
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
|
|
byte[] buffer = new byte[1024];
|