1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver.compactions;
19
20 import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
21 import static org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.createDummyRequest;
22 import static org.junit.Assert.assertEquals;
23 import static org.mockito.Matchers.any;
24 import static org.mockito.Matchers.anyBoolean;
25 import static org.mockito.Matchers.anyLong;
26 import static org.mockito.Mockito.mock;
27 import static org.mockito.Mockito.when;
28
29 import java.io.IOException;
30 import java.util.ArrayList;
31 import java.util.Arrays;
32 import java.util.List;
33
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.fs.FileSystem;
36 import org.apache.hadoop.fs.Path;
37 import org.apache.hadoop.hbase.CellUtil;
38 import org.apache.hadoop.hbase.HBaseConfiguration;
39 import org.apache.hadoop.hbase.HColumnDescriptor;
40 import org.apache.hadoop.hbase.HRegionInfo;
41 import org.apache.hadoop.hbase.KeyValue;
42 import org.apache.hadoop.hbase.KeyValue.KVComparator;
43 import org.apache.hadoop.hbase.TableName;
44 import org.apache.hadoop.hbase.io.compress.Compression;
45 import org.apache.hadoop.hbase.regionserver.InternalScanner;
46 import org.apache.hadoop.hbase.regionserver.ScanInfo;
47 import org.apache.hadoop.hbase.regionserver.ScanType;
48 import org.apache.hadoop.hbase.regionserver.Store;
49 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
50 import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner;
51 import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture;
52 import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
53 import org.apache.hadoop.hbase.testclassification.SmallTests;
54 import org.apache.hadoop.hbase.util.Bytes;
55 import org.junit.Test;
56 import org.junit.experimental.categories.Category;
57 import org.junit.runner.RunWith;
58 import org.junit.runners.Parameterized;
59 import org.junit.runners.Parameterized.Parameter;
60 import org.junit.runners.Parameterized.Parameters;
61
62 @RunWith(Parameterized.class)
63 @Category({ SmallTests.class })
64 public class TestStripeCompactor {
65 private static final byte[] NAME_OF_THINGS = Bytes.toBytes("foo");
66 private static final TableName TABLE_NAME = TableName.valueOf(NAME_OF_THINGS, NAME_OF_THINGS);
67
68 private static final byte[] KEY_B = Bytes.toBytes("bbb");
69 private static final byte[] KEY_C = Bytes.toBytes("ccc");
70 private static final byte[] KEY_D = Bytes.toBytes("ddd");
71
72 private static final KeyValue KV_A = kvAfter(Bytes.toBytes("aaa"));
73 private static final KeyValue KV_B = kvAfter(KEY_B);
74 private static final KeyValue KV_C = kvAfter(KEY_C);
75 private static final KeyValue KV_D = kvAfter(KEY_D);
76
77 @Parameters(name = "{index}: usePrivateReaders={0}")
78 public static Iterable<Object[]> data() {
79 return Arrays.asList(new Object[] { true }, new Object[] { false });
80 }
81
82 @Parameter
83 public boolean usePrivateReaders;
84
85 private static KeyValue kvAfter(byte[] key) {
86 return new KeyValue(Arrays.copyOf(key, key.length + 1), 0L);
87 }
88
89 @SuppressWarnings("unchecked")
90 private static <T> T[] a(T... a) {
91 return a;
92 }
93
94 private static KeyValue[] e() {
95 return TestStripeCompactor.<KeyValue> a();
96 }
97
98 @Test
99 public void testBoundaryCompactions() throws Exception {
100
101 verifyBoundaryCompaction(a(KV_A, KV_A, KV_B, KV_B, KV_C, KV_D),
102 a(OPEN_KEY, KEY_B, KEY_D, OPEN_KEY), a(a(KV_A, KV_A), a(KV_B, KV_B, KV_C), a(KV_D)));
103 verifyBoundaryCompaction(a(KV_B, KV_C), a(KEY_B, KEY_C, KEY_D), a(a(KV_B), a(KV_C)));
104 verifyBoundaryCompaction(a(KV_B, KV_C), a(KEY_B, KEY_D), new KeyValue[][] { a(KV_B, KV_C) });
105 }
106
107 @Test
108 public void testBoundaryCompactionEmptyFiles() throws Exception {
109
110 verifyBoundaryCompaction(a(KV_B), a(KEY_B, KEY_C, KEY_D, OPEN_KEY), a(a(KV_B), null, null),
111 null, null, false);
112 verifyBoundaryCompaction(a(KV_A, KV_C), a(OPEN_KEY, KEY_B, KEY_C, KEY_D),
113 a(a(KV_A), null, a(KV_C)), null, null, false);
114
115 verifyBoundaryCompaction(e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(null, null, e()), null,
116 null, false);
117
118 verifyBoundaryCompaction(e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(null, e(), null), KEY_B,
119 KEY_C, false);
120 verifyBoundaryCompaction(e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(e(), e(), null), OPEN_KEY,
121 KEY_C, false);
122
123 verifyBoundaryCompaction(a(KV_A), a(OPEN_KEY, KEY_B, KEY_C, KEY_D, OPEN_KEY),
124 a(a(KV_A), e(), e(), null), KEY_B, KEY_D, false);
125 verifyBoundaryCompaction(a(KV_C), a(OPEN_KEY, KEY_B, KEY_C, KEY_D, OPEN_KEY),
126 a(null, null, a(KV_C), e()), KEY_C, OPEN_KEY, false);
127
128 }
129
130 private void verifyBoundaryCompaction(KeyValue[] input, byte[][] boundaries, KeyValue[][] output)
131 throws Exception {
132 verifyBoundaryCompaction(input, boundaries, output, null, null, true);
133 }
134
135 private void verifyBoundaryCompaction(KeyValue[] input, byte[][] boundaries, KeyValue[][] output,
136 byte[] majorFrom, byte[] majorTo, boolean allFiles) throws Exception {
137 StoreFileWritersCapture writers = new StoreFileWritersCapture();
138 StripeCompactor sc = createCompactor(writers, input);
139 List<Path> paths = sc.compact(createDummyRequest(), Arrays.asList(boundaries), majorFrom,
140 majorTo, NoLimitCompactionThroughputController.INSTANCE, null);
141 writers.verifyKvs(output, allFiles, true);
142 if (allFiles) {
143 assertEquals(output.length, paths.size());
144 writers.verifyBoundaries(boundaries);
145 }
146 }
147
148 @Test
149 public void testSizeCompactions() throws Exception {
150
151 verifySizeCompaction(a(KV_A, KV_A, KV_B, KV_C, KV_D), 3, 2, OPEN_KEY, OPEN_KEY,
152 a(a(KV_A, KV_A), a(KV_B, KV_C), a(KV_D)));
153 verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), 4, 1, OPEN_KEY, OPEN_KEY,
154 a(a(KV_A), a(KV_B), a(KV_C), a(KV_D)));
155 verifySizeCompaction(a(KV_B, KV_C), 2, 1, KEY_B, KEY_D, a(a(KV_B), a(KV_C)));
156
157 verifySizeCompaction(a(KV_A, KV_A, KV_A, KV_C, KV_D), 3, 2, OPEN_KEY, OPEN_KEY,
158 a(a(KV_A, KV_A, KV_A), a(KV_C, KV_D)));
159 verifySizeCompaction(a(KV_A, KV_B, KV_B, KV_C), 3, 1, OPEN_KEY, OPEN_KEY,
160 a(a(KV_A), a(KV_B, KV_B), a(KV_C)));
161
162 verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), 2, 1, OPEN_KEY, OPEN_KEY,
163 a(a(KV_A), a(KV_B, KV_C, KV_D)));
164 verifySizeCompaction(a(KV_A, KV_B, KV_C), 1, Long.MAX_VALUE, OPEN_KEY, KEY_D,
165 new KeyValue[][] { a(KV_A, KV_B, KV_C) });
166
167 verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), Integer.MAX_VALUE, 2, OPEN_KEY, OPEN_KEY,
168 a(a(KV_A, KV_B), a(KV_C, KV_D)));
169 }
170
171 private void verifySizeCompaction(KeyValue[] input, int targetCount, long targetSize, byte[] left,
172 byte[] right, KeyValue[][] output) throws Exception {
173 StoreFileWritersCapture writers = new StoreFileWritersCapture();
174 StripeCompactor sc = createCompactor(writers, input);
175 List<Path> paths = sc.compact(createDummyRequest(), targetCount, targetSize, left, right, null,
176 null, NoLimitCompactionThroughputController.INSTANCE, null);
177 assertEquals(output.length, paths.size());
178 writers.verifyKvs(output, true, true);
179 List<byte[]> boundaries = new ArrayList<byte[]>();
180 boundaries.add(left);
181 for (int i = 1; i < output.length; ++i) {
182 boundaries.add(CellUtil.cloneRow(output[i][0]));
183 }
184 boundaries.add(right);
185 writers.verifyBoundaries(boundaries.toArray(new byte[][] {}));
186 }
187
188 private StripeCompactor createCompactor(StoreFileWritersCapture writers, KeyValue[] input)
189 throws Exception {
190 Configuration conf = HBaseConfiguration.create();
191 conf.setBoolean("hbase.regionserver.compaction.private.readers", usePrivateReaders);
192 final Scanner scanner = new Scanner(input);
193
194
195 HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS);
196 ScanInfo si = new ScanInfo(col, Long.MAX_VALUE, 0, new KVComparator());
197 Store store = mock(Store.class);
198 when(store.getFamily()).thenReturn(col);
199 when(store.getScanInfo()).thenReturn(si);
200 when(store.areWritesEnabled()).thenReturn(true);
201 when(store.getFileSystem()).thenReturn(mock(FileSystem.class));
202 when(store.getRegionInfo()).thenReturn(new HRegionInfo(TABLE_NAME));
203 when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(),
204 anyBoolean(), anyBoolean())).thenAnswer(writers);
205 when(store.getComparator()).thenReturn(new KVComparator());
206
207 return new StripeCompactor(conf, store) {
208 @Override
209 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
210 long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
211 byte[] dropDeletesToRow) throws IOException {
212 return scanner;
213 }
214
215 @Override
216 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
217 ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
218 return scanner;
219 }
220 };
221 }
222 }