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