View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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.testclassification.SmallTests;
38  import org.apache.hadoop.hbase.TableName;
39  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
40  import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
41  import org.apache.hadoop.hbase.testclassification.SmallTests;
42  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
43  import org.apache.hadoop.hbase.util.TimeOffsetEnvironmentEdge;
44  import org.junit.Assert;
45  import org.junit.Test;
46  import org.apache.hadoop.hbase.wal.DefaultWALProvider;
47  import org.apache.hadoop.hbase.wal.WALFactory;
48  import org.apache.hadoop.hbase.util.Bytes;
49  import org.apache.hadoop.hbase.util.FSUtils;
50  import org.junit.After;
51  import org.junit.experimental.categories.Category;
52  
53  import com.google.common.collect.Lists;
54  
55  @Category(SmallTests.class)
56  public class TestDefaultCompactSelection extends TestCompactionPolicy {
57    private final static Log LOG = LogFactory.getLog(TestDefaultCompactSelection.class);
58  
59    @Test
60    public void testCompactionRatio() throws IOException {
61      TimeOffsetEnvironmentEdge edge = new TimeOffsetEnvironmentEdge();
62      EnvironmentEdgeManager.injectEdge(edge);
63      /**
64       * NOTE: these tests are specific to describe the implementation of the
65       * current compaction algorithm.  Developed to ensure that refactoring
66       * doesn't implicitly alter this.
67       */
68      long tooBig = maxSize + 1;
69  
70      // default case. preserve user ratio on size
71      compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
72      // less than compact threshold = don't compact
73      compactEquals(sfCreate(100,50,25,12,12) /* empty */);
74      // greater than compact size = skip those
75      compactEquals(sfCreate(tooBig, tooBig, 700, 700, 700), 700, 700, 700);
76      // big size + threshold
77      compactEquals(sfCreate(tooBig, tooBig, 700,700) /* empty */);
78      // small files = don't care about ratio
79      compactEquals(sfCreate(7,1,1), 7,1,1);
80  
81      // don't exceed max file compact threshold
82      // note:  file selection starts with largest to smallest.
83      compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
84  
85      compactEquals(sfCreate(50, 10, 10 ,10, 10), 10, 10, 10, 10);
86  
87      compactEquals(sfCreate(10, 10, 10, 10, 50), 10, 10, 10, 10);
88  
89      compactEquals(sfCreate(251, 253, 251, maxSize -1), 251, 253, 251);
90  
91      compactEquals(sfCreate(maxSize -1,maxSize -1,maxSize -1) /* empty */);
92  
93      // Always try and compact something to get below blocking storefile count
94      this.conf.setLong("hbase.hstore.compaction.min.size", 1);
95      store.storeEngine.getCompactionPolicy().setConf(conf);
96      compactEquals(sfCreate(512,256,128,64,32,16,8,4,2,1), 4,2,1);
97      this.conf.setLong("hbase.hstore.compaction.min.size", minSize);
98      store.storeEngine.getCompactionPolicy().setConf(conf);
99  
100     /* MAJOR COMPACTION */
101     // if a major compaction has been forced, then compact everything
102     compactEquals(sfCreate(50,25,12,12), true, 50, 25, 12, 12);
103     // also choose files < threshold on major compaction
104     compactEquals(sfCreate(12,12), true, 12, 12);
105     // even if one of those files is too big
106     compactEquals(sfCreate(tooBig, 12,12), true, tooBig, 12, 12);
107     // don't exceed max file compact threshold, even with major compaction
108     store.forceMajor = true;
109     compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
110     store.forceMajor = false;
111     // if we exceed maxCompactSize, downgrade to minor
112     // if not, it creates a 'snowball effect' when files >> maxCompactSize:
113     // the last file in compaction is the aggregate of all previous compactions
114     compactEquals(sfCreate(100,50,23,12,12), true, 23, 12, 12);
115     conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1);
116     conf.setFloat("hbase.hregion.majorcompaction.jitter", 0);
117     store.storeEngine.getCompactionPolicy().setConf(conf);
118     try {
119       // The modTime of the mocked store file is currentTimeMillis, so we need to increase the
120       // timestamp a bit to make sure that now - lowestModTime is greater than major compaction
121       // period(1ms).
122       // trigger an aged major compaction
123       List<StoreFile> candidates = sfCreate(50, 25, 12, 12);
124       edge.increment(2);
125       compactEquals(candidates, 50, 25, 12, 12);
126       // major sure exceeding maxCompactSize also downgrades aged minors
127       candidates = sfCreate(100, 50, 23, 12, 12);
128       edge.increment(2);
129       compactEquals(candidates, 23, 12, 12);
130     } finally {
131       conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
132       conf.setFloat("hbase.hregion.majorcompaction.jitter", 0.20F);
133     }
134 
135     /* REFERENCES == file is from a region that was split */
136     // treat storefiles that have references like a major compaction
137     compactEquals(sfCreate(true, 100,50,25,12,12), 100, 50, 25, 12, 12);
138     // reference files shouldn't obey max threshold
139     compactEquals(sfCreate(true, tooBig, 12,12), tooBig, 12, 12);
140     // reference files should obey max file compact to avoid OOM
141     compactEquals(sfCreate(true, 7, 6, 5, 4, 3, 2, 1), 7, 6, 5, 4, 3);
142 
143     // empty case
144     compactEquals(new ArrayList<StoreFile>() /* empty */);
145     // empty case (because all files are too big)
146     compactEquals(sfCreate(tooBig, tooBig) /* empty */);
147   }
148 
149   @Test
150   public void testOffPeakCompactionRatio() throws IOException {
151     /*
152      * NOTE: these tests are specific to describe the implementation of the
153      * current compaction algorithm.  Developed to ensure that refactoring
154      * doesn't implicitly alter this.
155      */
156     // set an off-peak compaction threshold
157     this.conf.setFloat("hbase.hstore.compaction.ratio.offpeak", 5.0F);
158     store.storeEngine.getCompactionPolicy().setConf(this.conf);
159     // Test with and without the flag.
160     compactEquals(sfCreate(999, 50, 12, 12, 1), false, true, 50, 12, 12, 1);
161     compactEquals(sfCreate(999, 50, 12, 12, 1), 12, 12, 1);
162   }
163 
164   @Test
165   public void testStuckStoreCompaction() throws IOException {
166     // Select the smallest compaction if the store is stuck.
167     compactEquals(sfCreate(99,99,99,99,99,99, 30,30,30,30), 30, 30, 30);
168     // If not stuck, standard policy applies.
169     compactEquals(sfCreate(99,99,99,99,99, 30,30,30,30), 99, 30, 30, 30, 30);
170 
171     // Add sufficiently small files to compaction, though
172     compactEquals(sfCreate(99,99,99,99,99,99, 30,30,30,15), 30, 30, 30, 15);
173     // Prefer earlier compaction to latter if the benefit is not significant
174     compactEquals(sfCreate(99,99,99,99, 30,26,26,29,25,25), 30, 26, 26);
175     // Prefer later compaction if the benefit is significant.
176     compactEquals(sfCreate(99,99,99,99, 27,27,27,20,20,20), 20, 20, 20);
177   }
178 
179   @Test
180   public void testCompactionEmptyHFile() throws IOException {
181     // Set TTL
182     ScanInfo oldScanInfo = store.getScanInfo();
183     ScanInfo newScanInfo = new ScanInfo(oldScanInfo.getFamily(),
184         oldScanInfo.getMinVersions(), oldScanInfo.getMaxVersions(), 600,
185         oldScanInfo.getKeepDeletedCells(), oldScanInfo.getTimeToPurgeDeletes(),
186         oldScanInfo.getComparator());
187     store.setScanInfo(newScanInfo);
188     // Do not compact empty store file
189     List<StoreFile> candidates = sfCreate(0);
190     for (StoreFile file : candidates) {
191       if (file instanceof MockStoreFile) {
192         MockStoreFile mockFile = (MockStoreFile) file;
193         mockFile.setTimeRangeTracker(new TimeRangeTracker(-1, -1));
194         mockFile.setEntries(0);
195       }
196     }
197     // Test Default compactions
198     CompactionRequest result = ((RatioBasedCompactionPolicy) store.storeEngine
199         .getCompactionPolicy()).selectCompaction(candidates,
200         new ArrayList<StoreFile>(), false, false, false);
201     Assert.assertTrue(result.getFiles().size() == 0);
202     store.setScanInfo(oldScanInfo);
203   }
204 }