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