Skip to content

Commit

Permalink
HBASE-25213 Should request Compaction when bulkLoadHFiles is done (ad…
Browse files Browse the repository at this point in the history
…dendum) (#2701)

Signed-off-by: Guanghao Zhang <zghao@apache.org>
  • Loading branch information
nyl3532016 authored Nov 24, 2020
1 parent fc33137 commit 4affae5
Showing 1 changed file with 15 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,14 @@
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
Expand Down Expand Up @@ -212,7 +214,8 @@ public void testBulkLoading() throws Exception {
QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
tn, SpaceQuotaHelperForTests.ONE_GIGABYTE, SpaceViolationPolicy.NO_INSERTS);
admin.setQuota(settings);

admin.compactionSwitch(false,
admin.getRegionServers().stream().map(ServerName::toString).collect(Collectors.toList()));
Map<byte[], List<Path>> family2Files = helper.generateFileToLoad(tn, 3, 550);
// Make sure the files are about as long as we expect
FileSystem fs = TEST_UTIL.getTestFileSystem();
Expand All @@ -231,12 +234,17 @@ public void testBulkLoading() throws Exception {
BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(tn, family2Files).isEmpty());

final long finalTotalSize = totalSize;
TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, tn) {
@Override
boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
return snapshot.getUsage() >= finalTotalSize;
}
});
try {
TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, tn) {
@Override
boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
return snapshot.getUsage() >= finalTotalSize;
}
});
} finally {
admin.compactionSwitch(true,
admin.getRegionServers().stream().map(ServerName::toString).collect(Collectors.toList()));
}
}

@Test
Expand Down

0 comments on commit 4affae5

Please sign in to comment.