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.List;
26
27 import junit.framework.TestCase;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.fs.FileSystem;
33 import org.apache.hadoop.fs.Path;
34 import org.apache.hadoop.hbase.HBaseTestingUtility;
35 import org.apache.hadoop.hbase.HColumnDescriptor;
36 import org.apache.hadoop.hbase.HConstants;
37 import org.apache.hadoop.hbase.HRegionInfo;
38 import org.apache.hadoop.hbase.HTableDescriptor;
39 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
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
45 public class TestCompactSelection extends TestCase {
46 private final static Log LOG = LogFactory.getLog(TestCompactSelection.class);
47 private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
48
49 private Configuration conf;
50 private Store store;
51 private static final String DIR=
52 TEST_UTIL.getDataTestDir("TestCompactSelection").toString();
53 private static Path TEST_FILE;
54
55 private static final int minFiles = 3;
56 private static final int maxFiles = 5;
57
58 private static final long minSize = 10;
59 private static final long maxSize = 1000;
60
61
62 @Override
63 public void setUp() throws Exception {
64
65 this.conf = TEST_UTIL.getConfiguration();
66 this.conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0);
67 this.conf.setInt("hbase.hstore.compaction.min", minFiles);
68 this.conf.setInt("hbase.hstore.compaction.max", maxFiles);
69 this.conf.setLong("hbase.hregion.memstore.flush.size", minSize);
70 this.conf.setLong("hbase.hstore.compaction.max.size", maxSize);
71 this.conf.setFloat("hbase.hstore.compaction.ratio", 1.0F);
72
73
74 Path basedir = new Path(DIR);
75 Path logdir = new Path(DIR+"/logs");
76 Path oldLogDir = new Path(basedir, HConstants.HREGION_OLDLOGDIR_NAME);
77 HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("family"));
78 FileSystem fs = FileSystem.get(conf);
79
80 fs.delete(logdir, true);
81
82 HTableDescriptor htd = new HTableDescriptor(Bytes.toBytes("table"));
83 htd.addFamily(hcd);
84 HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
85
86 HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
87 HRegion.createHRegion(info, basedir, conf, htd);
88 Path tableDir = new Path(basedir, Bytes.toString(htd.getName()));
89 HRegion region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
90
91 store = new Store(basedir, region, hcd, fs, conf);
92 TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir());
93 fs.create(TEST_FILE);
94 }
95
96
97 static class MockStoreFile extends StoreFile {
98 long length = 0;
99 boolean isRef = false;
100
101 MockStoreFile(long length, boolean isRef) throws IOException {
102 super(TEST_UTIL.getTestFileSystem(), TEST_FILE,
103 TEST_UTIL.getConfiguration(),
104 new CacheConfig(TEST_UTIL.getConfiguration()), BloomType.NONE);
105 this.length = length;
106 this.isRef = isRef;
107 }
108
109 void setLength(long newLen) {
110 this.length = newLen;
111 }
112
113 @Override
114 boolean isMajorCompaction() {
115 return false;
116 }
117
118 @Override
119 boolean isReference() {
120 return this.isRef;
121 }
122
123 @Override
124 public StoreFile.Reader getReader() {
125 final long len = this.length;
126 return new StoreFile.Reader() {
127 @Override
128 public long length() {
129 return len;
130 }
131 };
132 }
133 }
134
135 List<StoreFile> sfCreate(long ... sizes) throws IOException {
136 return sfCreate(false, sizes);
137 }
138
139 List<StoreFile> sfCreate(boolean isReference, long ... sizes)
140 throws IOException {
141 List<StoreFile> ret = Lists.newArrayList();
142 for (long i : sizes) {
143 ret.add(new MockStoreFile(i, isReference));
144 }
145 return ret;
146 }
147
148 long[] getSizes(List<StoreFile> sfList) {
149 long[] aNums = new long[sfList.size()];
150 for (int i=0; i <sfList.size(); ++i) {
151 aNums[i] = sfList.get(i).getReader().length();
152 }
153 return aNums;
154 }
155
156 void compactEquals(List<StoreFile> candidates, long ... expected)
157 throws IOException {
158 compactEquals(candidates, false, expected);
159 }
160
161 void compactEquals(List<StoreFile> candidates, boolean forcemajor,
162 long ... expected)
163 throws IOException {
164 store.forceMajor = forcemajor;
165 List<StoreFile> actual = store.compactSelection(candidates);
166 store.forceMajor = false;
167 assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
168 }
169
170 public void testCompactionRatio() throws IOException {
171
172
173
174
175
176 long tooBig = maxSize + 1;
177
178
179 compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
180
181 compactEquals(sfCreate(100,50,25,12,12)
182
183 compactEquals(sfCreate(tooBig, tooBig, 700, 700, 700), 700, 700, 700);
184
185 compactEquals(sfCreate(tooBig, tooBig, 700,700)
186
187 compactEquals(sfCreate(8,3,1), 8,3,1);
188
189
190
191
192
193
194 assertEquals(maxFiles,
195 store.compactSelection(sfCreate(7,6,5,4,3,2,1)).size());
196
197
198
199 compactEquals(sfCreate(50,25,12,12), true, 50, 25, 12, 12);
200
201 compactEquals(sfCreate(12,12), true, 12, 12);
202
203 compactEquals(sfCreate(tooBig, 12,12), true, tooBig, 12, 12);
204
205 store.forceMajor = true;
206 assertEquals(maxFiles,
207 store.compactSelection(sfCreate(7,6,5,4,3,2,1)).size());
208 store.forceMajor = false;
209
210
211
212
213 compactEquals(sfCreate(100,50,23,12,12), true, 23, 12, 12);
214
215 store.majorCompactionTime = 1;
216 compactEquals(sfCreate(50,25,12,12), 50, 25, 12, 12);
217
218 store.majorCompactionTime = 1;
219 compactEquals(sfCreate(100,50,23,12,12), 23, 12, 12);
220
221
222
223 compactEquals(sfCreate(true, 100,50,25,12,12), 100, 50, 25, 12, 12);
224
225 compactEquals(sfCreate(true, tooBig, 12,12), tooBig, 12, 12);
226
227 assertEquals(maxFiles,
228 store.compactSelection(sfCreate(true, 7,6,5,4,3,2,1)).size());
229
230
231 compactEquals(new ArrayList<StoreFile>()
232
233 compactEquals(sfCreate(tooBig, tooBig)
234 }
235 }