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 org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.hbase.KeyValue;
26 import org.apache.hadoop.hbase.client.Scan;
27 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
28 import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
29
30 import java.io.IOException;
31 import java.util.ArrayList;
32 import java.util.Collection;
33 import java.util.List;
34 import java.util.SortedSet;
35
36
37
38
39
40 class StoreFileScanner implements KeyValueScanner {
41 static final Log LOG = LogFactory.getLog(Store.class);
42
43
44 private final StoreFile.Reader reader;
45 private final HFileScanner hfs;
46 private KeyValue cur = null;
47 private boolean enforceMVCC = false;
48
49
50
51
52
53 public StoreFileScanner(StoreFile.Reader reader, HFileScanner hfs, boolean useMVCC) {
54 this.reader = reader;
55 this.hfs = hfs;
56 this.enforceMVCC = useMVCC;
57 }
58
59
60
61
62
63 public static List<StoreFileScanner> getScannersForStoreFiles(
64 Collection<StoreFile> filesToCompact,
65 boolean cacheBlocks,
66 boolean usePread) throws IOException {
67 return getScannersForStoreFiles(filesToCompact, cacheBlocks, usePread, false);
68 }
69
70
71
72
73 public static List<StoreFileScanner> getScannersForStoreFiles(
74 Collection<StoreFile> files, boolean cacheBlocks, boolean usePread,
75 boolean isCompaction) throws IOException {
76 List<StoreFileScanner> scanners = new ArrayList<StoreFileScanner>(
77 files.size());
78 for (StoreFile file : files) {
79 StoreFile.Reader r = file.createReader();
80 scanners.add(r.getStoreFileScanner(cacheBlocks, usePread, isCompaction));
81 }
82 return scanners;
83 }
84
85 public String toString() {
86 return "StoreFileScanner[" + hfs.toString() + ", cur=" + cur + "]";
87 }
88
89 public KeyValue peek() {
90 return cur;
91 }
92
93 public KeyValue next() throws IOException {
94 KeyValue retKey = cur;
95
96 try {
97
98 if (cur != null) {
99 hfs.next();
100 cur = hfs.getKeyValue();
101 skipKVsNewerThanReadpoint();
102 }
103 } catch(IOException e) {
104 throw new IOException("Could not iterate " + this, e);
105 }
106 return retKey;
107 }
108
109 public boolean seek(KeyValue key) throws IOException {
110 try {
111 if(!seekAtOrAfter(hfs, key)) {
112 close();
113 return false;
114 }
115 cur = hfs.getKeyValue();
116 return skipKVsNewerThanReadpoint();
117 } catch(IOException ioe) {
118 throw new IOException("Could not seek " + this, ioe);
119 }
120 }
121
122 public boolean reseek(KeyValue key) throws IOException {
123 try {
124 if (!reseekAtOrAfter(hfs, key)) {
125 close();
126 return false;
127 }
128 cur = hfs.getKeyValue();
129 return skipKVsNewerThanReadpoint();
130 } catch (IOException ioe) {
131 throw new IOException("Could not seek " + this, ioe);
132 }
133 }
134
135 protected boolean skipKVsNewerThanReadpoint() throws IOException {
136 long readPoint = MultiVersionConsistencyControl.getThreadReadPoint();
137
138
139
140 while(enforceMVCC
141 && cur != null
142 && (cur.getMemstoreTS() > readPoint)) {
143 hfs.next();
144 cur = hfs.getKeyValue();
145 }
146
147 if (cur == null) {
148 close();
149 return false;
150 }
151
152
153
154
155
156
157
158 if (cur.getMemstoreTS() <= readPoint) {
159 cur.setMemstoreTS(0);
160 }
161 return true;
162 }
163
164 public void close() {
165
166 cur = null;
167 }
168
169
170
171
172
173
174
175
176 public static boolean seekAtOrAfter(HFileScanner s, KeyValue k)
177 throws IOException {
178 int result = s.seekTo(k.getBuffer(), k.getKeyOffset(), k.getKeyLength());
179 if(result < 0) {
180
181 return s.seekTo();
182 } else if(result > 0) {
183
184
185 return s.next();
186 }
187
188 return true;
189 }
190
191 static boolean reseekAtOrAfter(HFileScanner s, KeyValue k)
192 throws IOException {
193
194 int result = s.reseekTo(k.getBuffer(), k.getKeyOffset(), k.getKeyLength());
195 if (result <= 0) {
196 return true;
197 } else {
198
199
200 return s.next();
201 }
202 }
203
204
205 public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
206 return reader.shouldSeek(scan, columns);
207 }
208
209 @Override
210 public long getSequenceID() {
211 return reader.getSequenceID();
212 }
213
214 @Override
215 public boolean seekExactly(KeyValue kv, boolean forward)
216 throws IOException {
217 if (reader.getBloomFilterType() != StoreFile.BloomType.ROWCOL ||
218 kv.getRowLength() == 0 || kv.getQualifierLength() == 0) {
219 return forward ? reseek(kv) : seek(kv);
220 }
221
222 boolean isInBloom = reader.passesBloomFilter(kv.getBuffer(),
223 kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
224 kv.getQualifierOffset(), kv.getQualifierLength());
225 if (isInBloom) {
226
227 return forward ? reseek(kv) : seek(kv);
228 }
229
230
231
232
233
234 cur = kv.createLastOnRowCol();
235 return true;
236 }
237
238 Reader getReaderForTesting() {
239 return reader;
240 }
241 }