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 junit.framework.TestCase;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.fs.FileSystem;
31 import org.apache.hadoop.fs.Path;
32 import org.apache.hadoop.hbase.HBaseTestingUtility;
33 import org.apache.hadoop.hbase.HColumnDescriptor;
34 import org.apache.hadoop.hbase.HConstants;
35 import org.apache.hadoop.hbase.HRegionInfo;
36 import org.apache.hadoop.hbase.HTableDescriptor;
37 import org.apache.hadoop.hbase.SmallTests;
38 import org.apache.hadoop.hbase.TableName;
39 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
40 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
41 import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
42 import org.apache.hadoop.hbase.regionserver.wal.HLog;
43 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
44 import org.apache.hadoop.hbase.util.Bytes;
45 import org.apache.hadoop.hbase.util.FSUtils;
46 import org.junit.After;
47 import org.junit.experimental.categories.Category;
48
49 import com.google.common.collect.Lists;
50
51 @Category(SmallTests.class)
52 public class TestDefaultCompactSelection extends TestCase {
53 private final static Log LOG = LogFactory.getLog(TestDefaultCompactSelection.class);
54 private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
55
56 protected Configuration conf;
57 protected HStore store;
58 private static final String DIR=
59 TEST_UTIL.getDataTestDir(TestDefaultCompactSelection.class.getSimpleName()).toString();
60 private static Path TEST_FILE;
61
62 protected static final int minFiles = 3;
63 protected static final int maxFiles = 5;
64
65 protected static final long minSize = 10;
66 protected static final long maxSize = 2100;
67
68 private HLog hlog;
69 private HRegion region;
70
71 @Override
72 public void setUp() throws Exception {
73
74 this.conf = TEST_UTIL.getConfiguration();
75 this.conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0);
76 this.conf.setInt("hbase.hstore.compaction.min", minFiles);
77 this.conf.setInt("hbase.hstore.compaction.max", maxFiles);
78 this.conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, minSize);
79 this.conf.setLong("hbase.hstore.compaction.max.size", maxSize);
80 this.conf.setFloat("hbase.hstore.compaction.ratio", 1.0F);
81
82
83 Path basedir = new Path(DIR);
84 String logName = "logs";
85 Path logdir = new Path(DIR, logName);
86 HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("family"));
87 FileSystem fs = FileSystem.get(conf);
88
89 fs.delete(logdir, true);
90
91 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("table")));
92 htd.addFamily(hcd);
93 HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
94
95 hlog = HLogFactory.createHLog(fs, basedir, logName, conf);
96 region = HRegion.createHRegion(info, basedir, conf, htd);
97 HRegion.closeHRegion(region);
98 Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
99 region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
100
101 store = new HStore(region, hcd, conf);
102
103 TEST_FILE = region.getRegionFileSystem().createTempName();
104 fs.createNewFile(TEST_FILE);
105 }
106
107 @After
108 public void tearDown() throws IOException {
109 IOException ex = null;
110 try {
111 region.close();
112 } catch (IOException e) {
113 LOG.warn("Caught Exception", e);
114 ex = e;
115 }
116 try {
117 hlog.closeAndDelete();
118 } catch (IOException e) {
119 LOG.warn("Caught Exception", e);
120 ex = e;
121 }
122 if (ex != null) {
123 throw ex;
124 }
125 }
126
127
128 static class MockStoreFile extends StoreFile {
129 long length = 0;
130 boolean isRef = false;
131 long ageInDisk;
132 long sequenceid;
133 TimeRangeTracker timeRangeTracker;
134 long entryCount;
135
136 MockStoreFile(long length, long ageInDisk, boolean isRef, long sequenceid) throws IOException {
137 super(TEST_UTIL.getTestFileSystem(), TEST_FILE, TEST_UTIL.getConfiguration(),
138 new CacheConfig(TEST_UTIL.getConfiguration()), BloomType.NONE);
139 this.length = length;
140 this.isRef = isRef;
141 this.ageInDisk = ageInDisk;
142 this.sequenceid = sequenceid;
143 }
144
145 void setLength(long newLen) {
146 this.length = newLen;
147 }
148
149 @Override
150 public long getMaxSequenceId() {
151 return sequenceid;
152 }
153
154 @Override
155 public boolean isMajorCompaction() {
156 return false;
157 }
158
159 @Override
160 public boolean isReference() {
161 return this.isRef;
162 }
163
164 void setTimeRangeTracker(TimeRangeTracker timeRangeTracker) {
165 this.timeRangeTracker = timeRangeTracker;
166 }
167
168 void setEntries(long entryCount) {
169 this.entryCount = entryCount;
170 }
171
172 @Override
173 public StoreFile.Reader getReader() {
174 final long len = this.length;
175 final TimeRangeTracker timeRange = this.timeRangeTracker;
176 final long entries = this.entryCount;
177 return new StoreFile.Reader() {
178 @Override
179 public long length() {
180 return len;
181 }
182
183 @Override
184 public long getMaxTimestamp() {
185 return timeRange == null ? Long.MAX_VALUE
186 : timeRange.maximumTimestamp;
187 }
188
189 @Override
190 public long getEntries() {
191 return entries;
192 }
193 };
194 }
195
196 @Override
197 public String toString() {
198 return "MockStoreFile{" +
199 "length=" + length +
200 ", isRef=" + isRef +
201 ", ageInDisk=" + ageInDisk +
202 ", sequenceid=" + sequenceid +
203 '}';
204 }
205 }
206
207 ArrayList<Long> toArrayList(long... numbers) {
208 ArrayList<Long> result = new ArrayList<Long>();
209 for (long i : numbers) {
210 result.add(i);
211 }
212 return result;
213 }
214
215 List<StoreFile> sfCreate(long... sizes) throws IOException {
216 ArrayList<Long> ageInDisk = new ArrayList<Long>();
217 for (int i = 0; i < sizes.length; i++) {
218 ageInDisk.add(0L);
219 }
220 return sfCreate(toArrayList(sizes), ageInDisk);
221 }
222
223 List<StoreFile> sfCreate(ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
224 throws IOException {
225 return sfCreate(false, sizes, ageInDisk);
226 }
227
228 List<StoreFile> sfCreate(boolean isReference, long... sizes) throws IOException {
229 ArrayList<Long> ageInDisk = new ArrayList<Long>(sizes.length);
230 for (int i = 0; i < sizes.length; i++) {
231 ageInDisk.add(0L);
232 }
233 return sfCreate(isReference, toArrayList(sizes), ageInDisk);
234 }
235
236 List<StoreFile> sfCreate(boolean isReference, ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
237 throws IOException {
238 List<StoreFile> ret = Lists.newArrayList();
239 for (int i = 0; i < sizes.size(); i++) {
240 ret.add(new MockStoreFile(sizes.get(i), ageInDisk.get(i), isReference, i));
241 }
242 return ret;
243 }
244
245 long[] getSizes(List<StoreFile> sfList) {
246 long[] aNums = new long[sfList.size()];
247 for (int i = 0; i < sfList.size(); ++i) {
248 aNums[i] = sfList.get(i).getReader().length();
249 }
250 return aNums;
251 }
252
253 void compactEquals(List<StoreFile> candidates, long... expected)
254 throws IOException {
255 compactEquals(candidates, false, false, expected);
256 }
257
258 void compactEquals(List<StoreFile> candidates, boolean forcemajor, long... expected)
259 throws IOException {
260 compactEquals(candidates, forcemajor, false, expected);
261 }
262
263 void compactEquals(List<StoreFile> candidates, boolean forcemajor, boolean isOffPeak,
264 long ... expected)
265 throws IOException {
266 store.forceMajor = forcemajor;
267
268 CompactionRequest result = ((RatioBasedCompactionPolicy)store.storeEngine.getCompactionPolicy())
269 .selectCompaction(candidates, new ArrayList<StoreFile>(), false, isOffPeak, forcemajor);
270 List<StoreFile> actual = new ArrayList<StoreFile>(result.getFiles());
271 if (isOffPeak && !forcemajor) {
272 assertTrue(result.isOffPeak());
273 }
274 assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
275 store.forceMajor = false;
276 }
277
278 public void testCompactionRatio() throws IOException {
279
280
281
282
283
284 long tooBig = maxSize + 1;
285
286
287 compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
288
289 compactEquals(sfCreate(100,50,25,12,12)
290
291 compactEquals(sfCreate(tooBig, tooBig, 700, 700, 700), 700, 700, 700);
292
293 compactEquals(sfCreate(tooBig, tooBig, 700,700)
294
295 compactEquals(sfCreate(7,1,1), 7,1,1);
296
297
298
299 compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
300
301 compactEquals(sfCreate(50, 10, 10 ,10, 10), 10, 10, 10, 10);
302
303 compactEquals(sfCreate(10, 10, 10, 10, 50), 10, 10, 10, 10);
304
305 compactEquals(sfCreate(251, 253, 251, maxSize -1), 251, 253, 251);
306
307 compactEquals(sfCreate(maxSize -1,maxSize -1,maxSize -1)
308
309
310 this.conf.setLong("hbase.hstore.compaction.min.size", 1);
311 store.storeEngine.getCompactionPolicy().setConf(conf);
312 compactEquals(sfCreate(512,256,128,64,32,16,8,4,2,1), 4,2,1);
313 this.conf.setLong("hbase.hstore.compaction.min.size", minSize);
314 store.storeEngine.getCompactionPolicy().setConf(conf);
315
316
317
318 compactEquals(sfCreate(50,25,12,12), true, 50, 25, 12, 12);
319
320 compactEquals(sfCreate(12,12), true, 12, 12);
321
322 compactEquals(sfCreate(tooBig, 12,12), true, tooBig, 12, 12);
323
324 store.forceMajor = true;
325 compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
326 store.forceMajor = false;
327
328
329
330 compactEquals(sfCreate(100,50,23,12,12), true, 23, 12, 12);
331 conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1);
332 conf.setFloat("hbase.hregion.majorcompaction.jitter", 0);
333 store.storeEngine.getCompactionPolicy().setConf(conf);
334 try {
335
336 compactEquals(sfCreate(50,25,12,12), 50, 25, 12, 12);
337
338 compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
339 } finally {
340 conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
341 conf.setFloat("hbase.hregion.majorcompaction.jitter", 0.20F);
342 }
343
344
345
346 compactEquals(sfCreate(true, 100,50,25,12,12), 100, 50, 25, 12, 12);
347
348 compactEquals(sfCreate(true, tooBig, 12,12), tooBig, 12, 12);
349
350 compactEquals(sfCreate(true, 7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
351
352
353 compactEquals(new ArrayList<StoreFile>()
354
355 compactEquals(sfCreate(tooBig, tooBig)
356 }
357
358 public void testOffPeakCompactionRatio() throws IOException {
359
360
361
362
363
364
365 this.conf.setFloat("hbase.hstore.compaction.ratio.offpeak", 5.0F);
366 store.storeEngine.getCompactionPolicy().setConf(this.conf);
367
368 compactEquals(sfCreate(999, 50, 12, 12, 1), false, true, 50, 12, 12, 1);
369 compactEquals(sfCreate(999, 50, 12, 12, 1), 12, 12, 1);
370 }
371
372 public void testStuckStoreCompaction() throws IOException {
373
374 compactEquals(sfCreate(99,99,99,99,99,99, 30,30,30,30), 30, 30, 30);
375
376 compactEquals(sfCreate(99,99,99,99,99, 30,30,30,30), 99, 30, 30, 30, 30);
377
378
379 compactEquals(sfCreate(99,99,99,99,99,99, 30,30,30,15), 30, 30, 30, 15);
380
381 compactEquals(sfCreate(99,99,99,99, 30,26,26,29,25,25), 30, 26, 26);
382
383 compactEquals(sfCreate(99,99,99,99, 27,27,27,20,20,20), 20, 20, 20);
384 }
385
386 public void testCompactionEmptyHFile() throws IOException {
387
388 ScanInfo oldScanInfo = store.getScanInfo();
389 ScanInfo newScanInfo = new ScanInfo(oldScanInfo.getFamily(),
390 oldScanInfo.getMinVersions(), oldScanInfo.getMaxVersions(), 600,
391 oldScanInfo.getKeepDeletedCells(), oldScanInfo.getTimeToPurgeDeletes(),
392 oldScanInfo.getComparator());
393 store.setScanInfo(newScanInfo);
394
395 List<StoreFile> candidates = sfCreate(0);
396 for (StoreFile file : candidates) {
397 if (file instanceof MockStoreFile) {
398 MockStoreFile mockFile = (MockStoreFile) file;
399 mockFile.setTimeRangeTracker(new TimeRangeTracker(-1, -1));
400 mockFile.setEntries(0);
401 }
402 }
403
404 CompactionRequest result = ((RatioBasedCompactionPolicy) store.storeEngine
405 .getCompactionPolicy()).selectCompaction(candidates,
406 new ArrayList<StoreFile>(), false, false, false);
407 assertTrue(result.getFiles().size() == 0);
408 store.setScanInfo(oldScanInfo);
409 }
410 }