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 static org.junit.Assert.*;
21 import static org.mockito.Mockito.*;
22 import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_START_KEY;
23 import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_END_KEY;
24 import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
25
26 import java.io.IOException;
27 import java.util.ArrayList;
28 import java.util.Arrays;
29 import java.util.Collection;
30 import java.util.Iterator;
31 import java.util.List;
32 import java.util.ListIterator;
33 import java.util.TreeMap;
34
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.fs.FileSystem;
37 import org.apache.hadoop.fs.Path;
38 import org.apache.hadoop.hbase.Cell;
39 import org.apache.hadoop.hbase.HBaseConfiguration;
40 import org.apache.hadoop.hbase.HColumnDescriptor;
41 import org.apache.hadoop.hbase.HRegionInfo;
42 import org.apache.hadoop.hbase.KeyValue;
43 import org.apache.hadoop.hbase.KeyValue.KVComparator;
44 import org.apache.hadoop.hbase.SmallTests;
45 import org.apache.hadoop.hbase.TableName;
46 import org.apache.hadoop.hbase.io.compress.Compression;
47 import org.apache.hadoop.hbase.io.hfile.HFile;
48 import org.apache.hadoop.hbase.regionserver.StoreFile;
49 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
50 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor;
51 import org.apache.hadoop.hbase.util.Bytes;
52 import org.junit.Test;
53 import org.junit.experimental.categories.Category;
54 import org.mockito.invocation.InvocationOnMock;
55 import org.mockito.stubbing.Answer;
56
57
58 @Category(SmallTests.class)
59 public class TestStripeCompactor {
60 private static final byte[] NAME_OF_THINGS = Bytes.toBytes("foo");
61 private static final TableName TABLE_NAME = TableName.valueOf(NAME_OF_THINGS, NAME_OF_THINGS);
62
63 private static final byte[] KEY_B = Bytes.toBytes("bbb");
64 private static final byte[] KEY_C = Bytes.toBytes("ccc");
65 private static final byte[] KEY_D = Bytes.toBytes("ddd");
66
67 private static final KeyValue KV_A = kvAfter(Bytes.toBytes("aaa"));
68 private static final KeyValue KV_B = kvAfter(KEY_B);
69 private static final KeyValue KV_C = kvAfter(KEY_C);
70 private static final KeyValue KV_D = kvAfter(KEY_D);
71
72 private static KeyValue kvAfter(byte[] key) {
73 return new KeyValue(Arrays.copyOf(key, key.length + 1), 0L);
74 }
75
76 private static <T> T[] a(T... a) {
77 return a;
78 }
79
80 private static KeyValue[] e() {
81 return TestStripeCompactor.<KeyValue>a();
82 }
83
84 @Test
85 public void testBoundaryCompactions() throws Exception {
86
87 verifyBoundaryCompaction(a(KV_A, KV_A, KV_B, KV_B, KV_C, KV_D),
88 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)));
89 verifyBoundaryCompaction(a(KV_B, KV_C), a(KEY_B, KEY_C, KEY_D), a(a(KV_B), a(KV_C)));
90 verifyBoundaryCompaction(a(KV_B, KV_C), a(KEY_B, KEY_D), new KeyValue[][] { a(KV_B, KV_C) });
91 }
92
93 @Test
94 public void testBoundaryCompactionEmptyFiles() throws Exception {
95
96 verifyBoundaryCompaction(
97 a(KV_B), a(KEY_B, KEY_C, KEY_D, OPEN_KEY), a(a(KV_B), null, null), null, null, false);
98 verifyBoundaryCompaction(a(KV_A, KV_C),
99 a(OPEN_KEY, KEY_B, KEY_C, KEY_D), a(a(KV_A), null, a(KV_C)), null, null, false);
100
101 verifyBoundaryCompaction(
102 e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(null, null, e()), null, null, false);
103
104 verifyBoundaryCompaction(
105 e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(null, e(), null), KEY_B, KEY_C, false);
106 verifyBoundaryCompaction(
107 e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(e(), e(), null), OPEN_KEY, KEY_C, false);
108
109 verifyBoundaryCompaction(a(KV_A), a(OPEN_KEY, KEY_B, KEY_C, KEY_D, OPEN_KEY),
110 a(a(KV_A), e(), e(), null), KEY_B, KEY_D, false);
111 verifyBoundaryCompaction(a(KV_C), a(OPEN_KEY, KEY_B, KEY_C, KEY_D, OPEN_KEY),
112 a(null, null, a(KV_C), e()), KEY_C, OPEN_KEY, false);
113
114 }
115
116 public static void verifyBoundaryCompaction(
117 KeyValue[] input, byte[][] boundaries, KeyValue[][] output) throws Exception {
118 verifyBoundaryCompaction(input, boundaries, output, null, null, true);
119 }
120
121 public static void verifyBoundaryCompaction(KeyValue[] input, byte[][] boundaries,
122 KeyValue[][] output, byte[] majorFrom, byte[] majorTo, boolean allFiles)
123 throws Exception {
124 StoreFileWritersCapture writers = new StoreFileWritersCapture();
125 StripeCompactor sc = createCompactor(writers, input);
126 List<Path> paths =
127 sc.compact(createDummyRequest(), Arrays.asList(boundaries), majorFrom, majorTo);
128 writers.verifyKvs(output, allFiles, true);
129 if (allFiles) {
130 assertEquals(output.length, paths.size());
131 writers.verifyBoundaries(boundaries);
132 }
133 }
134
135 @Test
136 public void testSizeCompactions() throws Exception {
137
138 verifySizeCompaction(a(KV_A, KV_A, KV_B, KV_C, KV_D), 3, 2, OPEN_KEY, OPEN_KEY,
139 a(a(KV_A, KV_A), a(KV_B, KV_C), a(KV_D)));
140 verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), 4, 1, OPEN_KEY, OPEN_KEY,
141 a(a(KV_A), a(KV_B), a(KV_C), a(KV_D)));
142 verifySizeCompaction(a(KV_B, KV_C), 2, 1, KEY_B, KEY_D, a(a(KV_B), a(KV_C)));
143
144 verifySizeCompaction(a(KV_A, KV_A, KV_A, KV_C, KV_D), 3, 2, OPEN_KEY, OPEN_KEY,
145 a(a(KV_A, KV_A, KV_A), a(KV_C, KV_D)));
146 verifySizeCompaction(a(KV_A, KV_B, KV_B, KV_C), 3, 1, OPEN_KEY, OPEN_KEY,
147 a(a(KV_A), a(KV_B, KV_B), a(KV_C)));
148
149 verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), 2, 1, OPEN_KEY, OPEN_KEY,
150 a(a(KV_A), a(KV_B, KV_C, KV_D)));
151 verifySizeCompaction(a(KV_A, KV_B, KV_C), 1, Long.MAX_VALUE, OPEN_KEY, KEY_D,
152 new KeyValue[][] { a(KV_A, KV_B, KV_C) });
153
154 verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), Integer.MAX_VALUE, 2, OPEN_KEY, OPEN_KEY,
155 a(a(KV_A, KV_B), a(KV_C, KV_D)));
156 }
157
158 public static void verifySizeCompaction(KeyValue[] input, int targetCount, long targetSize,
159 byte[] left, byte[] right, KeyValue[][] output) throws Exception {
160 StoreFileWritersCapture writers = new StoreFileWritersCapture();
161 StripeCompactor sc = createCompactor(writers, input);
162 List<Path> paths = sc.compact(
163 createDummyRequest(), targetCount, targetSize, left, right, null, null);
164 assertEquals(output.length, paths.size());
165 writers.verifyKvs(output, true, true);
166 List<byte[]> boundaries = new ArrayList<byte[]>();
167 boundaries.add(left);
168 for (int i = 1; i < output.length; ++i) {
169 boundaries.add(output[i][0].getRow());
170 }
171 boundaries.add(right);
172 writers.verifyBoundaries(boundaries.toArray(new byte[][] {}));
173 }
174
175 private static StripeCompactor createCompactor(
176 StoreFileWritersCapture writers, KeyValue[] input) throws Exception {
177 Configuration conf = HBaseConfiguration.create();
178 final Scanner scanner = new Scanner(input);
179
180
181 HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS);
182 ScanInfo si = new ScanInfo(col, Long.MAX_VALUE, 0, new KVComparator());
183 Store store = mock(Store.class);
184 when(store.getFamily()).thenReturn(col);
185 when(store.getScanInfo()).thenReturn(si);
186 when(store.areWritesEnabled()).thenReturn(true);
187 when(store.getFileSystem()).thenReturn(mock(FileSystem.class));
188 when(store.getRegionInfo()).thenReturn(new HRegionInfo(TABLE_NAME));
189 when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class),
190 anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
191 when(store.getComparator()).thenReturn(new KVComparator());
192
193 return new StripeCompactor(conf, store) {
194 @Override
195 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
196 long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
197 byte[] dropDeletesToRow) throws IOException {
198 return scanner;
199 }
200
201 @Override
202 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
203 ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
204 return scanner;
205 }
206 };
207 }
208
209 private static CompactionRequest createDummyRequest() throws Exception {
210
211
212 StoreFile sf = mock(StoreFile.class);
213 StoreFile.Reader r = mock(StoreFile.Reader.class);
214 when(r.length()).thenReturn(1L);
215 when(r.getBloomFilterType()).thenReturn(BloomType.NONE);
216 when(r.getHFileReader()).thenReturn(mock(HFile.Reader.class));
217 when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong()))
218 .thenReturn(mock(StoreFileScanner.class));
219 when(sf.getReader()).thenReturn(r);
220 when(sf.createReader()).thenReturn(r);
221 return new CompactionRequest(Arrays.asList(sf));
222 }
223
224 private static class Scanner implements InternalScanner {
225 private final ArrayList<KeyValue> kvs;
226 public Scanner(KeyValue... kvs) {
227 this.kvs = new ArrayList<KeyValue>(Arrays.asList(kvs));
228 }
229
230 @Override
231 public boolean next(List<Cell> results) throws IOException {
232 if (kvs.isEmpty()) return false;
233 results.add(kvs.remove(0));
234 return !kvs.isEmpty();
235 }
236 @Override
237 public boolean next(List<Cell> result, int limit) throws IOException {
238 return next(result);
239 }
240 @Override
241 public void close() throws IOException {}
242 }
243
244
245 public static class StoreFileWritersCapture implements
246 Answer<StoreFile.Writer>, StripeMultiFileWriter.WriterFactory {
247 public static class Writer {
248 public ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
249 public TreeMap<byte[], byte[]> data = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
250 }
251
252 private List<Writer> writers = new ArrayList<Writer>();
253
254 @Override
255 public StoreFile.Writer createWriter() throws IOException {
256 final Writer realWriter = new Writer();
257 writers.add(realWriter);
258 StoreFile.Writer writer = mock(StoreFile.Writer.class);
259 doAnswer(new Answer<Object>() {
260 public Object answer(InvocationOnMock invocation) {
261 return realWriter.kvs.add((KeyValue)invocation.getArguments()[0]);
262 }}).when(writer).append(any(KeyValue.class));
263 doAnswer(new Answer<Object>() {
264 public Object answer(InvocationOnMock invocation) {
265 Object[] args = invocation.getArguments();
266 return realWriter.data.put((byte[])args[0], (byte[])args[1]);
267 }}).when(writer).appendFileInfo(any(byte[].class), any(byte[].class));
268 return writer;
269 }
270
271 @Override
272 public StoreFile.Writer answer(InvocationOnMock invocation) throws Throwable {
273 return createWriter();
274 }
275
276 public void verifyKvs(KeyValue[][] kvss, boolean allFiles, boolean requireMetadata) {
277 if (allFiles) {
278 assertEquals(kvss.length, writers.size());
279 }
280 int skippedWriters = 0;
281 for (int i = 0; i < kvss.length; ++i) {
282 KeyValue[] kvs = kvss[i];
283 if (kvs != null) {
284 Writer w = writers.get(i - skippedWriters);
285 if (requireMetadata) {
286 assertNotNull(w.data.get(STRIPE_START_KEY));
287 assertNotNull(w.data.get(STRIPE_END_KEY));
288 } else {
289 assertNull(w.data.get(STRIPE_START_KEY));
290 assertNull(w.data.get(STRIPE_END_KEY));
291 }
292 assertEquals(kvs.length, w.kvs.size());
293 for (int j = 0; j < kvs.length; ++j) {
294 assertEquals(kvs[j], w.kvs.get(j));
295 }
296 } else {
297 assertFalse(allFiles);
298 ++skippedWriters;
299 }
300 }
301 }
302
303 public void verifyBoundaries(byte[][] boundaries) {
304 assertEquals(boundaries.length - 1, writers.size());
305 for (int i = 0; i < writers.size(); ++i) {
306 assertArrayEquals("i = " + i, boundaries[i], writers.get(i).data.get(STRIPE_START_KEY));
307 assertArrayEquals("i = " + i, boundaries[i + 1], writers.get(i).data.get(STRIPE_END_KEY));
308 }
309 }
310 }
311 }