1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.Arrays;
25 import java.util.Calendar;
26 import java.util.GregorianCalendar;
27 import java.util.List;
28
29 import junit.framework.TestCase;
30
31 import org.apache.commons.logging.Log;
32 import org.apache.commons.logging.LogFactory;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.fs.FileSystem;
35 import org.apache.hadoop.fs.Path;
36 import org.apache.hadoop.hbase.HBaseTestingUtility;
37 import org.apache.hadoop.hbase.HColumnDescriptor;
38 import org.apache.hadoop.hbase.HConstants;
39 import org.apache.hadoop.hbase.HRegionInfo;
40 import org.apache.hadoop.hbase.HTableDescriptor;
41 import org.apache.hadoop.hbase.SmallTests;
42 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
43 import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
44 import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection;
45 import org.apache.hadoop.hbase.regionserver.wal.HLog;
46 import org.apache.hadoop.hbase.util.Bytes;
47 import org.junit.experimental.categories.Category;
48
49 import com.google.common.collect.Lists;
50
51 @Category(SmallTests.class)
52 public class TestCompactSelection extends TestCase {
53 private final static Log LOG = LogFactory.getLog(TestCompactSelection.class);
54 private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
55
56 private Configuration conf;
57 private Store store;
58 private static final String DIR=
59 TEST_UTIL.getDataTestDir("TestCompactSelection").toString();
60 private static Path TEST_FILE;
61
62 private static final int minFiles = 3;
63 private static final int maxFiles = 5;
64
65 private static final long minSize = 10;
66 private static final long maxSize = 1000;
67
68
69 @Override
70 public void setUp() throws Exception {
71
72 this.conf = TEST_UTIL.getConfiguration();
73 this.conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0);
74 this.conf.setInt("hbase.hstore.compaction.min", minFiles);
75 this.conf.setInt("hbase.hstore.compaction.max", maxFiles);
76 this.conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, minSize);
77 this.conf.setLong("hbase.hstore.compaction.max.size", maxSize);
78 this.conf.setFloat("hbase.hstore.compaction.ratio", 1.0F);
79
80
81 Path basedir = new Path(DIR);
82 Path logdir = new Path(DIR+"/logs");
83 Path oldLogDir = new Path(basedir, HConstants.HREGION_OLDLOGDIR_NAME);
84 HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("family"));
85 FileSystem fs = FileSystem.get(conf);
86
87 fs.delete(logdir, true);
88
89 HTableDescriptor htd = new HTableDescriptor(Bytes.toBytes("table"));
90 htd.addFamily(hcd);
91 HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
92
93 HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
94 HRegion region = HRegion.createHRegion(info, basedir, conf, htd);
95 HRegion.closeHRegion(region);
96 Path tableDir = new Path(basedir, Bytes.toString(htd.getName()));
97 region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
98
99 store = new Store(basedir, region, hcd, fs, conf);
100 TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir());
101 fs.create(TEST_FILE);
102 }
103
104
105 static class MockStoreFile extends StoreFile {
106 long length = 0;
107 boolean isRef = false;
108 TimeRangeTracker timeRangeTracker;
109 long entryCount;
110
111 MockStoreFile(long length, boolean isRef) throws IOException {
112 super(TEST_UTIL.getTestFileSystem(), TEST_FILE,
113 TEST_UTIL.getConfiguration(),
114 new CacheConfig(TEST_UTIL.getConfiguration()), BloomType.NONE,
115 NoOpDataBlockEncoder.INSTANCE);
116 this.length = length;
117 this.isRef = isRef;
118 }
119
120 void setLength(long newLen) {
121 this.length = newLen;
122 }
123
124 @Override
125 boolean isMajorCompaction() {
126 return false;
127 }
128
129 @Override
130 boolean isReference() {
131 return this.isRef;
132 }
133
134 void setTimeRangeTracker(TimeRangeTracker timeRangeTracker) {
135 this.timeRangeTracker = timeRangeTracker;
136 }
137
138 void setEntries(long entryCount) {
139 this.entryCount = entryCount;
140 }
141
142 @Override
143 public StoreFile.Reader getReader() {
144 final long len = this.length;
145 final TimeRangeTracker timeRange = this.timeRangeTracker;
146 final long entries = this.entryCount;
147 return new StoreFile.Reader() {
148 @Override
149 public long length() {
150 return len;
151 }
152
153 @Override
154 public long getMaxTimestamp() {
155 return timeRange == null ? Long.MAX_VALUE
156 : timeRange.maximumTimestamp;
157 }
158
159 @Override
160 public long getEntries() {
161 return entries;
162 }
163 };
164 }
165 }
166
167 List<StoreFile> sfCreate(long ... sizes) throws IOException {
168 return sfCreate(false, sizes);
169 }
170
171 List<StoreFile> sfCreate(boolean isReference, long ... sizes)
172 throws IOException {
173 List<StoreFile> ret = Lists.newArrayList();
174 for (long i : sizes) {
175 ret.add(new MockStoreFile(i, isReference));
176 }
177 return ret;
178 }
179
180 long[] getSizes(List<StoreFile> sfList) {
181 long[] aNums = new long[sfList.size()];
182 for (int i=0; i <sfList.size(); ++i) {
183 aNums[i] = sfList.get(i).getReader().length();
184 }
185 return aNums;
186 }
187
188 void compactEquals(List<StoreFile> candidates, long ... expected)
189 throws IOException {
190 compactEquals(candidates, false, expected);
191 }
192
193 void compactEquals(List<StoreFile> candidates, boolean forcemajor,
194 long ... expected)
195 throws IOException {
196 store.forceMajor = forcemajor;
197 List<StoreFile> actual = store.compactSelection(candidates).getFilesToCompact();
198 store.forceMajor = false;
199 assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
200 }
201
202 public void testCompactionRatio() throws IOException {
203
204
205
206
207
208 long tooBig = maxSize + 1;
209
210
211 compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
212
213 compactEquals(sfCreate(100,50,25,12,12)
214
215 compactEquals(sfCreate(tooBig, tooBig, 700, 700, 700), 700, 700, 700);
216
217 compactEquals(sfCreate(tooBig, tooBig, 700,700)
218
219 compactEquals(sfCreate(8,3,1), 8,3,1);
220
221
222
223
224
225
226 assertEquals(maxFiles,
227 store.compactSelection(sfCreate(7,6,5,4,3,2,1)).getFilesToCompact().size());
228
229 compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
230
231
232
233 compactEquals(sfCreate(50,25,12,12), true, 50, 25, 12, 12);
234
235 compactEquals(sfCreate(12,12), true, 12, 12);
236
237 compactEquals(sfCreate(tooBig, 12,12), true, tooBig, 12, 12);
238
239 store.forceMajor = true;
240 compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
241 store.forceMajor = false;
242
243
244
245
246 compactEquals(sfCreate(100,50,23,12,12), true, 23, 12, 12);
247 conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1);
248 conf.setFloat("hbase.hregion.majorcompaction.jitter", 0);
249 try {
250
251 compactEquals(sfCreate(50,25,12,12), 50, 25, 12, 12);
252
253 compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
254 } finally {
255 conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
256 conf.setFloat("hbase.hregion.majorcompaction.jitter", 0.20F);
257 }
258
259
260
261 compactEquals(sfCreate(true, 100,50,25,12,12), 100, 50, 25, 12, 12);
262
263 compactEquals(sfCreate(true, tooBig, 12,12), tooBig, 12, 12);
264
265 assertEquals(maxFiles,
266 store.compactSelection(sfCreate(true, 7,6,5,4,3,2,1)).getFilesToCompact().size());
267
268 compactEquals(sfCreate(true, 7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
269
270
271 compactEquals(new ArrayList<StoreFile>()
272
273 compactEquals(sfCreate(tooBig, tooBig)
274 }
275
276 public void testOffPeakCompactionRatio() throws IOException {
277
278
279
280
281
282 long tooBig = maxSize + 1;
283
284 Calendar calendar = new GregorianCalendar();
285 int hourOfDay = calendar.get(Calendar.HOUR_OF_DAY);
286 LOG.debug("Hour of day = " + hourOfDay);
287 int hourPlusOne = ((hourOfDay+1+24)%24);
288 int hourMinusOne = ((hourOfDay-1+24)%24);
289 int hourMinusTwo = ((hourOfDay-2+24)%24);
290
291
292 LOG.debug("Testing compact selection without off-peak settings...");
293 compactEquals(sfCreate(999,50,12,12,1), 12, 12, 1);
294
295
296 this.conf.setFloat("hbase.hstore.compaction.ratio.offpeak", 5.0F);
297
298
299 this.conf.setLong("hbase.offpeak.start.hour", hourMinusOne);
300 this.conf.setLong("hbase.offpeak.end.hour", hourPlusOne);
301 LOG.debug("Testing compact selection with off-peak settings (" +
302 hourMinusOne + ", " + hourPlusOne + ")");
303 compactEquals(sfCreate(999,50,12,12, 1), 50, 12, 12, 1);
304
305
306 this.conf.setLong("hbase.offpeak.start.hour", hourMinusTwo);
307 this.conf.setLong("hbase.offpeak.end.hour", hourMinusOne);
308 LOG.debug("Testing compact selection with off-peak settings (" +
309 hourMinusTwo + ", " + hourMinusOne + ")");
310 compactEquals(sfCreate(999,50,12,12, 1), 12, 12, 1);
311 }
312
313 public void testCompactionEmptyHFile() throws IOException {
314
315 List<StoreFile> candidates = sfCreate(0);
316 for (StoreFile file : candidates) {
317 if (file instanceof MockStoreFile) {
318 MockStoreFile mockFile = (MockStoreFile) file;
319 mockFile.setTimeRangeTracker(new TimeRangeTracker(-1, -1));
320 mockFile.setEntries(0);
321 }
322 }
323
324 CompactSelection compactSelection = new CompactSelection(conf, candidates);
325 CompactSelection result = compactSelection
326 .selectExpiredStoreFilesToCompact(600L);
327 assertTrue(result == null);
328 }
329
330 @org.junit.Rule
331 public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
332 new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
333 }
334