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 byte[] 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 ByteBuffer 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 getKeyValue() {
152 if (atEnd) return null;
153
154 return delegate.getKeyValue();
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 ByteBuffer bb = getKey();
167 if (getComparator().compareFlatKey(bb.array(), bb.arrayOffset(), bb.limit(),
168 splitkey, 0, splitkey.length) >= 0) {
169 atEnd = true;
170 return false;
171 }
172 }
173 return true;
174 }
175
176 @Override
177 public boolean seekBefore(byte[] key) throws IOException {
178 return seekBefore(key, 0, key.length);
179 }
180
181 @Override
182 public boolean seekBefore(byte [] key, int offset, int length)
183 throws IOException {
184 return seekBefore(new KeyValue.KeyOnlyKeyValue(key, offset, length));
185 }
186
187 @Override
188 public boolean seekTo() throws IOException {
189 if (top) {
190 int r = this.delegate.seekTo(new KeyValue.KeyOnlyKeyValue(splitkey, 0, splitkey.length));
191 if (r == HConstants.INDEX_KEY_MAGIC) {
192 return true;
193 }
194 if (r < 0) {
195
196 return this.delegate.seekTo();
197 }
198 if (r > 0) {
199 return this.delegate.next();
200 }
201 return true;
202 }
203
204 boolean b = delegate.seekTo();
205 if (!b) {
206 return b;
207 }
208
209 ByteBuffer k = this.delegate.getKey();
210 return this.delegate.getReader().getComparator().
211 compareFlatKey(k.array(), k.arrayOffset(), k.limit(),
212 splitkey, 0, splitkey.length) < 0;
213 }
214
215 @Override
216 public int seekTo(byte[] key) throws IOException {
217 return seekTo(key, 0, key.length);
218 }
219
220 @Override
221 public int seekTo(byte[] key, int offset, int length) throws IOException {
222 return seekTo(new KeyValue.KeyOnlyKeyValue(key, offset, length));
223 }
224
225 @Override
226 public int reseekTo(byte[] key) throws IOException {
227 return reseekTo(key, 0, key.length);
228 }
229
230 @Override
231 public int reseekTo(byte[] key, int offset, int length)
232 throws IOException {
233
234
235 return reseekTo(new KeyValue.KeyOnlyKeyValue(key, offset, length));
236 }
237
238 public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
239 return this.delegate.getReader();
240 }
241
242 public boolean isSeeked() {
243 return this.delegate.isSeeked();
244 }
245
246 @Override
247 public int seekTo(Cell key) throws IOException {
248 if (top) {
249 if (getComparator().compareOnlyKeyPortion(key, splitCell) < 0) {
250 return -1;
251 }
252 } else {
253 if (getComparator().compareOnlyKeyPortion(key, splitCell) >= 0) {
254
255
256 boolean res = delegate.seekBefore(splitCell);
257 if (!res) {
258 throw new IOException(
259 "Seeking for a key in bottom of file, but key exists in top of file, " +
260 "failed on seekBefore(midkey)");
261 }
262 return 1;
263 }
264 }
265 return delegate.seekTo(key);
266 }
267
268 @Override
269 public int reseekTo(Cell key) throws IOException {
270
271
272
273 if (top) {
274 if (getComparator().compareOnlyKeyPortion(key, splitCell) < 0) {
275 return -1;
276 }
277 } else {
278 if (getComparator().compareOnlyKeyPortion(key, splitCell) >= 0) {
279
280
281 boolean res = delegate.seekBefore(splitCell);
282 if (!res) {
283 throw new IOException("Seeking for a key in bottom of file, but"
284 + " key exists in top of file, failed on seekBefore(midkey)");
285 }
286 return 1;
287 }
288 }
289 if (atEnd) {
290
291 return 1;
292 }
293 return delegate.reseekTo(key);
294 }
295
296 @Override
297 public boolean seekBefore(Cell key) throws IOException {
298 if (top) {
299 Cell fk = new KeyValue.KeyOnlyKeyValue(getFirstKey(), 0, getFirstKey().length);
300 if (getComparator().compareOnlyKeyPortion(key, fk) <= 0) {
301 return false;
302 }
303 } else {
304
305
306 if (getComparator().compareOnlyKeyPortion(key, splitCell) >= 0) {
307 boolean ret = this.delegate.seekBefore(splitCell);
308 if (ret) {
309 atEnd = false;
310 }
311 return ret;
312 }
313 }
314 boolean ret = this.delegate.seekBefore(key);
315 if (ret) {
316 atEnd = false;
317 }
318 return ret;
319 }
320
321 @Override
322 public Cell getNextIndexedKey() {
323 return null;
324 }
325
326 @Override
327 public void close() {
328 }
329 };
330 }
331
332 @Override
333 public boolean passesKeyRangeFilter(Scan scan) {
334 return true;
335 }
336
337 @Override
338 public byte[] getLastKey() {
339 if (top) {
340 return super.getLastKey();
341 }
342
343 HFileScanner scanner = getScanner(true, true);
344 try {
345 if (scanner.seekBefore(this.splitkey)) {
346 return Bytes.toBytes(scanner.getKey());
347 }
348 } catch (IOException e) {
349 LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
350 }
351 return null;
352 }
353
354 @Override
355 public byte[] midkey() throws IOException {
356
357 return null;
358 }
359
360 @Override
361 public byte[] getFirstKey() {
362 if (!firstKeySeeked) {
363 HFileScanner scanner = getScanner(true, true, false);
364 try {
365 if (scanner.seekTo()) {
366 this.firstKey = Bytes.toBytes(scanner.getKey());
367 }
368 firstKeySeeked = true;
369 } catch (IOException e) {
370 LOG.warn("Failed seekTo first KV in the file", e);
371 }
372 }
373 return this.firstKey;
374 }
375
376 @Override
377 public long getEntries() {
378
379 return super.getEntries() / 2;
380 }
381
382 @Override
383 public long getFilterEntries() {
384
385 return super.getFilterEntries() / 2;
386 }
387 }