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.Collections;
24 import java.util.List;
25 import java.util.SortedSet;
26 import java.util.concurrent.atomic.AtomicLong;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.classification.InterfaceAudience;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.fs.Path;
33 import org.apache.hadoop.hbase.HConstants;
34 import org.apache.hadoop.hbase.KeyValue;
35 import org.apache.hadoop.hbase.client.Scan;
36 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
37 import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
38
39
40
41
42
43 @InterfaceAudience.Private
44 abstract class StoreFlusher {
45 protected Configuration conf;
46 protected Store store;
47
48 public StoreFlusher(Configuration conf, Store store) {
49 this.conf = conf;
50 this.store = store;
51 }
52
53
54
55
56
57
58
59
60
61
62
63 public abstract List<Path> flushSnapshot(SortedSet<KeyValue> snapshot, long cacheFlushSeqNum,
64 TimeRangeTracker snapshotTimeRangeTracker, AtomicLong flushedSize, MonitoredTask status)
65 throws IOException;
66
67 protected void finalizeWriter(StoreFile.Writer writer, long cacheFlushSeqNum,
68 MonitoredTask status) throws IOException {
69
70
71
72 status.setStatus("Flushing " + store + ": appending metadata");
73 writer.appendMetadata(cacheFlushSeqNum, false);
74 status.setStatus("Flushing " + store + ": closing flushed file");
75 writer.close();
76 }
77
78
79 protected InternalScanner preCreateCoprocScanner(
80 KeyValueScanner memstoreScanner) throws IOException {
81 if (store.getCoprocessorHost() != null) {
82 return store.getCoprocessorHost().preFlushScannerOpen(store, memstoreScanner);
83 }
84 return null;
85 }
86
87
88 protected InternalScanner createStoreScanner(long smallestReadPoint,
89 KeyValueScanner memstoreScanner) throws IOException {
90 Scan scan = new Scan();
91 scan.setMaxVersions(store.getScanInfo().getMaxVersions());
92 return new StoreScanner(store, store.getScanInfo(), scan,
93 Collections.singletonList(memstoreScanner), ScanType.COMPACT_RETAIN_DELETES,
94 smallestReadPoint, HConstants.OLDEST_TIMESTAMP);
95 }
96
97
98
99
100
101 protected InternalScanner postCreateCoprocScanner(InternalScanner scanner)
102 throws IOException {
103 if (store.getCoprocessorHost() != null) {
104 return store.getCoprocessorHost().preFlush(store, scanner);
105 }
106 return scanner;
107 }
108
109
110
111
112
113
114
115
116 protected long performFlush(InternalScanner scanner,
117 Compactor.CellSink sink, long smallestReadPoint) throws IOException {
118 int compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX, 10);
119 List<KeyValue> kvs = new ArrayList<KeyValue>();
120 boolean hasMore;
121 long flushed = 0;
122 do {
123 hasMore = scanner.next(kvs, compactionKVMax);
124 if (!kvs.isEmpty()) {
125 for (KeyValue kv : kvs) {
126
127
128
129 if (kv.getMvccVersion() <= smallestReadPoint) {
130
131
132 kv = kv.shallowCopy();
133 kv.setMvccVersion(0);
134 }
135 sink.append(kv);
136 flushed += MemStore.heapSizeChange(kv, true);
137 }
138 kvs.clear();
139 }
140 } while (hasMore);
141 return flushed;
142 }
143 }