1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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      // setup config values necessary for store
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      //Setting up a Store
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   // used so our tests don't deal with actual StoreFiles
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      * NOTE: these tests are specific to describe the implementation of the
205      * current compaction algorithm.  Developed to ensure that refactoring
206      * doesn't implicitly alter this.
207      */
208     long tooBig = maxSize + 1;
209 
210     // default case. preserve user ratio on size
211     compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
212     // less than compact threshold = don't compact
213     compactEquals(sfCreate(100,50,25,12,12) /* empty */);
214     // greater than compact size = skip those
215     compactEquals(sfCreate(tooBig, tooBig, 700, 700, 700), 700, 700, 700);
216     // big size + threshold
217     compactEquals(sfCreate(tooBig, tooBig, 700,700) /* empty */);
218     // small files = don't care about ratio
219     compactEquals(sfCreate(8,3,1), 8,3,1);
220     /* TODO: add sorting + unit test back in when HBASE-2856 is fixed 
221     // sort first so you don't include huge file the tail end
222     // happens with HFileOutputFormat bulk migration
223     compactEquals(sfCreate(100,50,23,12,12, 500), 23, 12, 12);
224      */
225     // don't exceed max file compact threshold
226     assertEquals(maxFiles,
227         store.compactSelection(sfCreate(7,6,5,4,3,2,1)).getFilesToCompact().size());
228     // note:  file selection starts with largest to smallest.
229     compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
230     
231     /* MAJOR COMPACTION */
232     // if a major compaction has been forced, then compact everything
233     compactEquals(sfCreate(50,25,12,12), true, 50, 25, 12, 12);
234     // also choose files < threshold on major compaction
235     compactEquals(sfCreate(12,12), true, 12, 12);
236     // even if one of those files is too big
237     compactEquals(sfCreate(tooBig, 12,12), true, tooBig, 12, 12);
238     // don't exceed max file compact threshold, even with major compaction
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     // if we exceed maxCompactSize, downgrade to minor
244     // if not, it creates a 'snowball effect' when files >> maxCompactSize:
245     // the last file in compaction is the aggregate of all previous compactions
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       // trigger an aged major compaction
251       compactEquals(sfCreate(50,25,12,12), 50, 25, 12, 12);
252       // major sure exceeding maxCompactSize also downgrades aged minors
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     /* REFERENCES == file is from a region that was split */
260     // treat storefiles that have references like a major compaction
261     compactEquals(sfCreate(true, 100,50,25,12,12), 100, 50, 25, 12, 12);
262     // reference files shouldn't obey max threshold
263     compactEquals(sfCreate(true, tooBig, 12,12), tooBig, 12, 12);
264     // reference files should obey max file compact to avoid OOM
265     assertEquals(maxFiles,
266         store.compactSelection(sfCreate(true, 7,6,5,4,3,2,1)).getFilesToCompact().size());
267     // reference compaction
268     compactEquals(sfCreate(true, 7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
269     
270     // empty case
271     compactEquals(new ArrayList<StoreFile>() /* empty */);
272     // empty case (because all files are too big)
273     compactEquals(sfCreate(tooBig, tooBig) /* empty */);
274   }
275 
276   public void testOffPeakCompactionRatio() throws IOException {
277     /*
278      * NOTE: these tests are specific to describe the implementation of the
279      * current compaction algorithm.  Developed to ensure that refactoring
280      * doesn't implicitly alter this.
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     // check compact selection without peak hour setting
292     LOG.debug("Testing compact selection without off-peak settings...");
293     compactEquals(sfCreate(999,50,12,12,1), 12, 12, 1);
294 
295     // set an off-peak compaction threshold
296     this.conf.setFloat("hbase.hstore.compaction.ratio.offpeak", 5.0F);
297 
298     // set peak hour to current time and check compact selection
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     // set peak hour outside current selection and check compact selection
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     // Do not compact empty store file
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     // Test Default compactions
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