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
134 MockStoreFile(long length, long ageInDisk, boolean isRef, long sequenceid) throws IOException {
135 super(TEST_UTIL.getTestFileSystem(), TEST_FILE, TEST_UTIL.getConfiguration(),
136 new CacheConfig(TEST_UTIL.getConfiguration()), BloomType.NONE);
137 this.length = length;
138 this.isRef = isRef;
139 this.ageInDisk = ageInDisk;
140 this.sequenceid = sequenceid;
141 }
142
143 void setLength(long newLen) {
144 this.length = newLen;
145 }
146
147 @Override
148 public long getMaxSequenceId() {
149 return sequenceid;
150 }
151
152 @Override
153 public boolean isMajorCompaction() {
154 return false;
155 }
156
157 @Override
158 public boolean isReference() {
159 return this.isRef;
160 }
161
162 @Override
163 public StoreFile.Reader getReader() {
164 final long len = this.length;
165 return new StoreFile.Reader() {
166 @Override
167 public long length() {
168 return len;
169 }
170 };
171 }
172
173 @Override
174 public String toString() {
175 return "MockStoreFile{" +
176 "length=" + length +
177 ", isRef=" + isRef +
178 ", ageInDisk=" + ageInDisk +
179 ", sequenceid=" + sequenceid +
180 '}';
181 }
182 }
183
184 ArrayList<Long> toArrayList(long... numbers) {
185 ArrayList<Long> result = new ArrayList<Long>();
186 for (long i : numbers) {
187 result.add(i);
188 }
189 return result;
190 }
191
192 List<StoreFile> sfCreate(long... sizes) throws IOException {
193 ArrayList<Long> ageInDisk = new ArrayList<Long>();
194 for (int i = 0; i < sizes.length; i++) {
195 ageInDisk.add(0L);
196 }
197 return sfCreate(toArrayList(sizes), ageInDisk);
198 }
199
200 List<StoreFile> sfCreate(ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
201 throws IOException {
202 return sfCreate(false, sizes, ageInDisk);
203 }
204
205 List<StoreFile> sfCreate(boolean isReference, long... sizes) throws IOException {
206 ArrayList<Long> ageInDisk = new ArrayList<Long>(sizes.length);
207 for (int i = 0; i < sizes.length; i++) {
208 ageInDisk.add(0L);
209 }
210 return sfCreate(isReference, toArrayList(sizes), ageInDisk);
211 }
212
213 List<StoreFile> sfCreate(boolean isReference, ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
214 throws IOException {
215 List<StoreFile> ret = Lists.newArrayList();
216 for (int i = 0; i < sizes.size(); i++) {
217 ret.add(new MockStoreFile(sizes.get(i), ageInDisk.get(i), isReference, i));
218 }
219 return ret;
220 }
221
222 long[] getSizes(List<StoreFile> sfList) {
223 long[] aNums = new long[sfList.size()];
224 for (int i = 0; i < sfList.size(); ++i) {
225 aNums[i] = sfList.get(i).getReader().length();
226 }
227 return aNums;
228 }
229
230 void compactEquals(List<StoreFile> candidates, long... expected)
231 throws IOException {
232 compactEquals(candidates, false, false, expected);
233 }
234
235 void compactEquals(List<StoreFile> candidates, boolean forcemajor, long... expected)
236 throws IOException {
237 compactEquals(candidates, forcemajor, false, expected);
238 }
239
240 void compactEquals(List<StoreFile> candidates, boolean forcemajor, boolean isOffPeak,
241 long ... expected)
242 throws IOException {
243 store.forceMajor = forcemajor;
244
245 CompactionRequest result = ((RatioBasedCompactionPolicy)store.storeEngine.getCompactionPolicy())
246 .selectCompaction(candidates, new ArrayList<StoreFile>(), false, isOffPeak, forcemajor);
247 List<StoreFile> actual = new ArrayList<StoreFile>(result.getFiles());
248 if (isOffPeak && !forcemajor) {
249 assertTrue(result.isOffPeak());
250 }
251 assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
252 store.forceMajor = false;
253 }
254
255 public void testCompactionRatio() throws IOException {
256
257
258
259
260
261 long tooBig = maxSize + 1;
262
263
264 compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
265
266 compactEquals(sfCreate(100,50,25,12,12)
267
268 compactEquals(sfCreate(tooBig, tooBig, 700, 700, 700), 700, 700, 700);
269
270 compactEquals(sfCreate(tooBig, tooBig, 700,700)
271
272 compactEquals(sfCreate(7,1,1), 7,1,1);
273
274
275
276 compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
277
278 compactEquals(sfCreate(50, 10, 10 ,10, 10), 10, 10, 10, 10);
279
280 compactEquals(sfCreate(10, 10, 10, 10, 50), 10, 10, 10, 10);
281
282 compactEquals(sfCreate(251, 253, 251, maxSize -1), 251, 253, 251);
283
284 compactEquals(sfCreate(maxSize -1,maxSize -1,maxSize -1)
285
286
287 this.conf.setLong("hbase.hstore.compaction.min.size", 1);
288 store.storeEngine.getCompactionPolicy().setConf(conf);
289 compactEquals(sfCreate(512,256,128,64,32,16,8,4,2,1), 4,2,1);
290 this.conf.setLong("hbase.hstore.compaction.min.size", minSize);
291 store.storeEngine.getCompactionPolicy().setConf(conf);
292
293
294
295 compactEquals(sfCreate(50,25,12,12), true, 50, 25, 12, 12);
296
297 compactEquals(sfCreate(12,12), true, 12, 12);
298
299 compactEquals(sfCreate(tooBig, 12,12), true, tooBig, 12, 12);
300
301 store.forceMajor = true;
302 compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
303 store.forceMajor = false;
304
305
306
307 compactEquals(sfCreate(100,50,23,12,12), true, 23, 12, 12);
308 conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1);
309 conf.setFloat("hbase.hregion.majorcompaction.jitter", 0);
310 store.storeEngine.getCompactionPolicy().setConf(conf);
311 try {
312
313 compactEquals(sfCreate(50,25,12,12), 50, 25, 12, 12);
314
315 compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
316 } finally {
317 conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
318 conf.setFloat("hbase.hregion.majorcompaction.jitter", 0.20F);
319 }
320
321
322
323 compactEquals(sfCreate(true, 100,50,25,12,12), 100, 50, 25, 12, 12);
324
325 compactEquals(sfCreate(true, tooBig, 12,12), tooBig, 12, 12);
326
327 compactEquals(sfCreate(true, 7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
328
329
330 compactEquals(new ArrayList<StoreFile>()
331
332 compactEquals(sfCreate(tooBig, tooBig)
333 }
334
335 public void testOffPeakCompactionRatio() throws IOException {
336
337
338
339
340
341
342 this.conf.setFloat("hbase.hstore.compaction.ratio.offpeak", 5.0F);
343 store.storeEngine.getCompactionPolicy().setConf(this.conf);
344
345 compactEquals(sfCreate(999, 50, 12, 12, 1), false, true, 50, 12, 12, 1);
346 compactEquals(sfCreate(999, 50, 12, 12, 1), 12, 12, 1);
347 }
348
349 public void testStuckStoreCompaction() throws IOException {
350
351 compactEquals(sfCreate(99,99,99,99,99,99, 30,30,30,30), 30, 30, 30);
352
353 compactEquals(sfCreate(99,99,99,99,99, 30,30,30,30), 99, 30, 30, 30, 30);
354
355
356 compactEquals(sfCreate(99,99,99,99,99,99, 30,30,30,15), 30, 30, 30, 15);
357
358 compactEquals(sfCreate(99,99,99,99, 30,26,26,29,25,25), 30, 26, 26);
359
360 compactEquals(sfCreate(99,99,99,99, 27,27,27,20,20,20), 20, 20, 20);
361 }
362 }