1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.regionserver;
22
23 import java.io.IOException;
24 import java.lang.ref.SoftReference;
25 import java.util.ArrayList;
26 import java.util.Collections;
27 import java.util.Iterator;
28 import java.util.List;
29 import java.util.NavigableSet;
30 import java.util.concurrent.ConcurrentSkipListSet;
31
32 import junit.framework.TestCase;
33
34 import org.apache.commons.logging.Log;
35 import org.apache.commons.logging.LogFactory;
36 import org.apache.hadoop.conf.Configuration;
37 import org.apache.hadoop.fs.FSDataOutputStream;
38 import org.apache.hadoop.fs.FileStatus;
39 import org.apache.hadoop.fs.FileSystem;
40 import org.apache.hadoop.fs.FileUtil;
41 import org.apache.hadoop.fs.FilterFileSystem;
42 import org.apache.hadoop.fs.LocalFileSystem;
43 import org.apache.hadoop.fs.Path;
44 import org.apache.hadoop.hbase.HBaseConfiguration;
45 import org.apache.hadoop.hbase.HBaseTestingUtility;
46 import org.apache.hadoop.hbase.HColumnDescriptor;
47 import org.apache.hadoop.hbase.HConstants;
48 import org.apache.hadoop.hbase.HRegionInfo;
49 import org.apache.hadoop.hbase.HTableDescriptor;
50 import org.apache.hadoop.hbase.KeyValue;
51 import org.apache.hadoop.hbase.client.Get;
52 import org.apache.hadoop.hbase.regionserver.wal.HLog;
53 import org.apache.hadoop.hbase.util.Bytes;
54 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
55 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
56 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
57 import org.apache.hadoop.security.UnixUserGroupInformation;
58
59 import com.google.common.base.Joiner;
60
61
62
63
64 public class TestStore extends TestCase {
65 public static final Log LOG = LogFactory.getLog(TestStore.class);
66
67 Store store;
68 byte [] table = Bytes.toBytes("table");
69 byte [] family = Bytes.toBytes("family");
70
71 byte [] row = Bytes.toBytes("row");
72 byte [] qf1 = Bytes.toBytes("qf1");
73 byte [] qf2 = Bytes.toBytes("qf2");
74 byte [] qf3 = Bytes.toBytes("qf3");
75 byte [] qf4 = Bytes.toBytes("qf4");
76 byte [] qf5 = Bytes.toBytes("qf5");
77 byte [] qf6 = Bytes.toBytes("qf6");
78
79 NavigableSet<byte[]> qualifiers =
80 new ConcurrentSkipListSet<byte[]>(Bytes.BYTES_COMPARATOR);
81
82 List<KeyValue> expected = new ArrayList<KeyValue>();
83 List<KeyValue> result = new ArrayList<KeyValue>();
84
85 long id = System.currentTimeMillis();
86 Get get = new Get(row);
87
88 private static final String DIR = HBaseTestingUtility.getTestDir() + "/TestStore/";
89
90
91
92
93
94 @Override
95 public void setUp() throws IOException {
96 qualifiers.add(qf1);
97 qualifiers.add(qf3);
98 qualifiers.add(qf5);
99
100 Iterator<byte[]> iter = qualifiers.iterator();
101 while(iter.hasNext()){
102 byte [] next = iter.next();
103 expected.add(new KeyValue(row, family, next, 1, (byte[])null));
104 get.addColumn(family, next);
105 }
106 }
107
108 private void init(String methodName) throws IOException {
109 init(methodName, HBaseConfiguration.create());
110 }
111
112 private void init(String methodName, Configuration conf)
113 throws IOException {
114
115 Path basedir = new Path(DIR+methodName);
116 Path logdir = new Path(DIR+methodName+"/logs");
117 Path oldLogDir = new Path(basedir, HConstants.HREGION_OLDLOGDIR_NAME);
118 HColumnDescriptor hcd = new HColumnDescriptor(family);
119 FileSystem fs = FileSystem.get(conf);
120
121 fs.delete(logdir, true);
122
123 HTableDescriptor htd = new HTableDescriptor(table);
124 htd.addFamily(hcd);
125 HRegionInfo info = new HRegionInfo(htd, null, null, false);
126 HLog hlog = new HLog(fs, logdir, oldLogDir, conf, null);
127 HRegion region = new HRegion(basedir, hlog, fs, conf, info, null);
128
129 store = new Store(basedir, region, hcd, fs, conf);
130 }
131
132
133
134
135
136
137
138
139
140
141 public void testEmptyStoreFile() throws IOException {
142 init(this.getName());
143
144 this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
145 this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
146 flush(1);
147
148
149 StoreFile f = this.store.getStorefiles().get(0);
150 Path storedir = f.getPath().getParent();
151 long seqid = f.getMaxSequenceId();
152 Configuration c = HBaseConfiguration.create();
153 FileSystem fs = FileSystem.get(c);
154 StoreFile.Writer w = StoreFile.createWriter(fs, storedir,
155 StoreFile.DEFAULT_BLOCKSIZE_SMALL);
156 w.appendMetadata(seqid + 1, false);
157 w.close();
158 this.store.close();
159
160 this.store = new Store(storedir.getParent().getParent(),
161 this.store.getHRegion(),
162 this.store.getFamily(), fs, c);
163 System.out.println(this.store.getHRegionInfo().getEncodedName());
164 assertEquals(2, this.store.getStorefilesCount());
165
166 result = HBaseTestingUtility.getFromStoreFile(store,
167 get.getRow(),
168 qualifiers);
169 assertEquals(1, result.size());
170 }
171
172
173
174
175
176 public void testGet_FromMemStoreOnly() throws IOException {
177 init(this.getName());
178
179
180 this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
181 this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
182 this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
183 this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null));
184 this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null));
185 this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null));
186
187
188 result = HBaseTestingUtility.getFromStoreFile(store,
189 get.getRow(), qualifiers);
190
191
192 assertCheck();
193 }
194
195
196
197
198
199 public void testGet_FromFilesOnly() throws IOException {
200 init(this.getName());
201
202
203 this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
204 this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
205
206 flush(1);
207
208
209 this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
210 this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null));
211
212 flush(2);
213
214
215 this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null));
216 this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null));
217
218 flush(3);
219
220
221 result = HBaseTestingUtility.getFromStoreFile(store,
222 get.getRow(),
223 qualifiers);
224
225
226
227 Collections.sort(result, KeyValue.COMPARATOR);
228
229
230 assertCheck();
231 }
232
233
234
235
236
237 public void testGet_FromMemStoreAndFiles() throws IOException {
238 init(this.getName());
239
240
241 this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
242 this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
243
244 flush(1);
245
246
247 this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
248 this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null));
249
250 flush(2);
251
252
253 this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null));
254 this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null));
255
256
257 result = HBaseTestingUtility.getFromStoreFile(store,
258 get.getRow(), qualifiers);
259
260
261 Collections.sort(result, KeyValue.COMPARATOR);
262
263
264 assertCheck();
265 }
266
267 private void flush(int storeFilessize) throws IOException{
268 this.store.snapshot();
269 flushStore(store, id++);
270 assertEquals(storeFilessize, this.store.getStorefiles().size());
271 assertEquals(0, this.store.memstore.kvset.size());
272 }
273
274 private void assertCheck() {
275 assertEquals(expected.size(), result.size());
276 for(int i=0; i<expected.size(); i++) {
277 assertEquals(expected.get(i), result.get(i));
278 }
279 }
280
281
282
283
284
285
286
287 public void testIncrementColumnValue_ICVDuringFlush()
288 throws IOException, InterruptedException {
289 init(this.getName());
290
291 long oldValue = 1L;
292 long newValue = 3L;
293 this.store.add(new KeyValue(row, family, qf1,
294 System.currentTimeMillis(),
295 Bytes.toBytes(oldValue)));
296
297
298 this.store.snapshot();
299
300
301 this.store.add(new KeyValue(row, family, qf2,
302 System.currentTimeMillis(),
303 Bytes.toBytes(oldValue)));
304
305
306 long ret = this.store.updateColumnValue(row, family, qf1, newValue);
307
308
309 assertTrue(ret > 0);
310
311
312 flushStore(store, id++);
313 assertEquals(1, this.store.getStorefiles().size());
314
315 assertEquals(2, this.store.memstore.kvset.size());
316
317
318 Get get = new Get(row);
319 get.addColumn(family, qf1);
320 get.setMaxVersions();
321 List<KeyValue> results = new ArrayList<KeyValue>();
322
323 results = HBaseTestingUtility.getFromStoreFile(store, get);
324 assertEquals(2, results.size());
325
326 long ts1 = results.get(0).getTimestamp();
327 long ts2 = results.get(1).getTimestamp();
328
329 assertTrue(ts1 > ts2);
330
331 assertEquals(newValue, Bytes.toLong(results.get(0).getValue()));
332 assertEquals(oldValue, Bytes.toLong(results.get(1).getValue()));
333 }
334
335 public void testIncrementColumnValue_SnapshotFlushCombo() throws Exception {
336 ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
337 EnvironmentEdgeManagerTestHelper.injectEdge(mee);
338 init(this.getName());
339
340 long oldValue = 1L;
341 long newValue = 3L;
342 this.store.add(new KeyValue(row, family, qf1,
343 EnvironmentEdgeManager.currentTimeMillis(),
344 Bytes.toBytes(oldValue)));
345
346
347 this.store.snapshot();
348
349
350 long ret = this.store.updateColumnValue(row, family, qf1, newValue);
351
352
353 assertTrue(ret > 0);
354
355
356 flushStore(store, id++);
357 assertEquals(1, this.store.getStorefiles().size());
358 assertEquals(1, this.store.memstore.kvset.size());
359
360
361 newValue += 1;
362 this.store.updateColumnValue(row, family, qf1, newValue);
363
364
365 newValue += 1;
366 this.store.updateColumnValue(row, family, qf1, newValue);
367
368
369
370
371
372 Get get = new Get(row);
373 get.addColumn(family, qf1);
374 get.setMaxVersions();
375 List<KeyValue> results = new ArrayList<KeyValue>();
376
377 results = HBaseTestingUtility.getFromStoreFile(store, get);
378 assertEquals(2, results.size());
379
380 long ts1 = results.get(0).getTimestamp();
381 long ts2 = results.get(1).getTimestamp();
382
383 assertTrue(ts1 > ts2);
384 assertEquals(newValue, Bytes.toLong(results.get(0).getValue()));
385 assertEquals(oldValue, Bytes.toLong(results.get(1).getValue()));
386
387 mee.setValue(2);
388 newValue += 1;
389 this.store.updateColumnValue(row, family, qf1, newValue);
390
391 results = HBaseTestingUtility.getFromStoreFile(store, get);
392 assertEquals(2, results.size());
393
394 ts1 = results.get(0).getTimestamp();
395 ts2 = results.get(1).getTimestamp();
396
397 assertTrue(ts1 > ts2);
398 assertEquals(newValue, Bytes.toLong(results.get(0).getValue()));
399 assertEquals(oldValue, Bytes.toLong(results.get(1).getValue()));
400 }
401
402 public void testHandleErrorsInFlush() throws Exception {
403 LOG.info("Setting up a faulty file system that cannot write");
404
405 Configuration conf = HBaseConfiguration.create();
406
407 conf.set(UnixUserGroupInformation.UGI_PROPERTY_NAME,
408 "testhandleerrorsinflush,foo");
409
410 conf.setClass("fs.file.impl", FaultyFileSystem.class,
411 FileSystem.class);
412
413 FileSystem fs = FileSystem.get(conf);
414 assertEquals(FaultyFileSystem.class, fs.getClass());
415
416
417 init(getName(), conf);
418
419 LOG.info("Adding some data");
420 this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
421 this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
422 this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
423
424 LOG.info("Before flush, we should have no files");
425 FileStatus[] files = fs.listStatus(store.getHomedir());
426 Path[] paths = FileUtil.stat2Paths(files);
427 System.err.println("Got paths: " + Joiner.on(",").join(paths));
428 assertEquals(0, paths.length);
429
430
431 try {
432 LOG.info("Flushing");
433 flush(1);
434 fail("Didn't bubble up IOE!");
435 } catch (IOException ioe) {
436 assertTrue(ioe.getMessage().contains("Fault injected"));
437 }
438
439 LOG.info("After failed flush, we should still have no files!");
440 files = fs.listStatus(store.getHomedir());
441 paths = FileUtil.stat2Paths(files);
442 System.err.println("Got paths: " + Joiner.on(",").join(paths));
443 assertEquals(0, paths.length);
444 }
445
446
447 static class FaultyFileSystem extends FilterFileSystem {
448 List<SoftReference<FaultyOutputStream>> outStreams =
449 new ArrayList<SoftReference<FaultyOutputStream>>();
450 private long faultPos = 200;
451
452 public FaultyFileSystem() {
453 super(new LocalFileSystem());
454 System.err.println("Creating faulty!");
455 }
456
457 @Override
458 public FSDataOutputStream create(Path p) throws IOException {
459 return new FaultyOutputStream(super.create(p), faultPos);
460 }
461
462 }
463
464 static class FaultyOutputStream extends FSDataOutputStream {
465 volatile long faultPos = Long.MAX_VALUE;
466
467 public FaultyOutputStream(FSDataOutputStream out,
468 long faultPos) throws IOException {
469 super(out, null);
470 this.faultPos = faultPos;
471 }
472
473 @Override
474 public void write(byte[] buf, int offset, int length) throws IOException {
475 System.err.println("faulty stream write at pos " + getPos());
476 injectFault();
477 super.write(buf, offset, length);
478 }
479
480 private void injectFault() throws IOException {
481 if (getPos() >= faultPos) {
482 throw new IOException("Fault injected");
483 }
484 }
485 }
486
487
488
489 private static void flushStore(Store store, long id) throws IOException {
490 StoreFlusher storeFlusher = store.getStoreFlusher(id);
491 storeFlusher.prepare();
492 storeFlusher.flushCache();
493 storeFlusher.commit();
494 }
495
496
497
498
499
500
501
502
503
504
505
506 List<KeyValue> getKeyValueSet(long[] timestamps, int numRows,
507 byte[] qualifier, byte[] family) {
508 List<KeyValue> kvList = new ArrayList<KeyValue>();
509 for (int i=1;i<=numRows;i++) {
510 byte[] b = Bytes.toBytes(i);
511 for (long timestamp: timestamps) {
512 kvList.add(new KeyValue(b, family, qualifier, timestamp, b));
513 }
514 }
515 return kvList;
516 }
517
518
519
520
521
522 public void testMultipleTimestamps() throws IOException {
523 int numRows = 1;
524 long[] timestamps1 = new long[] {1,5,10,20};
525 long[] timestamps2 = new long[] {30,80};
526
527 init(this.getName());
528
529 List<KeyValue> kvList1 = getKeyValueSet(timestamps1,numRows, qf1, family);
530 for (KeyValue kv : kvList1) {
531 this.store.add(kv);
532 }
533
534 this.store.snapshot();
535 flushStore(store, id++);
536
537 List<KeyValue> kvList2 = getKeyValueSet(timestamps2,numRows, qf1, family);
538 for(KeyValue kv : kvList2) {
539 this.store.add(kv);
540 }
541
542 List<KeyValue> result;
543 Get get = new Get(Bytes.toBytes(1));
544 get.addColumn(family,qf1);
545
546 get.setTimeRange(0,15);
547 result = HBaseTestingUtility.getFromStoreFile(store, get);
548 assertTrue(result.size()>0);
549
550 get.setTimeRange(40,90);
551 result = HBaseTestingUtility.getFromStoreFile(store, get);
552 assertTrue(result.size()>0);
553
554 get.setTimeRange(10,45);
555 result = HBaseTestingUtility.getFromStoreFile(store, get);
556 assertTrue(result.size()>0);
557
558 get.setTimeRange(80,145);
559 result = HBaseTestingUtility.getFromStoreFile(store, get);
560 assertTrue(result.size()>0);
561
562 get.setTimeRange(1,2);
563 result = HBaseTestingUtility.getFromStoreFile(store, get);
564 assertTrue(result.size()>0);
565
566 get.setTimeRange(90,200);
567 result = HBaseTestingUtility.getFromStoreFile(store, get);
568 assertTrue(result.size()==0);
569 }
570 }