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