1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver;
19
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.Arrays;
23 import java.util.List;
24
25 import com.google.common.collect.ImmutableList;
26 import com.google.common.collect.Lists;
27
28 import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy;
29 import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest;
30 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
31 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
32 import static org.junit.Assert.*;
33
34 public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy {
35
36 protected ArrayList<StoreFile> sfCreate(long[] minTimestamps, long[] maxTimestamps, long[] sizes)
37 throws IOException {
38 ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
39 EnvironmentEdgeManager.injectEdge(timeMachine);
40
41 timeMachine.setValue(1);
42 ArrayList<Long> ageInDisk = new ArrayList<Long>();
43 for (int i = 0; i < sizes.length; i++) {
44 ageInDisk.add(0L);
45 }
46
47 ArrayList<StoreFile> ret = Lists.newArrayList();
48 for (int i = 0; i < sizes.length; i++) {
49 MockStoreFile msf =
50 new MockStoreFile(TEST_UTIL, TEST_FILE, sizes[i], ageInDisk.get(i), false, i);
51 msf.setTimeRangeTracker(new TimeRangeTracker(minTimestamps[i], maxTimestamps[i]));
52 ret.add(msf);
53 }
54 return ret;
55 }
56
57 protected void compactEquals(long now, ArrayList<StoreFile> candidates, long[] expectedFileSizes,
58 long[] expectedBoundaries, boolean isMajor, boolean toCompact) throws IOException {
59 ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
60 EnvironmentEdgeManager.injectEdge(timeMachine);
61 timeMachine.setValue(now);
62 DateTieredCompactionRequest request;
63 DateTieredCompactionPolicy policy =
64 (DateTieredCompactionPolicy) store.storeEngine.getCompactionPolicy();
65 if (isMajor) {
66 for (StoreFile file : candidates) {
67 ((MockStoreFile) file).setIsMajor(true);
68 }
69 assertEquals(toCompact, policy.shouldPerformMajorCompaction(candidates));
70 request = (DateTieredCompactionRequest) policy.selectMajorCompaction(candidates);
71 } else {
72 assertEquals(toCompact, policy.needsCompaction(candidates, ImmutableList.<StoreFile> of()));
73 request =
74 (DateTieredCompactionRequest) policy.selectMinorCompaction(candidates, false, false);
75 }
76 List<StoreFile> actual = Lists.newArrayList(request.getFiles());
77 assertEquals(Arrays.toString(expectedFileSizes), Arrays.toString(getSizes(actual)));
78 assertEquals(Arrays.toString(expectedBoundaries),
79 Arrays.toString(request.getBoundaries().toArray()));
80 }
81 }