HDFS-3995. Use DFSTestUtil.createFile() for file creation and writing in test cases. Contributed by Jing Zhao.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1394376 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Suresh Srinivas 2012-10-05 06:22:26 +00:00
parent 6582da034b
commit a41f808c3c
19 changed files with 127 additions and 265 deletions

View File

@ -140,6 +140,9 @@ Trunk (Unreleased)
HDFS-2127. Add a test that ensure AccessControlExceptions contain HDFS-2127. Add a test that ensure AccessControlExceptions contain
a full path. (Stephen Chu via eli) a full path. (Stephen Chu via eli)
HDFS-3995. Use DFSTestUtil.createFile() for file creation and
writing in test cases. (Jing Zhao via suresh)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES

View File

@ -210,27 +210,40 @@ public class DFSTestUtil {
public static void createFile(FileSystem fs, Path fileName, long fileLen, public static void createFile(FileSystem fs, Path fileName, long fileLen,
short replFactor, long seed) throws IOException { short replFactor, long seed) throws IOException {
createFile(fs, fileName, 1024, fileLen, fs.getDefaultBlockSize(fileName),
replFactor, seed);
}
public static void createFile(FileSystem fs, Path fileName, int bufferLen,
long fileLen, long blockSize, short replFactor, long seed)
throws IOException {
assert bufferLen > 0;
if (!fs.mkdirs(fileName.getParent())) { if (!fs.mkdirs(fileName.getParent())) {
throw new IOException("Mkdirs failed to create " + throw new IOException("Mkdirs failed to create " +
fileName.getParent().toString()); fileName.getParent().toString());
} }
FSDataOutputStream out = null; FSDataOutputStream out = null;
try { try {
out = fs.create(fileName, replFactor); out = fs.create(fileName, true, fs.getConf()
byte[] toWrite = new byte[1024]; .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
Random rb = new Random(seed); replFactor, blockSize);
long bytesToWrite = fileLen; if (fileLen > 0) {
while (bytesToWrite>0) { byte[] toWrite = new byte[bufferLen];
rb.nextBytes(toWrite); Random rb = new Random(seed);
int bytesToWriteNext = (1024<bytesToWrite)?1024:(int)bytesToWrite; long bytesToWrite = fileLen;
while (bytesToWrite>0) {
out.write(toWrite, 0, bytesToWriteNext); rb.nextBytes(toWrite);
bytesToWrite -= bytesToWriteNext; int bytesToWriteNext = (bufferLen < bytesToWrite) ? bufferLen
: (int) bytesToWrite;
out.write(toWrite, 0, bytesToWriteNext);
bytesToWrite -= bytesToWriteNext;
}
} }
out.close();
out = null;
} finally { } finally {
IOUtils.closeStream(out); if (out != null) {
out.close();
}
} }
} }

View File

@ -37,7 +37,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
@ -141,13 +140,6 @@ public class TestDataTransferProtocol {
} }
} }
void createFile(FileSystem fs, Path path, int fileLen) throws IOException {
byte [] arr = new byte[fileLen];
FSDataOutputStream out = fs.create(path);
out.write(arr);
out.close();
}
void readFile(FileSystem fs, Path path, int fileLen) throws IOException { void readFile(FileSystem fs, Path path, int fileLen) throws IOException {
byte [] arr = new byte[fileLen]; byte [] arr = new byte[fileLen];
FSDataInputStream in = fs.open(path); FSDataInputStream in = fs.open(path);
@ -357,7 +349,9 @@ public class TestDataTransferProtocol {
int fileLen = Math.min(conf.getInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096), 4096); int fileLen = Math.min(conf.getInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096), 4096);
createFile(fileSys, file, fileLen); DFSTestUtil.createFile(fileSys, file, fileLen, fileLen,
fileSys.getDefaultBlockSize(file),
fileSys.getDefaultReplication(file), 0L);
// get the first blockid for the file // get the first blockid for the file
final ExtendedBlock firstBlock = DFSTestUtil.getFirstBlock(fileSys, file); final ExtendedBlock firstBlock = DFSTestUtil.getFirstBlock(fileSys, file);

View File

@ -79,7 +79,8 @@ public class TestFileStatus {
hftpfs = cluster.getHftpFileSystem(0); hftpfs = cluster.getHftpFileSystem(0);
dfsClient = new DFSClient(NameNode.getAddress(conf), conf); dfsClient = new DFSClient(NameNode.getAddress(conf), conf);
file1 = new Path("filestatus.dat"); file1 = new Path("filestatus.dat");
writeFile(fs, file1, 1, fileSize, blockSize); DFSTestUtil.createFile(fs, file1, fileSize, fileSize, blockSize, (short) 1,
seed);
} }
@AfterClass @AfterClass
@ -87,18 +88,6 @@ public class TestFileStatus {
fs.close(); fs.close();
cluster.shutdown(); cluster.shutdown();
} }
private static void writeFile(FileSystem fileSys, Path name, int repl,
int fileSize, int blockSize) throws IOException {
// Create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, true,
HdfsConstants.IO_FILE_BUFFER_SIZE, (short)repl, (long)blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
private void checkFile(FileSystem fileSys, Path name, int repl) private void checkFile(FileSystem fileSys, Path name, int repl)
throws IOException, InterruptedException, TimeoutException { throws IOException, InterruptedException, TimeoutException {
@ -218,7 +207,8 @@ public class TestFileStatus {
// create another file that is smaller than a block. // create another file that is smaller than a block.
Path file2 = new Path(dir, "filestatus2.dat"); Path file2 = new Path(dir, "filestatus2.dat");
writeFile(fs, file2, 1, blockSize/4, blockSize); DFSTestUtil.createFile(fs, file2, blockSize/4, blockSize/4, blockSize,
(short) 1, seed);
checkFile(fs, file2, 1); checkFile(fs, file2, 1);
// verify file attributes // verify file attributes
@ -230,7 +220,8 @@ public class TestFileStatus {
// Create another file in the same directory // Create another file in the same directory
Path file3 = new Path(dir, "filestatus3.dat"); Path file3 = new Path(dir, "filestatus3.dat");
writeFile(fs, file3, 1, blockSize/4, blockSize); DFSTestUtil.createFile(fs, file3, blockSize/4, blockSize/4, blockSize,
(short) 1, seed);
checkFile(fs, file3, 1); checkFile(fs, file3, 1);
file3 = fs.makeQualified(file3); file3 = fs.makeQualified(file3);

View File

@ -110,9 +110,7 @@ public class TestGetBlocks {
// do the writing but do not close the FSDataOutputStream // do the writing but do not close the FSDataOutputStream
// in order to mimic the ongoing writing // in order to mimic the ongoing writing
final Path fileName = new Path("/file1"); final Path fileName = new Path("/file1");
stm = fileSys.create( stm = fileSys.create(fileName, true,
fileName,
true,
fileSys.getConf().getInt( fileSys.getConf().getInt(
CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) 3, blockSize); (short) 3, blockSize);
@ -180,29 +178,15 @@ public class TestGetBlocks {
final short REPLICATION_FACTOR = (short) 2; final short REPLICATION_FACTOR = (short) 2;
final int DEFAULT_BLOCK_SIZE = 1024; final int DEFAULT_BLOCK_SIZE = 1024;
final Random r = new Random();
CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE); CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes( MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(
REPLICATION_FACTOR).build(); REPLICATION_FACTOR).build();
try { try {
cluster.waitActive(); cluster.waitActive();
// create a file with two blocks
FileSystem fs = cluster.getFileSystem();
FSDataOutputStream out = fs.create(new Path("/tmp.txt"),
REPLICATION_FACTOR);
byte[] data = new byte[1024];
long fileLen = 2 * DEFAULT_BLOCK_SIZE; long fileLen = 2 * DEFAULT_BLOCK_SIZE;
long bytesToWrite = fileLen; DFSTestUtil.createFile(cluster.getFileSystem(), new Path("/tmp.txt"),
while (bytesToWrite > 0) { fileLen, REPLICATION_FACTOR, 0L);
r.nextBytes(data);
int bytesToWriteNext = (1024 < bytesToWrite) ? 1024
: (int) bytesToWrite;
out.write(data, 0, bytesToWriteNext);
bytesToWrite -= bytesToWriteNext;
}
out.close();
// get blocks & data nodes // get blocks & data nodes
List<LocatedBlock> locatedBlocks; List<LocatedBlock> locatedBlocks;

View File

@ -52,22 +52,6 @@ public class TestInjectionForSimulatedStorage {
private static final Log LOG = LogFactory.getLog( private static final Log LOG = LogFactory.getLog(
"org.apache.hadoop.hdfs.TestInjectionForSimulatedStorage"); "org.apache.hadoop.hdfs.TestInjectionForSimulatedStorage");
private void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
// create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) repl, blockSize);
byte[] buffer = new byte[filesize];
for (int i=0; i<buffer.length; i++) {
buffer[i] = '1';
}
stm.write(buffer);
stm.close();
}
// Waits for all of the blocks to have expected replication
// Waits for all of the blocks to have expected replication // Waits for all of the blocks to have expected replication
private void waitForBlockReplication(String filename, private void waitForBlockReplication(String filename,
@ -149,7 +133,8 @@ public class TestInjectionForSimulatedStorage {
cluster.getNameNodePort()), cluster.getNameNodePort()),
conf); conf);
writeFile(cluster.getFileSystem(), testPath, numDataNodes); DFSTestUtil.createFile(cluster.getFileSystem(), testPath, filesize,
filesize, blockSize, (short) numDataNodes, 0L);
waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, 20); waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, 20);
Iterable<Block>[] blocksList = cluster.getAllBlockReports(bpid); Iterable<Block>[] blocksList = cluster.getAllBlockReports(bpid);

View File

@ -50,19 +50,6 @@ public class TestModTime {
Random myrand = new Random(); Random myrand = new Random();
Path hostsFile; Path hostsFile;
Path excludeFile; Path excludeFile;
private void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
// create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) repl, blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
private void cleanupFile(FileSystem fileSys, Path name) throws IOException { private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
assertTrue(fileSys.exists(name)); assertTrue(fileSys.exists(name));
@ -105,7 +92,8 @@ public class TestModTime {
System.out.println("Creating testdir1 and testdir1/test1.dat."); System.out.println("Creating testdir1 and testdir1/test1.dat.");
Path dir1 = new Path("testdir1"); Path dir1 = new Path("testdir1");
Path file1 = new Path(dir1, "test1.dat"); Path file1 = new Path(dir1, "test1.dat");
writeFile(fileSys, file1, replicas); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
(short) replicas, seed);
FileStatus stat = fileSys.getFileStatus(file1); FileStatus stat = fileSys.getFileStatus(file1);
long mtime1 = stat.getModificationTime(); long mtime1 = stat.getModificationTime();
assertTrue(mtime1 != 0); assertTrue(mtime1 != 0);
@ -120,7 +108,8 @@ public class TestModTime {
// //
System.out.println("Creating testdir1/test2.dat."); System.out.println("Creating testdir1/test2.dat.");
Path file2 = new Path(dir1, "test2.dat"); Path file2 = new Path(dir1, "test2.dat");
writeFile(fileSys, file2, replicas); DFSTestUtil.createFile(fileSys, file2, fileSize, fileSize, blockSize,
(short) replicas, seed);
stat = fileSys.getFileStatus(file2); stat = fileSys.getFileStatus(file2);
// //

View File

@ -41,11 +41,9 @@ public class TestPread {
boolean simulatedStorage = false; boolean simulatedStorage = false;
private void writeFile(FileSystem fileSys, Path name) throws IOException { private void writeFile(FileSystem fileSys, Path name) throws IOException {
// create and write a file that contains three blocks of data
DataOutputStream stm = fileSys.create(name, true, 4096, (short)1,
blockSize);
// test empty file open and read // test empty file open and read
stm.close(); DFSTestUtil.createFile(fileSys, name, 12 * blockSize, 0,
blockSize, (short) 1, seed);
FSDataInputStream in = fileSys.open(name); FSDataInputStream in = fileSys.open(name);
byte[] buffer = new byte[12 * blockSize]; byte[] buffer = new byte[12 * blockSize];
in.readFully(0, buffer, 0, 0); in.readFully(0, buffer, 0, 0);
@ -62,11 +60,8 @@ public class TestPread {
assertTrue("Cannot delete file", false); assertTrue("Cannot delete file", false);
// now create the real file // now create the real file
stm = fileSys.create(name, true, 4096, (short)1, blockSize); DFSTestUtil.createFile(fileSys, name, 12 * blockSize, 12 * blockSize,
Random rand = new Random(seed); blockSize, (short) 1, seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
} }
private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) { private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {

View File

@ -26,15 +26,12 @@ import java.io.OutputStream;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.Iterator; import java.util.Iterator;
import java.util.Random;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.CommonConfigurationKeys;
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.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -61,19 +58,6 @@ public class TestReplication {
private static final int numDatanodes = racks.length; private static final int numDatanodes = racks.length;
private static final Log LOG = LogFactory.getLog( private static final Log LOG = LogFactory.getLog(
"org.apache.hadoop.hdfs.TestReplication"); "org.apache.hadoop.hdfs.TestReplication");
private void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
// create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) repl, blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
/* check if there are at least two nodes are on the same rack */ /* check if there are at least two nodes are on the same rack */
private void checkFile(FileSystem fileSys, Path name, int repl) private void checkFile(FileSystem fileSys, Path name, int repl)
@ -222,19 +206,25 @@ public class TestReplication {
FileSystem fileSys = cluster.getFileSystem(); FileSystem fileSys = cluster.getFileSystem();
try { try {
Path file1 = new Path("/smallblocktest.dat"); Path file1 = new Path("/smallblocktest.dat");
writeFile(fileSys, file1, 3); //writeFile(fileSys, file1, 3);
DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
(short) 3, seed);
checkFile(fileSys, file1, 3); checkFile(fileSys, file1, 3);
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
writeFile(fileSys, file1, 10); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
(short) 10, seed);
checkFile(fileSys, file1, 10); checkFile(fileSys, file1, 10);
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
writeFile(fileSys, file1, 4); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
(short) 4, seed);
checkFile(fileSys, file1, 4); checkFile(fileSys, file1, 4);
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
writeFile(fileSys, file1, 1); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
(short) 1, seed);
checkFile(fileSys, file1, 1); checkFile(fileSys, file1, 1);
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
writeFile(fileSys, file1, 2); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
(short) 2, seed);
checkFile(fileSys, file1, 2); checkFile(fileSys, file1, 2);
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
} finally { } finally {

View File

@ -40,16 +40,6 @@ public class TestSeekBug {
static final long seed = 0xDEADBEEFL; static final long seed = 0xDEADBEEFL;
static final int ONEMB = 1 << 20; static final int ONEMB = 1 << 20;
private void writeFile(FileSystem fileSys, Path name) throws IOException {
// create and write a file that contains 1MB
DataOutputStream stm = fileSys.create(name);
byte[] buffer = new byte[ONEMB];
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) { private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {
for (int idx = 0; idx < actual.length; idx++) { for (int idx = 0; idx < actual.length; idx++) {
assertEquals(message+" byte "+(from+idx)+" differs. expected "+ assertEquals(message+" byte "+(from+idx)+" differs. expected "+
@ -132,7 +122,9 @@ public class TestSeekBug {
FileSystem fileSys = cluster.getFileSystem(); FileSystem fileSys = cluster.getFileSystem();
try { try {
Path file1 = new Path("seektest.dat"); Path file1 = new Path("seektest.dat");
writeFile(fileSys, file1); DFSTestUtil.createFile(fileSys, file1, ONEMB, ONEMB,
fileSys.getDefaultBlockSize(file1),
fileSys.getDefaultReplication(file1), seed);
seekReadFile(fileSys, file1); seekReadFile(fileSys, file1);
smallReadSeek(fileSys, file1); smallReadSeek(fileSys, file1);
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
@ -151,7 +143,9 @@ public class TestSeekBug {
FileSystem fileSys = FileSystem.getLocal(conf); FileSystem fileSys = FileSystem.getLocal(conf);
try { try {
Path file1 = new Path("build/test/data", "seektest.dat"); Path file1 = new Path("build/test/data", "seektest.dat");
writeFile(fileSys, file1); DFSTestUtil.createFile(fileSys, file1, ONEMB, ONEMB,
fileSys.getDefaultBlockSize(file1),
fileSys.getDefaultReplication(file1), seed);
seekReadFile(fileSys, file1); seekReadFile(fileSys, file1);
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
} finally { } finally {

View File

@ -42,18 +42,6 @@ public class TestSmallBlock {
static final int blockSize = 1; static final int blockSize = 1;
static final int fileSize = 20; static final int fileSize = 20;
boolean simulatedStorage = false; boolean simulatedStorage = false;
private void writeFile(FileSystem fileSys, Path name) throws IOException {
// create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) 1, blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) { private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {
for (int idx = 0; idx < actual.length; idx++) { for (int idx = 0; idx < actual.length; idx++) {
@ -105,7 +93,8 @@ public class TestSmallBlock {
FileSystem fileSys = cluster.getFileSystem(); FileSystem fileSys = cluster.getFileSystem();
try { try {
Path file1 = new Path("smallblocktest.dat"); Path file1 = new Path("smallblocktest.dat");
writeFile(fileSys, file1); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
(short) 1, seed);
checkFile(fileSys, file1); checkFile(fileSys, file1);
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
} finally { } finally {

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
@ -64,6 +65,10 @@ public class TestBackupNode {
} }
static final String BASE_DIR = MiniDFSCluster.getBaseDirectory(); static final String BASE_DIR = MiniDFSCluster.getBaseDirectory();
static final long seed = 0xDEADBEEFL;
static final int blockSize = 4096;
static final int fileSize = 8192;
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
@ -350,14 +355,17 @@ public class TestBackupNode {
+ NetUtils.getHostPortString(add)).toUri(), conf); + NetUtils.getHostPortString(add)).toUri(), conf);
boolean canWrite = true; boolean canWrite = true;
try { try {
TestCheckpoint.writeFile(bnFS, file3, replication); DFSTestUtil.createFile(bnFS, file3, fileSize, fileSize, blockSize,
replication, seed);
} catch (IOException eio) { } catch (IOException eio) {
LOG.info("Write to BN failed as expected: ", eio); LOG.info("Write to BN failed as expected: ", eio);
canWrite = false; canWrite = false;
} }
assertFalse("Write to BackupNode must be prohibited.", canWrite); assertFalse("Write to BackupNode must be prohibited.", canWrite);
TestCheckpoint.writeFile(fileSys, file3, replication); DFSTestUtil.createFile(fileSys, file3, fileSize, fileSize, blockSize,
replication, seed);
TestCheckpoint.checkFile(fileSys, file3, replication); TestCheckpoint.checkFile(fileSys, file3, replication);
// should also be on BN right away // should also be on BN right away
assertTrue("file3 does not exist on BackupNode", assertTrue("file3 does not exist on BackupNode",

View File

@ -28,7 +28,6 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import java.io.File; import java.io.File;
import java.io.FilenameFilter;
import java.io.IOException; import java.io.IOException;
import java.lang.management.ManagementFactory; import java.lang.management.ManagementFactory;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
@ -36,14 +35,12 @@ import java.net.URI;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Random;
import org.apache.commons.cli.ParseException; import org.apache.commons.cli.ParseException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger; import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileContext;
@ -51,6 +48,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSUtil; 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;
@ -116,19 +114,6 @@ public class TestCheckpoint {
faultInjector = Mockito.mock(CheckpointFaultInjector.class); faultInjector = Mockito.mock(CheckpointFaultInjector.class);
CheckpointFaultInjector.instance = faultInjector; CheckpointFaultInjector.instance = faultInjector;
} }
static void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) repl, blockSize);
byte[] buffer = new byte[TestCheckpoint.fileSize];
Random rand = new Random(TestCheckpoint.seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
static void checkFile(FileSystem fileSys, Path name, int repl) static void checkFile(FileSystem fileSys, Path name, int repl)
throws IOException { throws IOException {
@ -259,7 +244,8 @@ public class TestCheckpoint {
// //
// Create a new file // Create a new file
// //
writeFile(fileSys, file1, replication); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
replication, seed);
checkFile(fileSys, file1, replication); checkFile(fileSys, file1, replication);
} finally { } finally {
fileSys.close(); fileSys.close();
@ -323,7 +309,8 @@ public class TestCheckpoint {
// //
// Create a new file // Create a new file
// //
writeFile(fileSys, file1, replication); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
replication, seed);
checkFile(fileSys, file1, replication); checkFile(fileSys, file1, replication);
} finally { } finally {
fileSys.close(); fileSys.close();
@ -394,7 +381,8 @@ public class TestCheckpoint {
// //
// Create a new file // Create a new file
// //
writeFile(fileSys, file1, replication); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
replication, seed);
checkFile(fileSys, file1, replication); checkFile(fileSys, file1, replication);
} finally { } finally {
fileSys.close(); fileSys.close();
@ -580,7 +568,8 @@ public class TestCheckpoint {
// //
// Create a new file // Create a new file
// //
writeFile(fileSys, file1, replication); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
replication, seed);
checkFile(fileSys, file1, replication); checkFile(fileSys, file1, replication);
} finally { } finally {
fileSys.close(); fileSys.close();
@ -906,7 +895,8 @@ public class TestCheckpoint {
// //
// Create file1 // Create file1
// //
writeFile(fileSys, file1, replication); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
replication, seed);
checkFile(fileSys, file1, replication); checkFile(fileSys, file1, replication);
// //
@ -933,7 +923,8 @@ public class TestCheckpoint {
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
// create new file file2 // create new file file2
writeFile(fileSys, file2, replication); DFSTestUtil.createFile(fileSys, file2, fileSize, fileSize, blockSize,
replication, seed);
checkFile(fileSys, file2, replication); checkFile(fileSys, file2, replication);
// //
@ -999,7 +990,8 @@ public class TestCheckpoint {
} }
// create new file // create new file
Path file = new Path("namespace.dat"); Path file = new Path("namespace.dat");
writeFile(fs, file, replication); DFSTestUtil.createFile(fs, file, fileSize, fileSize, blockSize,
replication, seed);
checkFile(fs, file, replication); checkFile(fs, file, replication);
// create new link // create new link

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -116,19 +117,6 @@ public class TestDecommissioningStatus {
stm.close(); stm.close();
} }
private void writeFile(FileSystem fileSys, Path name, short repl)
throws IOException {
// create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl,
blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
private FSDataOutputStream writeIncompleteFile(FileSystem fileSys, Path name, private FSDataOutputStream writeIncompleteFile(FileSystem fileSys, Path name,
short repl) throws IOException { short repl) throws IOException {
// create and write a file that contains three blocks of data // create and write a file that contains three blocks of data
@ -198,7 +186,8 @@ public class TestDecommissioningStatus {
// Decommission one node. Verify the decommission status // Decommission one node. Verify the decommission status
// //
Path file1 = new Path("decommission.dat"); Path file1 = new Path("decommission.dat");
writeFile(fileSys, file1, replicas); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
replicas, seed);
Path file2 = new Path("decommission1.dat"); Path file2 = new Path("decommission1.dat");
FSDataOutputStream st1 = writeIncompleteFile(fileSys, file2, replicas); FSDataOutputStream st1 = writeIncompleteFile(fileSys, file2, replicas);

View File

@ -20,14 +20,12 @@ package org.apache.hadoop.hdfs.server.namenode;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.io.IOException; import java.io.IOException;
import java.util.Random;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
@ -43,19 +41,6 @@ public class TestFileLimit {
static final int blockSize = 8192; static final int blockSize = 8192;
boolean simulatedStorage = false; boolean simulatedStorage = false;
// creates a zero file.
private void createFile(FileSystem fileSys, Path name)
throws IOException {
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) 1, blockSize);
byte[] buffer = new byte[1024];
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
private void waitForLimit(FSNamesystem namesys, long num) private void waitForLimit(FSNamesystem namesys, long num)
{ {
// wait for number of blocks to decrease // wait for number of blocks to decrease
@ -106,7 +91,7 @@ public class TestFileLimit {
// //
for (int i = 0; i < maxObjects/2; i++) { for (int i = 0; i < maxObjects/2; i++) {
Path file = new Path("/filestatus" + i); Path file = new Path("/filestatus" + i);
createFile(fs, file); DFSTestUtil.createFile(fs, file, 1024, 1024, blockSize, (short) 1, seed);
System.out.println("Created file " + file); System.out.println("Created file " + file);
currentNodes += 2; // two more objects for this creation. currentNodes += 2; // two more objects for this creation.
} }
@ -115,7 +100,7 @@ public class TestFileLimit {
boolean hitException = false; boolean hitException = false;
try { try {
Path file = new Path("/filestatus"); Path file = new Path("/filestatus");
createFile(fs, file); DFSTestUtil.createFile(fs, file, 1024, 1024, blockSize, (short) 1, seed);
System.out.println("Created file " + file); System.out.println("Created file " + file);
} catch (IOException e) { } catch (IOException e) {
hitException = true; hitException = true;
@ -132,7 +117,7 @@ public class TestFileLimit {
waitForLimit(namesys, currentNodes); waitForLimit(namesys, currentNodes);
// now, we shud be able to create a new file // now, we shud be able to create a new file
createFile(fs, file0); DFSTestUtil.createFile(fs, file0, 1024, 1024, blockSize, (short) 1, seed);
System.out.println("Created file " + file0 + " again."); System.out.println("Created file " + file0 + " again.");
currentNodes += 2; currentNodes += 2;

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -48,17 +49,6 @@ public class TestMetaSave {
private static MiniDFSCluster cluster = null; private static MiniDFSCluster cluster = null;
private static FileSystem fileSys = null; private static FileSystem fileSys = null;
private void createFile(FileSystem fileSys, Path name) throws IOException {
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) 2, blockSize);
byte[] buffer = new byte[1024];
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
@BeforeClass @BeforeClass
public static void setUp() throws IOException { public static void setUp() throws IOException {
// start a cluster // start a cluster
@ -84,7 +74,8 @@ public class TestMetaSave {
for (int i = 0; i < 2; i++) { for (int i = 0; i < 2; i++) {
Path file = new Path("/filestatus" + i); Path file = new Path("/filestatus" + i);
createFile(fileSys, file); DFSTestUtil.createFile(fileSys, file, 1024, 1024, blockSize, (short) 2,
seed);
} }
cluster.stopDataNode(1); cluster.stopDataNode(1);

View File

@ -25,17 +25,15 @@ import static org.junit.Assert.fail;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@ -71,18 +69,6 @@ public class TestNameEditsConfigs {
} }
} }
private void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) repl, BLOCK_SIZE);
byte[] buffer = new byte[FILE_SIZE];
Random rand = new Random(SEED);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
void checkImageAndEditsFilesExistence(File dir, void checkImageAndEditsFilesExistence(File dir,
boolean shouldHaveImages, boolean shouldHaveImages,
boolean shouldHaveEdits) boolean shouldHaveEdits)
@ -187,7 +173,8 @@ public class TestNameEditsConfigs {
try { try {
assertTrue(!fileSys.exists(file1)); assertTrue(!fileSys.exists(file1));
writeFile(fileSys, file1, replication); DFSTestUtil.createFile(fileSys, file1, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
(short) replication, SEED);
checkFile(fileSys, file1, replication); checkFile(fileSys, file1, replication);
secondary.doCheckpoint(); secondary.doCheckpoint();
} finally { } finally {
@ -224,7 +211,8 @@ public class TestNameEditsConfigs {
assertTrue(fileSys.exists(file1)); assertTrue(fileSys.exists(file1));
checkFile(fileSys, file1, replication); checkFile(fileSys, file1, replication);
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
writeFile(fileSys, file2, replication); DFSTestUtil.createFile(fileSys, file2, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
(short) replication, SEED);
checkFile(fileSys, file2, replication); checkFile(fileSys, file2, replication);
secondary.doCheckpoint(); secondary.doCheckpoint();
} finally { } finally {
@ -260,7 +248,8 @@ public class TestNameEditsConfigs {
assertTrue(fileSys.exists(file2)); assertTrue(fileSys.exists(file2));
checkFile(fileSys, file2, replication); checkFile(fileSys, file2, replication);
cleanupFile(fileSys, file2); cleanupFile(fileSys, file2);
writeFile(fileSys, file3, replication); DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
(short) replication, SEED);
checkFile(fileSys, file3, replication); checkFile(fileSys, file3, replication);
secondary.doCheckpoint(); secondary.doCheckpoint();
} finally { } finally {
@ -364,7 +353,8 @@ public class TestNameEditsConfigs {
fileSys = cluster.getFileSystem(); fileSys = cluster.getFileSystem();
assertTrue(!fileSys.exists(file1)); assertTrue(!fileSys.exists(file1));
writeFile(fileSys, file1, replication); DFSTestUtil.createFile(fileSys, file1, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
(short) replication, SEED);
checkFile(fileSys, file1, replication); checkFile(fileSys, file1, replication);
} finally { } finally {
fileSys.close(); fileSys.close();
@ -402,7 +392,8 @@ public class TestNameEditsConfigs {
assertTrue(fileSys.exists(file1)); assertTrue(fileSys.exists(file1));
checkFile(fileSys, file1, replication); checkFile(fileSys, file1, replication);
cleanupFile(fileSys, file1); cleanupFile(fileSys, file1);
writeFile(fileSys, file2, replication); DFSTestUtil.createFile(fileSys, file2, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
(short) replication, SEED);
checkFile(fileSys, file2, replication); checkFile(fileSys, file2, replication);
} finally { } finally {
fileSys.close(); fileSys.close();
@ -429,7 +420,8 @@ public class TestNameEditsConfigs {
assertTrue(fileSys.exists(file2)); assertTrue(fileSys.exists(file2));
checkFile(fileSys, file2, replication); checkFile(fileSys, file2, replication);
cleanupFile(fileSys, file2); cleanupFile(fileSys, file2);
writeFile(fileSys, file3, replication); DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
(short) replication, SEED);
checkFile(fileSys, file3, replication); checkFile(fileSys, file3, replication);
} finally { } finally {
fileSys.close(); fileSys.close();
@ -483,7 +475,8 @@ public class TestNameEditsConfigs {
assertTrue(fileSys.exists(file3)); assertTrue(fileSys.exists(file3));
checkFile(fileSys, file3, replication); checkFile(fileSys, file3, replication);
cleanupFile(fileSys, file3); cleanupFile(fileSys, file3);
writeFile(fileSys, file3, replication); DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
(short) replication, SEED);
checkFile(fileSys, file3, replication); checkFile(fileSys, file3, replication);
} finally { } finally {
fileSys.close(); fileSys.close();

View File

@ -77,20 +77,6 @@ public class TestStartup {
static final int fileSize = 8192; static final int fileSize = 8192;
private long editsLength=0, fsimageLength=0; private long editsLength=0, fsimageLength=0;
private void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) repl, blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
config = new HdfsConfiguration(); config = new HdfsConfiguration();
@ -150,7 +136,8 @@ public class TestStartup {
// create a file // create a file
FileSystem fileSys = cluster.getFileSystem(); FileSystem fileSys = cluster.getFileSystem();
Path file1 = new Path("t1"); Path file1 = new Path("t1");
this.writeFile(fileSys, file1, 1); DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
(short) 1, seed);
LOG.info("--doing checkpoint"); LOG.info("--doing checkpoint");
sn.doCheckpoint(); // this shouldn't fail sn.doCheckpoint(); // this shouldn't fail

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.tools.JMXGet; import org.apache.hadoop.hdfs.tools.JMXGet;
@ -53,19 +54,6 @@ public class TestJMXGet {
static final int blockSize = 4096; static final int blockSize = 4096;
static final int fileSize = 8192; static final int fileSize = 8192;
private void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
FSDataOutputStream stm = fileSys.create(name, true,
fileSys.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short)repl, blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
stm.close();
}
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
config = new HdfsConfiguration(); config = new HdfsConfiguration();
@ -96,7 +84,8 @@ public class TestJMXGet {
cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes).build(); cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes).build();
cluster.waitActive(); cluster.waitActive();
writeFile(cluster.getFileSystem(), new Path("/test1"), 2); DFSTestUtil.createFile(cluster.getFileSystem(), new Path("/test1"),
fileSize, fileSize, blockSize, (short) 2, seed);
JMXGet jmx = new JMXGet(); JMXGet jmx = new JMXGet();
//jmx.setService("*"); // list all hadoop services //jmx.setService("*"); // list all hadoop services
@ -125,7 +114,8 @@ public class TestJMXGet {
cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes).build(); cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes).build();
cluster.waitActive(); cluster.waitActive();
writeFile(cluster.getFileSystem(), new Path("/test"), 2); DFSTestUtil.createFile(cluster.getFileSystem(), new Path("/test"),
fileSize, fileSize, blockSize, (short) 2, seed);
JMXGet jmx = new JMXGet(); JMXGet jmx = new JMXGet();
//jmx.setService("*"); // list all hadoop services //jmx.setService("*"); // list all hadoop services