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