HBASE-7947: Custom compaction requests should have test for adding to wrong list
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1451747 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1046174cc8
commit
8cdefe6bca
|
@ -24,6 +24,7 @@ import static org.mockito.Mockito.spy;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
|
||||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactionPolicy;
|
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactionPolicy;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
import org.mockito.invocation.InvocationOnMock;
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
|
@ -574,8 +576,12 @@ public class TestCompaction extends HBaseTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void createStoreFile(final HRegion region) throws IOException {
|
private void createStoreFile(final HRegion region) throws IOException {
|
||||||
|
createStoreFile(region, Bytes.toString(COLUMN_FAMILY));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void createStoreFile(final HRegion region, String family) throws IOException {
|
||||||
HRegionIncommon loader = new HRegionIncommon(region);
|
HRegionIncommon loader = new HRegionIncommon(region);
|
||||||
addContent(loader, Bytes.toString(COLUMN_FAMILY));
|
addContent(loader, family);
|
||||||
loader.flushcache();
|
loader.flushcache();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -688,6 +694,43 @@ public class TestCompaction extends HBaseTestCase {
|
||||||
thread.interruptIfNecessary();
|
thread.interruptIfNecessary();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* HBASE-7947: Regression test to ensure adding to the correct list in the
|
||||||
|
* {@link CompactSplitThread}
|
||||||
|
* @throws Exception on failure
|
||||||
|
*/
|
||||||
|
public void testMultipleCustomCompactionRequests() throws Exception {
|
||||||
|
// setup a compact/split thread on a mock server
|
||||||
|
HRegionServer mockServer = Mockito.mock(HRegionServer.class);
|
||||||
|
Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
|
||||||
|
CompactSplitThread thread = new CompactSplitThread(mockServer);
|
||||||
|
Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
|
||||||
|
|
||||||
|
// setup a region/store with some files
|
||||||
|
int numStores = r.getStores().size();
|
||||||
|
List<Pair<CompactionRequest, Store>> requests =
|
||||||
|
new ArrayList<Pair<CompactionRequest, Store>>(numStores);
|
||||||
|
CountDownLatch latch = new CountDownLatch(numStores);
|
||||||
|
// create some store files and setup requests for each store on which we want to do a
|
||||||
|
// compaction
|
||||||
|
for (Store store : r.getStores().values()) {
|
||||||
|
createStoreFile(r, store.getColumnFamilyName());
|
||||||
|
createStoreFile(r, store.getColumnFamilyName());
|
||||||
|
createStoreFile(r, store.getColumnFamilyName());
|
||||||
|
requests
|
||||||
|
.add(new Pair<CompactionRequest, Store>(new TrackableCompactionRequest(latch), store));
|
||||||
|
}
|
||||||
|
|
||||||
|
thread.requestCompaction(r, "test mulitple custom comapctions", Store.PRIORITY_USER,
|
||||||
|
Collections.unmodifiableList(requests));
|
||||||
|
|
||||||
|
// wait for the latch to complete.
|
||||||
|
latch.await();
|
||||||
|
|
||||||
|
thread.interruptIfNecessary();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Simple {@link CompactionRequest} on which you can wait until the requested compaction finishes.
|
* Simple {@link CompactionRequest} on which you can wait until the requested compaction finishes.
|
||||||
*/
|
*/
|
||||||
|
|
Loading…
Reference in New Issue