HBASE-13181 TestHRegionReplayEvents.testReplayBulkLoadEvent fails frequently
This commit is contained in:
parent
29104da013
commit
7fa1a6b968
|
@ -90,7 +90,7 @@ public class TestBulkLoad {
|
|||
private final Expectations callOnce;
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
||||
|
||||
public TestBulkLoad() throws IOException {
|
||||
callOnce = new Expectations() {
|
||||
{
|
||||
|
@ -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)));
|
||||
hFileFactory.withFileContext(new HFileContext());
|
||||
HFile.Writer writer = hFileFactory.create();
|
||||
|
||||
writer.append(new KeyValue(CellUtil.createCell(randomBytes,
|
||||
family,
|
||||
randomBytes,
|
||||
0l,
|
||||
KeyValue.Type.Put.getCode(),
|
||||
randomBytes)));
|
||||
writer.close();
|
||||
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();
|
||||
}
|
||||
|
||||
|
@ -286,7 +293,7 @@ public class TestBulkLoad {
|
|||
assertNotNull(desc);
|
||||
|
||||
if (tableName != null) {
|
||||
assertTrue(Bytes.equals(ProtobufUtil.toTableName(desc.getTableName()).getName(),
|
||||
assertTrue(Bytes.equals(ProtobufUtil.toTableName(desc.getTableName()).getName(),
|
||||
tableName));
|
||||
}
|
||||
|
||||
|
@ -297,7 +304,7 @@ public class TestBulkLoad {
|
|||
assertTrue(Bytes.equals(Bytes.toBytes(store.getStoreHomeDir()), familyName));
|
||||
assertEquals(storeFileNames.size(), store.getStoreFileCount());
|
||||
}
|
||||
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
@ -1452,7 +1452,7 @@ public class TestHRegionReplayEvents {
|
|||
// close the region and open again.
|
||||
primaryRegion.close();
|
||||
primaryRegion = HRegion.openHRegion(rootDir, primaryHri, htd, walPrimary, CONF, rss, null);
|
||||
|
||||
|
||||
// bulk load a file into primary region
|
||||
Random random = new Random();
|
||||
byte[] randomValues = new byte[20];
|
||||
|
@ -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));
|
||||
hFileFactory.withFileContext(new HFileContext());
|
||||
HFile.Writer writer = hFileFactory.create();
|
||||
|
||||
writer.append(new KeyValue(CellUtil.createCell(valueBytes, family, valueBytes, 0l,
|
||||
KeyValue.Type.Put.getCode(), valueBytes)));
|
||||
writer.close();
|
||||
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();
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue