1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.List;
24 import java.util.SortedSet;
25 import java.util.concurrent.atomic.AtomicLong;
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.Path;
31 import org.apache.hadoop.hbase.KeyValue;
32 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
33 import org.apache.hadoop.hbase.util.CollectionBackedScanner;
34 import org.apache.hadoop.util.StringUtils;
35
36
37
38
39 public class DefaultStoreFlusher extends StoreFlusher {
40 private static final Log LOG = LogFactory.getLog(DefaultStoreFlusher.class);
41 private final Object flushLock = new Object();
42
43 public DefaultStoreFlusher(Configuration conf, Store store) {
44 super(conf, store);
45 }
46
47 @Override
48 public List<Path> flushSnapshot(SortedSet<KeyValue> snapshot, long cacheFlushId,
49 TimeRangeTracker snapshotTimeRangeTracker, AtomicLong flushedSize,
50 MonitoredTask status) throws IOException {
51 ArrayList<Path> result = new ArrayList<Path>();
52 if (snapshot.size() == 0) return result;
53
54
55 long smallestReadPoint = store.getSmallestReadPoint();
56 KeyValueScanner memstoreScanner =
57 new CollectionBackedScanner(snapshot, store.getComparator());
58 InternalScanner scanner = preCreateCoprocScanner(memstoreScanner);
59 if (scanner == null) {
60 scanner = createStoreScanner(smallestReadPoint, memstoreScanner);
61 }
62 scanner = postCreateCoprocScanner(scanner);
63 if (scanner == null) {
64 return result;
65 }
66
67 StoreFile.Writer writer;
68 long flushed = 0;
69 try {
70
71
72 synchronized (flushLock) {
73 status.setStatus("Flushing " + store + ": creating writer");
74
75 writer = store.createWriterInTmp(
76 snapshot.size(), store.getFamily().getCompression(), false, true);
77 writer.setTimeRangeTracker(snapshotTimeRangeTracker);
78 try {
79 flushed = performFlush(scanner, writer, smallestReadPoint);
80 } finally {
81 finalizeWriter(writer, cacheFlushId, status);
82 }
83 }
84 } finally {
85 flushedSize.set(flushed);
86 scanner.close();
87 }
88 LOG.info("Flushed, sequenceid=" + cacheFlushId +", memsize="
89 + StringUtils.humanReadableInt(flushed) +
90 ", hasBloomFilter=" + writer.hasGeneralBloom() +
91 ", into tmp file " + writer.getPath());
92 result.add(writer.getPath());
93 return result;
94 }
95 }