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