HBASE-13181 TestHRegionReplayEvents.testReplayBulkLoadEvent fails frequently

This commit is contained in:
Matteo Bertozzi 2015-03-10 17:21:34 +00:00
parent 78d6a3402c
commit c4369d9a90
2 changed files with 36 additions and 22 deletions

View File

@ -233,17 +233,24 @@ public class TestBulkLoad {
HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(conf);
// TODO We need a way to do this without creating files
File hFileLocation = testFolder.newFile();
hFileFactory.withOutputStream(new FSDataOutputStream(new FileOutputStream(hFileLocation)));
FSDataOutputStream out = new FSDataOutputStream(new FileOutputStream(hFileLocation));
try {
hFileFactory.withOutputStream(out);
hFileFactory.withFileContext(new HFileContext());
HFile.Writer writer = hFileFactory.create();
try {
writer.append(new KeyValue(CellUtil.createCell(randomBytes,
family,
randomBytes,
0l,
KeyValue.Type.Put.getCode(),
randomBytes)));
} finally {
writer.close();
}
} finally {
out.close();
}
return hFileLocation.getAbsoluteFile().getAbsolutePath();
}

View File

@ -1515,13 +1515,20 @@ public class TestHRegionReplayEvents {
HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(TEST_UTIL.getConfiguration());
// TODO We need a way to do this without creating files
Path testFile = new Path(testPath, UUID.randomUUID().toString());
hFileFactory.withOutputStream(TEST_UTIL.getTestFileSystem().create(testFile));
FSDataOutputStream out = TEST_UTIL.getTestFileSystem().create(testFile);
try {
hFileFactory.withOutputStream(out);
hFileFactory.withFileContext(new HFileContext());
HFile.Writer writer = hFileFactory.create();
try {
writer.append(new KeyValue(CellUtil.createCell(valueBytes, family, valueBytes, 0l,
KeyValue.Type.Put.getCode(), valueBytes)));
} finally {
writer.close();
}
} finally {
out.close();
}
return testFile.toString();
}