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