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.*;
37  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
38  import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
39  import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection;
40  import org.apache.hadoop.hbase.regionserver.wal.HLog;
41  import org.apache.hadoop.hbase.util.Bytes;
42  
43  import com.google.common.collect.Lists;
44  import org.junit.experimental.categories.Category;
45  
46  @Category(SmallTests.class)
47  public class TestCompactSelection extends TestCase {
48    private final static Log LOG = LogFactory.getLog(TestCompactSelection.class);
49    private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
50  
51    private Configuration conf;
52    private Store store;
53    private static final String DIR=
54      TEST_UTIL.getDataTestDir("TestCompactSelection").toString();
55    private static Path TEST_FILE;
56  
57    private static final int minFiles = 3;
58    private static final int maxFiles = 5;
59  
60    private static final long minSize = 10;
61    private static final long maxSize = 1000;
62  
63  
64    @Override
65    public void setUp() throws Exception {
66      // setup config values necessary for store
67      this.conf = TEST_UTIL.getConfiguration();
68      this.conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0);
69      this.conf.setInt("hbase.hstore.compaction.min", minFiles);
70      this.conf.setInt("hbase.hstore.compaction.max", maxFiles);
71      this.conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, minSize);
72      this.conf.setLong("hbase.hstore.compaction.max.size", maxSize);
73      this.conf.setFloat("hbase.hstore.compaction.ratio", 1.0F);
74  
75      //Setting up a Store
76      Path basedir = new Path(DIR);
77      Path logdir = new Path(DIR+"/logs");
78      Path oldLogDir = new Path(basedir, HConstants.HREGION_OLDLOGDIR_NAME);
79      HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("family"));
80      FileSystem fs = FileSystem.get(conf);
81  
82      fs.delete(logdir, true);
83  
84      HTableDescriptor htd = new HTableDescriptor(Bytes.toBytes("table"));
85      htd.addFamily(hcd);
86      HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
87  
88      HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
89      HRegion region = HRegion.createHRegion(info, basedir, conf, htd);
90      HRegion.closeHRegion(region);
91      Path tableDir = new Path(basedir, Bytes.toString(htd.getName()));
92      region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
93  
94      store = new Store(basedir, region, hcd, fs, conf);
95      TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir());
96      fs.create(TEST_FILE);
97    }
98  
99    // used so our tests don't deal with actual StoreFiles
100   static class MockStoreFile extends StoreFile {
101     long length = 0;
102     boolean isRef = false;
103 
104     MockStoreFile(long length, boolean isRef) throws IOException {
105       super(TEST_UTIL.getTestFileSystem(), TEST_FILE,
106             TEST_UTIL.getConfiguration(),
107             new CacheConfig(TEST_UTIL.getConfiguration()), BloomType.NONE,
108             NoOpDataBlockEncoder.INSTANCE);
109       this.length = length;
110       this.isRef  = isRef;
111     }
112 
113     void setLength(long newLen) {
114       this.length = newLen;
115     }
116 
117     @Override
118     boolean isMajorCompaction() {
119       return false;
120     }
121 
122     @Override
123     boolean isReference() {
124       return this.isRef;
125     }
126 
127     @Override
128     public StoreFile.Reader getReader() {
129       final long len = this.length;
130       return new StoreFile.Reader() {
131         @Override
132         public long length() {
133           return len;
134         }
135       };
136     }
137   }
138 
139   List<StoreFile> sfCreate(long ... sizes) throws IOException {
140     return sfCreate(false, sizes);
141   }
142 
143   List<StoreFile> sfCreate(boolean isReference, long ... sizes)
144   throws IOException {
145     List<StoreFile> ret = Lists.newArrayList();
146     for (long i : sizes) {
147       ret.add(new MockStoreFile(i, isReference));
148     }
149     return ret;
150   }
151 
152   long[] getSizes(List<StoreFile> sfList) {
153     long[] aNums = new long[sfList.size()];
154     for (int i=0; i <sfList.size(); ++i) {
155       aNums[i] = sfList.get(i).getReader().length();
156     }
157     return aNums;
158   }
159   
160   void compactEquals(List<StoreFile> candidates, long ... expected) 
161   throws IOException {
162     compactEquals(candidates, false, expected);
163   }
164 
165   void compactEquals(List<StoreFile> candidates, boolean forcemajor, 
166       long ... expected)
167   throws IOException {
168     store.forceMajor = forcemajor;
169     List<StoreFile> actual = store.compactSelection(candidates).getFilesToCompact();
170     store.forceMajor = false;
171     assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
172   }
173 
174   public void testCompactionRatio() throws IOException {
175     /*
176      * NOTE: these tests are specific to describe the implementation of the
177      * current compaction algorithm.  Developed to ensure that refactoring
178      * doesn't implicitly alter this.
179      */
180     long tooBig = maxSize + 1;
181 
182     // default case. preserve user ratio on size
183     compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
184     // less than compact threshold = don't compact
185     compactEquals(sfCreate(100,50,25,12,12) /* empty */);
186     // greater than compact size = skip those
187     compactEquals(sfCreate(tooBig, tooBig, 700, 700, 700), 700, 700, 700);
188     // big size + threshold
189     compactEquals(sfCreate(tooBig, tooBig, 700,700) /* empty */);
190     // small files = don't care about ratio
191     compactEquals(sfCreate(8,3,1), 8,3,1);
192     /* TODO: add sorting + unit test back in when HBASE-2856 is fixed 
193     // sort first so you don't include huge file the tail end
194     // happens with HFileOutputFormat bulk migration
195     compactEquals(sfCreate(100,50,23,12,12, 500), 23, 12, 12);
196      */
197     // don't exceed max file compact threshold
198     assertEquals(maxFiles,
199         store.compactSelection(sfCreate(7,6,5,4,3,2,1)).getFilesToCompact().size());
200     // note:  file selection starts with largest to smallest.
201     compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
202     
203     /* MAJOR COMPACTION */
204     // if a major compaction has been forced, then compact everything
205     compactEquals(sfCreate(50,25,12,12), true, 50, 25, 12, 12);
206     // also choose files < threshold on major compaction
207     compactEquals(sfCreate(12,12), true, 12, 12);
208     // even if one of those files is too big
209     compactEquals(sfCreate(tooBig, 12,12), true, tooBig, 12, 12);
210     // don't exceed max file compact threshold, even with major compaction
211     store.forceMajor = true;
212     compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
213     store.forceMajor = false;
214 
215     // if we exceed maxCompactSize, downgrade to minor
216     // if not, it creates a 'snowball effect' when files >> maxCompactSize:
217     // the last file in compaction is the aggregate of all previous compactions
218     compactEquals(sfCreate(100,50,23,12,12), true, 23, 12, 12);
219     conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1);
220     conf.setFloat("hbase.hregion.majorcompaction.jitter", 0);
221     try {
222       // trigger an aged major compaction
223       compactEquals(sfCreate(50,25,12,12), 50, 25, 12, 12);
224       // major sure exceeding maxCompactSize also downgrades aged minors
225       compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
226     } finally {
227       conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
228       conf.setFloat("hbase.hregion.majorcompaction.jitter", 0.20F);
229     }
230 
231     /* REFERENCES == file is from a region that was split */
232     // treat storefiles that have references like a major compaction
233     compactEquals(sfCreate(true, 100,50,25,12,12), 100, 50, 25, 12, 12);
234     // reference files shouldn't obey max threshold
235     compactEquals(sfCreate(true, tooBig, 12,12), tooBig, 12, 12);
236     // reference files should obey max file compact to avoid OOM
237     assertEquals(maxFiles,
238         store.compactSelection(sfCreate(true, 7,6,5,4,3,2,1)).getFilesToCompact().size());
239     // reference compaction
240     compactEquals(sfCreate(true, 7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
241     
242     // empty case
243     compactEquals(new ArrayList<StoreFile>() /* empty */);
244     // empty case (because all files are too big)
245     compactEquals(sfCreate(tooBig, tooBig) /* empty */);
246   }
247 
248   public void testOffPeakCompactionRatio() throws IOException {
249     /*
250      * NOTE: these tests are specific to describe the implementation of the
251      * current compaction algorithm.  Developed to ensure that refactoring
252      * doesn't implicitly alter this.
253      */
254     long tooBig = maxSize + 1;
255 
256     Calendar calendar = new GregorianCalendar();
257     int hourOfDay = calendar.get(Calendar.HOUR_OF_DAY);
258     LOG.debug("Hour of day = " + hourOfDay);
259     int hourPlusOne = ((hourOfDay+1+24)%24);
260     int hourMinusOne = ((hourOfDay-1+24)%24);
261     int hourMinusTwo = ((hourOfDay-2+24)%24);
262 
263     // check compact selection without peak hour setting
264     LOG.debug("Testing compact selection without off-peak settings...");
265     compactEquals(sfCreate(999,50,12,12,1), 12, 12, 1);
266 
267     // set an off-peak compaction threshold
268     this.conf.setFloat("hbase.hstore.compaction.ratio.offpeak", 5.0F);
269 
270     // set peak hour to current time and check compact selection
271     this.conf.setLong("hbase.offpeak.start.hour", hourMinusOne);
272     this.conf.setLong("hbase.offpeak.end.hour", hourPlusOne);
273     LOG.debug("Testing compact selection with off-peak settings (" +
274         hourMinusOne + ", " + hourPlusOne + ")");
275     compactEquals(sfCreate(999,50,12,12, 1), 50, 12, 12, 1);
276 
277     // set peak hour outside current selection and check compact selection
278     this.conf.setLong("hbase.offpeak.start.hour", hourMinusTwo);
279     this.conf.setLong("hbase.offpeak.end.hour", hourMinusOne);
280     LOG.debug("Testing compact selection with off-peak settings (" +
281         hourMinusTwo + ", " + hourMinusOne + ")");
282     compactEquals(sfCreate(999,50,12,12, 1), 12, 12, 1);
283   }
284 
285   @org.junit.Rule
286   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
287     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
288 }
289