1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.io;
20
21 import java.io.IOException;
22 import java.nio.ByteBuffer;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.fs.FileSystem;
29 import org.apache.hadoop.fs.Path;
30 import org.apache.hadoop.hbase.Cell;
31 import org.apache.hadoop.hbase.HConstants;
32 import org.apache.hadoop.hbase.KeyValue;
33 import org.apache.hadoop.hbase.client.Scan;
34 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
35 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
36 import org.apache.hadoop.hbase.regionserver.StoreFile;
37 import org.apache.hadoop.hbase.util.Bytes;
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52 @InterfaceAudience.Private
53 public class HalfStoreFileReader extends StoreFile.Reader {
54 private static final Log LOG = LogFactory.getLog(HalfStoreFileReader.class);
55 final boolean top;
56
57
58 protected final byte [] splitkey;
59
60 protected final Cell splitCell;
61
62 private Cell firstKey = null;
63
64 private boolean firstKeySeeked = false;
65
66
67
68
69
70
71
72
73
74
75 public HalfStoreFileReader(final FileSystem fs, final Path p,
76 final CacheConfig cacheConf, final Reference r, final Configuration conf)
77 throws IOException {
78 super(fs, p, cacheConf, conf);
79
80
81
82
83
84 this.splitkey = r.getSplitKey();
85 this.splitCell = new KeyValue.KeyOnlyKeyValue(this.splitkey, 0, this.splitkey.length);
86
87 this.top = Reference.isTopFileRegion(r.getFileRegion());
88 }
89
90
91
92
93
94
95
96
97
98
99
100
101 public HalfStoreFileReader(final FileSystem fs, final Path p, final FSDataInputStreamWrapper in,
102 long size, final CacheConfig cacheConf, final Reference r, final Configuration conf)
103 throws IOException {
104 super(fs, p, in, size, cacheConf, conf);
105
106
107
108
109
110 this.splitkey = r.getSplitKey();
111 this.splitCell = new KeyValue.KeyOnlyKeyValue(this.splitkey, 0, this.splitkey.length);
112
113 this.top = Reference.isTopFileRegion(r.getFileRegion());
114 }
115
116 protected boolean isTop() {
117 return this.top;
118 }
119
120 @Override
121 public HFileScanner getScanner(final boolean cacheBlocks,
122 final boolean pread, final boolean isCompaction) {
123 final HFileScanner s = super.getScanner(cacheBlocks, pread, isCompaction);
124 return new HFileScanner() {
125 final HFileScanner delegate = s;
126 public boolean atEnd = false;
127
128 public Cell getKey() {
129 if (atEnd) return null;
130 return delegate.getKey();
131 }
132
133 public String getKeyString() {
134 if (atEnd) return null;
135
136 return delegate.getKeyString();
137 }
138
139 public ByteBuffer getValue() {
140 if (atEnd) return null;
141
142 return delegate.getValue();
143 }
144
145 public String getValueString() {
146 if (atEnd) return null;
147
148 return delegate.getValueString();
149 }
150
151 public Cell getCell() {
152 if (atEnd) return null;
153
154 return delegate.getCell();
155 }
156
157 public boolean next() throws IOException {
158 if (atEnd) return false;
159
160 boolean b = delegate.next();
161 if (!b) {
162 return b;
163 }
164
165 if (!top) {
166 if (getComparator().compare(splitCell, getKey()) <= 0) {
167 atEnd = true;
168 return false;
169 }
170 }
171 return true;
172 }
173
174 @Override
175 public boolean seekTo() throws IOException {
176 if (top) {
177 int r = this.delegate.seekTo(splitCell);
178 if (r == HConstants.INDEX_KEY_MAGIC) {
179 return true;
180 }
181 if (r < 0) {
182
183 return this.delegate.seekTo();
184 }
185 if (r > 0) {
186 return this.delegate.next();
187 }
188 return true;
189 }
190
191 boolean b = delegate.seekTo();
192 if (!b) {
193 return b;
194 }
195
196 return (this.delegate.getReader().getComparator().compare(splitCell, getKey())) > 0;
197 }
198
199 public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
200 return this.delegate.getReader();
201 }
202
203 public boolean isSeeked() {
204 return this.delegate.isSeeked();
205 }
206
207 @Override
208 public int seekTo(Cell key) throws IOException {
209 if (top) {
210 if (getComparator().compareKeyIgnoresMvcc(key, splitCell) < 0) {
211 return -1;
212 }
213 } else {
214 if (getComparator().compareKeyIgnoresMvcc(key, splitCell) >= 0) {
215
216
217 boolean res = delegate.seekBefore(splitCell);
218 if (!res) {
219 throw new IOException(
220 "Seeking for a key in bottom of file, but key exists in top of file, " +
221 "failed on seekBefore(midkey)");
222 }
223 return 1;
224 }
225 }
226 return delegate.seekTo(key);
227 }
228
229 @Override
230 public int reseekTo(Cell key) throws IOException {
231
232
233
234 if (top) {
235 if (getComparator().compareKeyIgnoresMvcc(key, splitCell) < 0) {
236 return -1;
237 }
238 } else {
239 if (getComparator().compareKeyIgnoresMvcc(key, splitCell) >= 0) {
240
241
242 boolean res = delegate.seekBefore(splitCell);
243 if (!res) {
244 throw new IOException("Seeking for a key in bottom of file, but"
245 + " key exists in top of file, failed on seekBefore(midkey)");
246 }
247 return 1;
248 }
249 }
250 if (atEnd) {
251
252 return 1;
253 }
254 return delegate.reseekTo(key);
255 }
256
257 @Override
258 public boolean seekBefore(Cell key) throws IOException {
259 if (top) {
260 Cell fk = getFirstKey();
261 if (getComparator().compareKeyIgnoresMvcc(key, fk) <= 0) {
262 return false;
263 }
264 } else {
265
266
267 if (getComparator().compareKeyIgnoresMvcc(key, splitCell) >= 0) {
268 boolean ret = this.delegate.seekBefore(splitCell);
269 if (ret) {
270 atEnd = false;
271 }
272 return ret;
273 }
274 }
275 boolean ret = this.delegate.seekBefore(key);
276 if (ret) {
277 atEnd = false;
278 }
279 return ret;
280 }
281
282 @Override
283 public Cell getNextIndexedKey() {
284 return null;
285 }
286
287 @Override
288 public void close() {
289 this.delegate.close();
290 }
291
292 @Override
293 public void shipped() throws IOException {
294 this.delegate.shipped();
295 }
296 };
297 }
298
299 @Override
300 public boolean passesKeyRangeFilter(Scan scan) {
301 return true;
302 }
303
304 @Override
305 public Cell getLastKey() {
306 if (top) {
307 return super.getLastKey();
308 }
309
310 HFileScanner scanner = getScanner(true, true);
311 try {
312 if (scanner.seekBefore(this.splitCell)) {
313 return scanner.getKey();
314 }
315 } catch (IOException e) {
316 LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
317 }
318 return null;
319 }
320
321 @Override
322 public Cell midkey() throws IOException {
323
324 return null;
325 }
326
327 @Override
328 public Cell getFirstKey() {
329 if (!firstKeySeeked) {
330 HFileScanner scanner = getScanner(true, true, false);
331 try {
332 if (scanner.seekTo()) {
333 this.firstKey = scanner.getKey();
334 }
335 firstKeySeeked = true;
336 } catch (IOException e) {
337 LOG.warn("Failed seekTo first KV in the file", e);
338 }
339 }
340 return this.firstKey;
341 }
342
343 @Override
344 public long getEntries() {
345
346 return super.getEntries() / 2;
347 }
348
349 @Override
350 public long getFilterEntries() {
351
352 return super.getFilterEntries() / 2;
353 }
354 }