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.junit.Assert.*;
22 import static org.mockito.AdditionalMatchers.aryEq;
23 import static org.mockito.Matchers.any;
24 import static org.mockito.Matchers.anyInt;
25 import static org.mockito.Matchers.anyLong;
26 import static org.mockito.Matchers.argThat;
27 import static org.mockito.Matchers.eq;
28 import static org.mockito.Matchers.isNull;
29 import static org.mockito.Mockito.*;
30
31 import java.io.IOException;
32 import java.util.ArrayList;
33 import java.util.Arrays;
34 import java.util.Collection;
35 import java.util.List;
36
37 import org.apache.commons.lang.NotImplementedException;
38 import org.apache.hadoop.conf.Configuration;
39 import org.apache.hadoop.fs.Path;
40 import org.apache.hadoop.hbase.HBaseConfiguration;
41 import org.apache.hadoop.hbase.KeyValue;
42 import org.apache.hadoop.hbase.SmallTests;
43 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
44 import org.apache.hadoop.hbase.regionserver.StoreFile;
45 import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
46 import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
47 import org.apache.hadoop.hbase.regionserver.StripeStoreFileManager;
48 import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher;
49 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy.StripeInformationProvider;
50 import org.apache.hadoop.hbase.util.Bytes;
51 import org.apache.hadoop.hbase.util.ConcatenatedLists;
52 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
53 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
54 import org.apache.hadoop.hbase.regionserver.TestStripeCompactor.StoreFileWritersCapture;
55 import org.junit.Test;
56 import org.junit.experimental.categories.Category;
57 import org.mockito.ArgumentMatcher;
58
59 import com.google.common.collect.ImmutableList;
60 import com.google.common.collect.Lists;
61
62 @Category(SmallTests.class)
63 public class TestStripeCompactionPolicy {
64 private static final byte[] KEY_A = Bytes.toBytes("aaa");
65 private static final byte[] KEY_B = Bytes.toBytes("bbb");
66 private static final byte[] KEY_C = Bytes.toBytes("ccc");
67 private static final byte[] KEY_D = Bytes.toBytes("ddd");
68 private static final byte[] KEY_E = Bytes.toBytes("eee");
69 private static final KeyValue KV_A = new KeyValue(KEY_A, 0L);
70 private static final KeyValue KV_B = new KeyValue(KEY_B, 0L);
71 private static final KeyValue KV_C = new KeyValue(KEY_C, 0L);
72 private static final KeyValue KV_D = new KeyValue(KEY_D, 0L);
73 private static final KeyValue KV_E = new KeyValue(KEY_E, 0L);
74
75
76 private static long defaultSplitSize = 18;
77 private static float defaultSplitCount = 1.8F;
78 private final static int defaultInitialCount = 1;
79 private static long defaultTtl = 1000 * 1000;
80
81 @Test
82 public void testNoStripesFromFlush() throws Exception {
83 Configuration conf = HBaseConfiguration.create();
84 conf.setBoolean(StripeStoreConfig.FLUSH_TO_L0_KEY, true);
85 StripeCompactionPolicy policy = createPolicy(conf);
86 StripeInformationProvider si = createStripesL0Only(0, 0);
87
88 KeyValue[] input = new KeyValue[] { KV_A, KV_B, KV_C, KV_D, KV_E };
89 KeyValue[][] expected = new KeyValue[][] { input };
90 verifyFlush(policy, si, input, expected, null);
91 }
92
93 @Test
94 public void testOldStripesFromFlush() throws Exception {
95 StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create());
96 StripeInformationProvider si = createStripes(0, KEY_C, KEY_D);
97
98 KeyValue[] input = new KeyValue[] { KV_B, KV_C, KV_C, KV_D, KV_E };
99 KeyValue[][] expected = new KeyValue[][] { new KeyValue[] { KV_B },
100 new KeyValue[] { KV_C, KV_C }, new KeyValue[] { KV_D, KV_E } };
101 verifyFlush(policy, si, input, expected, new byte[][] { OPEN_KEY, KEY_C, KEY_D, OPEN_KEY });
102 }
103
104 @Test
105 public void testNewStripesFromFlush() throws Exception {
106 StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create());
107 StripeInformationProvider si = createStripesL0Only(0, 0);
108 KeyValue[] input = new KeyValue[] { KV_B, KV_C, KV_C, KV_D, KV_E };
109
110 KeyValue[][] expected = new KeyValue[][] { input };
111 verifyFlush(policy, si, input, expected, new byte[][] { OPEN_KEY, OPEN_KEY });
112 }
113
114 @Test
115 public void testSingleStripeCompaction() throws Exception {
116
117 Configuration conf = HBaseConfiguration.create();
118 conf.setFloat(CompactionConfiguration.RATIO_KEY, 1.0F);
119 conf.setInt(StripeStoreConfig.MIN_FILES_KEY, 3);
120 conf.setInt(StripeStoreConfig.MAX_FILES_KEY, 4);
121 conf.setLong(StripeStoreConfig.SIZE_TO_SPLIT_KEY, 1000);
122 StoreConfigInformation sci = mock(StoreConfigInformation.class);
123 StripeStoreConfig ssc = new StripeStoreConfig(conf, sci);
124 StripeCompactionPolicy policy = new StripeCompactionPolicy(conf, sci, ssc) {
125 @Override
126 public StripeCompactionRequest selectCompaction(StripeInformationProvider si,
127 List<StoreFile> filesCompacting, boolean isOffpeak) throws IOException {
128 if (!filesCompacting.isEmpty()) return null;
129 return selectSingleStripeCompaction(si, false, false, isOffpeak);
130 }
131
132 @Override
133 public boolean needsCompactions(
134 StripeInformationProvider si, List<StoreFile> filesCompacting) {
135 if (!filesCompacting.isEmpty()) return false;
136 return needsSingleStripeCompaction(si);
137 }
138 };
139
140
141 StripeInformationProvider si = createStripesWithSizes(0, 0,
142 new Long[] { 2L }, new Long[] { 3L, 3L }, new Long[] { 5L, 1L });
143 verifyNoCompaction(policy, si);
144
145 si = createStripesWithSizes(0, 0,
146 new Long[] { 2L }, new Long[] { 3L, 3L }, new Long[] { 5L, 1L, 1L });
147 assertNull(policy.selectCompaction(si, al(), false));
148 assertTrue(policy.needsCompactions(si, al()));
149
150 si = createStripesWithSizes(0, 0,
151 new Long[] { 2L }, new Long[] { 3L, 3L }, new Long[] { 5L, 4L, 3L });
152 verifySingleStripeCompaction(policy, si, 2, null);
153
154 si = createStripesWithSizes(0, 0,
155 new Long[] { 3L, 2L, 2L }, new Long[] { 2L, 2L, 1L }, new Long[] { 3L, 2L, 2L, 1L });
156 verifySingleStripeCompaction(policy, si, 2, null);
157 si = createStripesWithSizes(0, 0,
158 new Long[] { 5L }, new Long[] { 3L, 2L, 2L, 1L }, new Long[] { 3L, 2L, 2L });
159 verifySingleStripeCompaction(policy, si, 1, null);
160
161 si = createStripesWithSizes(0, 0,
162 new Long[] { 3L, 3L, 3L }, new Long[] { 3L, 1L, 2L }, new Long[] { 3L, 2L, 2L });
163 verifySingleStripeCompaction(policy, si, 1, null);
164
165 si = createStripesWithSizes(0, 0, new Long[] { 5L }, new Long[] { 5L, 4L, 4L, 4L, 4L });
166 List<StoreFile> sfs = si.getStripes().get(1).subList(1, 5);
167 verifyCompaction(policy, si, sfs, null, 1, null, si.getStartRow(1), si.getEndRow(1), true);
168
169 si = createStripesWithSizes(0, 0, new Long[] { 5L }, new Long[] { 50L, 4L, 4L, 4L, 4L });
170 sfs = si.getStripes().get(1).subList(1, 5);
171 verifyCompaction(policy, si, sfs, null, 1, null, si.getStartRow(1), si.getEndRow(1), true);
172 }
173
174 @Test
175 public void testWithParallelCompaction() throws Exception {
176
177
178 assertNull(createPolicy(HBaseConfiguration.create()).selectCompaction(
179 mock(StripeInformationProvider.class), al(createFile()), false));
180 }
181
182 @Test
183 public void testWithReferences() throws Exception {
184 StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create());
185 StripeCompactor sc = mock(StripeCompactor.class);
186 StoreFile ref = createFile();
187 when(ref.isReference()).thenReturn(true);
188 StripeInformationProvider si = mock(StripeInformationProvider.class);
189 Collection<StoreFile> sfs = al(ref, createFile());
190 when(si.getStorefiles()).thenReturn(sfs);
191
192 assertTrue(policy.needsCompactions(si, al()));
193 StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false);
194 assertEquals(si.getStorefiles(), scr.getRequest().getFiles());
195 scr.execute(sc);
196 verify(sc, only()).compact(eq(scr.getRequest()), anyInt(), anyLong(),
197 aryEq(OPEN_KEY), aryEq(OPEN_KEY), aryEq(OPEN_KEY), aryEq(OPEN_KEY));
198 }
199
200 @Test
201 public void testInitialCountFromL0() throws Exception {
202 Configuration conf = HBaseConfiguration.create();
203 conf.setInt(StripeStoreConfig.MIN_FILES_L0_KEY, 2);
204 StripeCompactionPolicy policy = createPolicy(
205 conf, defaultSplitSize, defaultSplitCount, 2, false);
206 StripeCompactionPolicy.StripeInformationProvider si = createStripesL0Only(3, 8);
207 verifyCompaction(policy, si, si.getStorefiles(), true, 2, 12L, OPEN_KEY, OPEN_KEY, true);
208 si = createStripesL0Only(3, 10);
209 verifyCompaction(policy, si, si.getStorefiles(), true, 3, 10L, OPEN_KEY, OPEN_KEY, true);
210 policy = createPolicy(conf, defaultSplitSize, defaultSplitCount, 6, false);
211 verifyCompaction(policy, si, si.getStorefiles(), true, 6, 5L, OPEN_KEY, OPEN_KEY, true);
212 }
213
214 @Test
215 public void testExistingStripesFromL0() throws Exception {
216 Configuration conf = HBaseConfiguration.create();
217 conf.setInt(StripeStoreConfig.MIN_FILES_L0_KEY, 3);
218 StripeCompactionPolicy.StripeInformationProvider si = createStripes(3, KEY_A);
219 verifyCompaction(
220 createPolicy(conf), si, si.getLevel0Files(), null, null, si.getStripeBoundaries());
221 }
222
223 @Test
224 public void testNothingToCompactFromL0() throws Exception {
225 Configuration conf = HBaseConfiguration.create();
226 conf.setInt(StripeStoreConfig.MIN_FILES_L0_KEY, 4);
227 StripeCompactionPolicy.StripeInformationProvider si = createStripesL0Only(3, 10);
228 StripeCompactionPolicy policy = createPolicy(conf);
229 verifyNoCompaction(policy, si);
230
231 si = createStripes(3, KEY_A);
232 verifyNoCompaction(policy, si);
233 }
234
235 @Test
236 public void testSplitOffStripe() throws Exception {
237 Configuration conf = HBaseConfiguration.create();
238
239 conf.setInt(StripeStoreConfig.MIN_FILES_KEY, 2);
240 Long[] toSplit = new Long[] { defaultSplitSize - 2, 1L, 1L };
241 Long[] noSplit = new Long[] { defaultSplitSize - 2, 1L };
242 long splitTargetSize = (long)(defaultSplitSize / defaultSplitCount);
243
244 StripeCompactionPolicy.StripeInformationProvider si =
245 createStripesWithSizes(0, 0, new Long[] { defaultSplitSize - 2, 2L });
246 assertNull(createPolicy(conf).selectCompaction(si, al(), false));
247
248 conf.setFloat(CompactionConfiguration.RATIO_KEY, 500f);
249 StripeCompactionPolicy policy = createPolicy(conf);
250 verifyWholeStripesCompaction(policy, si, 0, 0, null, 2, splitTargetSize);
251
252 si = createStripesWithSizes(0, 0, noSplit, noSplit, toSplit);
253 verifyWholeStripesCompaction(policy, si, 2, 2, null, 2, splitTargetSize);
254
255 si = createStripesWithSizes(0, 0, noSplit, toSplit, noSplit);
256 verifyWholeStripesCompaction(policy, si, 1, 1, null, 2, splitTargetSize);
257
258
259 StripeCompactionPolicy specPolicy = createPolicy(
260 conf, defaultSplitSize + 1, defaultSplitCount, defaultInitialCount, false);
261 verifySingleStripeCompaction(specPolicy, si, 1, null);
262 }
263
264 @Test
265 public void testSplitOffStripeDropDeletes() throws Exception {
266 Configuration conf = HBaseConfiguration.create();
267 conf.setInt(StripeStoreConfig.MIN_FILES_KEY, 2);
268 StripeCompactionPolicy policy = createPolicy(conf);
269 Long[] toSplit = new Long[] { defaultSplitSize / 2, defaultSplitSize / 2 };
270 Long[] noSplit = new Long[] { 1L };
271 long splitTargetSize = (long)(defaultSplitSize / defaultSplitCount);
272
273
274 StripeCompactionPolicy.StripeInformationProvider si =
275 createStripesWithSizes(0, 0, noSplit, toSplit);
276 verifyWholeStripesCompaction(policy, si, 1, 1, true, null, splitTargetSize);
277
278 si = createStripesWithSizes(2, 2, noSplit, toSplit);
279 verifyWholeStripesCompaction(policy, si, 1, 1, false, null, splitTargetSize);
280 }
281
282 @SuppressWarnings("unchecked")
283 @Test
284 public void testMergeExpiredFiles() throws Exception {
285 ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
286 long now = defaultTtl + 2;
287 edge.setValue(now);
288 EnvironmentEdgeManager.injectEdge(edge);
289 try {
290 StoreFile expiredFile = createFile(), notExpiredFile = createFile();
291 when(expiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl - 1);
292 when(notExpiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl + 1);
293 List<StoreFile> expired = Lists.newArrayList(expiredFile, expiredFile);
294 List<StoreFile> notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile);
295 List<StoreFile> mixed = Lists.newArrayList(expiredFile, notExpiredFile);
296
297 StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create(),
298 defaultSplitSize, defaultSplitCount, defaultInitialCount, true);
299
300 StripeCompactionPolicy.StripeInformationProvider si =
301 createStripesWithFiles(expired, expired, expired);
302 verifyWholeStripesCompaction(policy, si, 0, 2, null, 1, Long.MAX_VALUE, false);
303
304 si = createStripesWithFiles(notExpired, notExpired, notExpired);
305 assertNull(policy.selectCompaction(si, al(), false));
306
307 si = createStripesWithFiles(notExpired, expired, notExpired);
308 verifyWholeStripesCompaction(policy, si, 1, 2, null, 1, Long.MAX_VALUE, false);
309
310
311 si = createStripesWithFiles(notExpired, expired, notExpired, expired, expired, notExpired);
312 verifyWholeStripesCompaction(policy, si, 3, 4, null, 1, Long.MAX_VALUE, false);
313
314 si = createStripesWithFiles(expired, expired, notExpired, expired, mixed);
315 verifyWholeStripesCompaction(policy, si, 0, 1, null, 1, Long.MAX_VALUE, false);
316 } finally {
317 EnvironmentEdgeManager.reset();
318 }
319 }
320
321 private static StripeCompactionPolicy.StripeInformationProvider createStripesWithFiles(
322 List<StoreFile>... stripeFiles) throws Exception {
323 return createStripesWithFiles(createBoundaries(stripeFiles.length),
324 Lists.newArrayList(stripeFiles), new ArrayList<StoreFile>());
325 }
326
327 @Test
328 public void testSingleStripeDropDeletes() throws Exception {
329 Configuration conf = HBaseConfiguration.create();
330 StripeCompactionPolicy policy = createPolicy(conf);
331
332 Long[][] stripes = new Long[][] { new Long[] { 3L, 2L, 2L, 2L }, new Long[] { 6L } };
333 StripeInformationProvider si = createStripesWithSizes(0, 0, stripes);
334 verifySingleStripeCompaction(policy, si, 0, true);
335
336 si = createStripesWithSizes(2, 2, stripes);
337 verifySingleStripeCompaction(policy, si, 0, false);
338
339 si = createStripesWithSizes(6, 2, stripes);
340 ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<StoreFile>();
341 sfs.addSublist(si.getLevel0Files());
342 sfs.addSublist(si.getStripes().get(0));
343 verifyCompaction(
344 policy, si, sfs, si.getStartRow(0), si.getEndRow(0), si.getStripeBoundaries());
345
346 si = createStripesWithSizes(6, 2,
347 new Long[][] { new Long[] { 10L, 1L, 1L, 1L, 1L }, new Long[] { 12L } });
348 verifyCompaction(policy, si, si.getLevel0Files(), null, null, si.getStripeBoundaries());
349 }
350
351
352 private static StripeCompactionPolicy createPolicy(
353 Configuration conf) throws Exception {
354 return createPolicy(conf, defaultSplitSize, defaultSplitCount, defaultInitialCount, false);
355 }
356
357 private static StripeCompactionPolicy createPolicy(Configuration conf,
358 long splitSize, float splitCount, int initialCount, boolean hasTtl) throws Exception {
359 conf.setLong(StripeStoreConfig.SIZE_TO_SPLIT_KEY, splitSize);
360 conf.setFloat(StripeStoreConfig.SPLIT_PARTS_KEY, splitCount);
361 conf.setInt(StripeStoreConfig.INITIAL_STRIPE_COUNT_KEY, initialCount);
362 StoreConfigInformation sci = mock(StoreConfigInformation.class);
363 when(sci.getStoreFileTtl()).thenReturn(hasTtl ? defaultTtl : Long.MAX_VALUE);
364 StripeStoreConfig ssc = new StripeStoreConfig(conf, sci);
365 return new StripeCompactionPolicy(conf, sci, ssc);
366 }
367
368 private static ArrayList<StoreFile> al(StoreFile... sfs) {
369 return new ArrayList<StoreFile>(Arrays.asList(sfs));
370 }
371
372
373
374
375
376
377
378
379
380
381
382 private void verifyWholeStripesCompaction(StripeCompactionPolicy policy,
383 StripeInformationProvider si, int from, int to, Boolean dropDeletes,
384 Integer count, Long size, boolean needsCompaction) throws IOException {
385 verifyCompaction(policy, si, getAllFiles(si, from, to), dropDeletes,
386 count, size, si.getStartRow(from), si.getEndRow(to), needsCompaction);
387 }
388
389 private void verifyWholeStripesCompaction(StripeCompactionPolicy policy,
390 StripeInformationProvider si, int from, int to, Boolean dropDeletes,
391 Integer count, Long size) throws IOException {
392 verifyWholeStripesCompaction(policy, si, from, to, dropDeletes, count, size, true);
393 }
394
395 private void verifySingleStripeCompaction(StripeCompactionPolicy policy,
396 StripeInformationProvider si, int index, Boolean dropDeletes) throws IOException {
397 verifyWholeStripesCompaction(policy, si, index, index, dropDeletes, 1, null, true);
398 }
399
400
401
402
403
404
405 private void verifyNoCompaction(
406 StripeCompactionPolicy policy, StripeInformationProvider si) throws IOException {
407 assertNull(policy.selectCompaction(si, al(), false));
408 assertFalse(policy.needsCompactions(si, al()));
409 }
410
411
412
413
414
415
416
417
418
419
420 private void verifyCompaction(StripeCompactionPolicy policy, StripeInformationProvider si,
421 Collection<StoreFile> sfs, byte[] dropDeletesFrom, byte[] dropDeletesTo,
422 final List<byte[]> boundaries) throws Exception {
423 StripeCompactor sc = mock(StripeCompactor.class);
424 assertTrue(policy.needsCompactions(si, al()));
425 StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false);
426 verifyCollectionsEqual(sfs, scr.getRequest().getFiles());
427 scr.execute(sc);
428 verify(sc, times(1)).compact(eq(scr.getRequest()), argThat(
429 new ArgumentMatcher<List<byte[]>>() {
430 @Override
431 public boolean matches(Object argument) {
432 @SuppressWarnings("unchecked")
433 List<byte[]> other = (List<byte[]>)argument;
434 if (other.size() != boundaries.size()) return false;
435 for (int i = 0; i < other.size(); ++i) {
436 if (!Bytes.equals(other.get(i), boundaries.get(i))) return false;
437 }
438 return true;
439 }
440 }),
441 dropDeletesFrom == null ? isNull(byte[].class) : aryEq(dropDeletesFrom),
442 dropDeletesTo == null ? isNull(byte[].class) : aryEq(dropDeletesTo));
443 }
444
445
446
447
448
449
450
451
452
453
454
455
456 private void verifyCompaction(StripeCompactionPolicy policy, StripeInformationProvider si,
457 Collection<StoreFile> sfs, Boolean dropDeletes, Integer count, Long size,
458 byte[] start, byte[] end, boolean needsCompaction) throws IOException {
459 StripeCompactor sc = mock(StripeCompactor.class);
460 assertTrue(!needsCompaction || policy.needsCompactions(si, al()));
461 StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false);
462 verifyCollectionsEqual(sfs, scr.getRequest().getFiles());
463 scr.execute(sc);
464 verify(sc, times(1)).compact(eq(scr.getRequest()),
465 count == null ? anyInt() : eq(count.intValue()),
466 size == null ? anyLong() : eq(size.longValue()), aryEq(start), aryEq(end),
467 dropDeletesMatcher(dropDeletes, start), dropDeletesMatcher(dropDeletes, end));
468 }
469
470
471 protected void verifyFlush(StripeCompactionPolicy policy, StripeInformationProvider si,
472 KeyValue[] input, KeyValue[][] expected, byte[][] boundaries) throws IOException {
473 StoreFileWritersCapture writers = new StoreFileWritersCapture();
474 StripeStoreFlusher.StripeFlushRequest req = policy.selectFlush(si, input.length);
475 StripeMultiFileWriter mw = req.createWriter();
476 mw.init(null, writers, new KeyValue.KVComparator());
477 for (KeyValue kv : input) {
478 mw.append(kv);
479 }
480 boolean hasMetadata = boundaries != null;
481 mw.commitWriters(0, false);
482 writers.verifyKvs(expected, true, hasMetadata);
483 if (hasMetadata) {
484 writers.verifyBoundaries(boundaries);
485 }
486 }
487
488
489 private byte[] dropDeletesMatcher(Boolean dropDeletes, byte[] value) {
490 return dropDeletes == null ? any(byte[].class)
491 : (dropDeletes.booleanValue() ? aryEq(value) : isNull(byte[].class));
492 }
493
494 private void verifyCollectionsEqual(Collection<StoreFile> sfs, Collection<StoreFile> scr) {
495
496 assertEquals(sfs.size(), scr.size());
497 assertTrue(scr.containsAll(sfs));
498 }
499
500 private static List<StoreFile> getAllFiles(
501 StripeInformationProvider si, int fromStripe, int toStripe) {
502 ArrayList<StoreFile> expected = new ArrayList<StoreFile>();
503 for (int i = fromStripe; i <= toStripe; ++i) {
504 expected.addAll(si.getStripes().get(i));
505 }
506 return expected;
507 }
508
509
510
511
512
513
514 private static StripeInformationProvider createStripes(
515 int l0Count, byte[]... boundaries) throws Exception {
516 List<Long> l0Sizes = new ArrayList<Long>();
517 for (int i = 0; i < l0Count; ++i) {
518 l0Sizes.add(5L);
519 }
520 List<List<Long>> sizes = new ArrayList<List<Long>>();
521 for (int i = 0; i <= boundaries.length; ++i) {
522 sizes.add(Arrays.asList(Long.valueOf(5)));
523 }
524 return createStripes(Arrays.asList(boundaries), sizes, l0Sizes);
525 }
526
527
528
529
530
531
532 private static StripeInformationProvider createStripesL0Only(
533 int l0Count, long l0Size) throws Exception {
534 List<Long> l0Sizes = new ArrayList<Long>();
535 for (int i = 0; i < l0Count; ++i) {
536 l0Sizes.add(l0Size);
537 }
538 return createStripes(null, new ArrayList<List<Long>>(), l0Sizes);
539 }
540
541
542
543
544
545
546
547 private static StripeInformationProvider createStripesWithSizes(
548 int l0Count, long l0Size, Long[]... sizes) throws Exception {
549 ArrayList<List<Long>> sizeList = new ArrayList<List<Long>>();
550 for (Long[] size : sizes) {
551 sizeList.add(Arrays.asList(size));
552 }
553 return createStripesWithSizes(l0Count, l0Size, sizeList);
554 }
555
556 private static StripeInformationProvider createStripesWithSizes(
557 int l0Count, long l0Size, List<List<Long>> sizes) throws Exception {
558 List<byte[]> boundaries = createBoundaries(sizes.size());
559 List<Long> l0Sizes = new ArrayList<Long>();
560 for (int i = 0; i < l0Count; ++i) {
561 l0Sizes.add(l0Size);
562 }
563 return createStripes(boundaries, sizes, l0Sizes);
564 }
565
566 private static List<byte[]> createBoundaries(int stripeCount) {
567 byte[][] keys = new byte[][] { KEY_A, KEY_B, KEY_C, KEY_D, KEY_E };
568 assert stripeCount <= keys.length + 1;
569 List<byte[]> boundaries = new ArrayList<byte[]>();
570 for (int i = 0; i < stripeCount - 1; ++i) {
571 boundaries.add(keys[i]);
572 }
573 return boundaries;
574 }
575
576 private static StripeInformationProvider createStripes(List<byte[]> boundaries,
577 List<List<Long>> stripeSizes, List<Long> l0Sizes) throws Exception {
578 List<List<StoreFile>> stripeFiles = new ArrayList<List<StoreFile>>(stripeSizes.size());
579 for (List<Long> sizes : stripeSizes) {
580 List<StoreFile> sfs = new ArrayList<StoreFile>();
581 for (Long size : sizes) {
582 sfs.add(createFile(size));
583 }
584 stripeFiles.add(sfs);
585 }
586 List<StoreFile> l0Files = new ArrayList<StoreFile>();
587 for (Long size : l0Sizes) {
588 l0Files.add(createFile(size));
589 }
590 return createStripesWithFiles(boundaries, stripeFiles, l0Files);
591 }
592
593
594
595
596 private static StripeInformationProvider createStripesWithFiles(List<byte[]> boundaries,
597 List<List<StoreFile>> stripeFiles, List<StoreFile> l0Files) throws Exception {
598 ArrayList<ImmutableList<StoreFile>> stripes = new ArrayList<ImmutableList<StoreFile>>();
599 ArrayList<byte[]> boundariesList = new ArrayList<byte[]>();
600 StripeInformationProvider si = mock(StripeInformationProvider.class);
601 if (!stripeFiles.isEmpty()) {
602 assert stripeFiles.size() == (boundaries.size() + 1);
603 boundariesList.add(OPEN_KEY);
604 for (int i = 0; i <= boundaries.size(); ++i) {
605 byte[] startKey = ((i == 0) ? OPEN_KEY : boundaries.get(i - 1));
606 byte[] endKey = ((i == boundaries.size()) ? OPEN_KEY : boundaries.get(i));
607 boundariesList.add(endKey);
608 for (StoreFile sf : stripeFiles.get(i)) {
609 setFileStripe(sf, startKey, endKey);
610 }
611 stripes.add(ImmutableList.copyOf(stripeFiles.get(i)));
612 when(si.getStartRow(eq(i))).thenReturn(startKey);
613 when(si.getEndRow(eq(i))).thenReturn(endKey);
614 }
615 }
616 ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<StoreFile>();
617 sfs.addAllSublists(stripes);
618 sfs.addSublist(l0Files);
619 when(si.getStorefiles()).thenReturn(sfs);
620 when(si.getStripes()).thenReturn(stripes);
621 when(si.getStripeBoundaries()).thenReturn(boundariesList);
622 when(si.getStripeCount()).thenReturn(stripes.size());
623 when(si.getLevel0Files()).thenReturn(l0Files);
624 return si;
625 }
626
627 private static StoreFile createFile(long size) throws Exception {
628 StoreFile sf = mock(StoreFile.class);
629 when(sf.getPath()).thenReturn(new Path("moo"));
630 StoreFile.Reader r = mock(StoreFile.Reader.class);
631 when(r.getEntries()).thenReturn(size);
632 when(r.length()).thenReturn(size);
633 when(sf.getReader()).thenReturn(r);
634 return sf;
635 }
636
637 private static StoreFile createFile() throws Exception {
638 return createFile(0);
639 }
640
641 private static void setFileStripe(StoreFile sf, byte[] startKey, byte[] endKey) {
642 when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_START_KEY)).thenReturn(startKey);
643 when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_END_KEY)).thenReturn(endKey);
644 }
645 }