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
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.hbase.classification.InterfaceAudience;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.fs.Path;
30 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
31 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
32 import org.apache.hadoop.util.StringUtils;
33
34
35
36
37 @InterfaceAudience.Private
38 public class DefaultStoreFlusher extends StoreFlusher {
39 private static final Log LOG = LogFactory.getLog(DefaultStoreFlusher.class);
40 private final Object flushLock = new Object();
41
42 public DefaultStoreFlusher(Configuration conf, Store store) {
43 super(conf, store);
44 }
45
46 @Override
47 public List<Path> flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushId,
48 MonitoredTask status, ThroughputController throughputController) throws IOException {
49 ArrayList<Path> result = new ArrayList<Path>();
50 int cellsCount = snapshot.getCellsCount();
51 if (cellsCount == 0) return result;
52
53
54 long smallestReadPoint = store.getSmallestReadPoint();
55 InternalScanner scanner = createScanner(snapshot.getScanner(), smallestReadPoint);
56 if (scanner == null) {
57 return result;
58 }
59
60 StoreFile.Writer writer;
61 try {
62
63
64 synchronized (flushLock) {
65 status.setStatus("Flushing " + store + ": creating writer");
66
67 writer = store.createWriterInTmp(cellsCount, store.getFamily().getCompression(),
68
69
70
71
72 snapshot.getTimeRangeTracker(), -1);
73 IOException e = null;
74 try {
75 performFlush(scanner, writer, smallestReadPoint, throughputController);
76 } catch (IOException ioe) {
77 e = ioe;
78
79 throw ioe;
80 } finally {
81 if (e != null) {
82 writer.close();
83 } else {
84 finalizeWriter(writer, cacheFlushId, status);
85 }
86 }
87 }
88 } finally {
89 scanner.close();
90 }
91 LOG.info("Flushed, sequenceid=" + cacheFlushId +", memsize="
92 + StringUtils.humanReadableInt(snapshot.getSize()) +
93 ", hasBloomFilter=" + writer.hasGeneralBloom() +
94 ", into tmp file " + writer.getPath());
95 result.add(writer.getPath());
96 return result;
97 }
98 }