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.classification.InterfaceAudience;
27 import org.apache.hadoop.fs.FileSystem;
28 import org.apache.hadoop.fs.Path;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.KeyValue;
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 @InterfaceAudience.Private
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 public HalfStoreFileReader(final FileSystem fs, final Path p,
70 final CacheConfig cacheConf, final Reference r) throws IOException {
71 super(fs, p, cacheConf);
72
73
74
75
76
77 this.splitkey = r.getSplitKey();
78
79 this.top = Reference.isTopFileRegion(r.getFileRegion());
80 }
81
82
83
84
85
86
87
88
89
90
91
92 public HalfStoreFileReader(final FileSystem fs, final Path p, final FSDataInputStreamWrapper in,
93 long size, final CacheConfig cacheConf, final Reference r) throws IOException {
94 super(fs, p, in, size, cacheConf);
95
96
97
98
99
100 this.splitkey = r.getSplitKey();
101
102 this.top = Reference.isTopFileRegion(r.getFileRegion());
103 }
104
105 protected boolean isTop() {
106 return this.top;
107 }
108
109 @Override
110 public HFileScanner getScanner(final boolean cacheBlocks,
111 final boolean pread, final boolean isCompaction) {
112 final HFileScanner s = super.getScanner(cacheBlocks, pread, isCompaction);
113 return new HFileScanner() {
114 final HFileScanner delegate = s;
115 public boolean atEnd = false;
116
117 public ByteBuffer getKey() {
118 if (atEnd) return null;
119 return delegate.getKey();
120 }
121
122 public String getKeyString() {
123 if (atEnd) return null;
124
125 return delegate.getKeyString();
126 }
127
128 public ByteBuffer getValue() {
129 if (atEnd) return null;
130
131 return delegate.getValue();
132 }
133
134 public String getValueString() {
135 if (atEnd) return null;
136
137 return delegate.getValueString();
138 }
139
140 public KeyValue getKeyValue() {
141 if (atEnd) return null;
142
143 return delegate.getKeyValue();
144 }
145
146 public boolean next() throws IOException {
147 if (atEnd) return false;
148
149 boolean b = delegate.next();
150 if (!b) {
151 return b;
152 }
153
154 if (!top) {
155 ByteBuffer bb = getKey();
156 if (getComparator().compareFlatKey(bb.array(), bb.arrayOffset(), bb.limit(),
157 splitkey, 0, splitkey.length) >= 0) {
158 atEnd = true;
159 return false;
160 }
161 }
162 return true;
163 }
164
165 public boolean seekBefore(byte[] key) throws IOException {
166 return seekBefore(key, 0, key.length);
167 }
168
169 public boolean seekBefore(byte [] key, int offset, int length)
170 throws IOException {
171 if (top) {
172 byte[] fk = getFirstKey();
173
174 if (fk == null) return false;
175 if (getComparator().compareFlatKey(key, offset, length, fk, 0,
176 fk.length) <= 0) {
177 return false;
178 }
179 } else {
180
181 if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
182 splitkey.length) >= 0) {
183 return this.delegate.seekBefore(splitkey, 0, splitkey.length);
184 }
185 }
186 return this.delegate.seekBefore(key, offset, length);
187 }
188
189 public boolean seekTo() throws IOException {
190 if (top) {
191 int r = this.delegate.seekTo(splitkey);
192 if (r == HConstants.INDEX_KEY_MAGIC) {
193 return true;
194 }
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 compareFlatKey(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().compareFlatKey(key, offset, length, splitkey, 0,
223 splitkey.length) < 0) {
224 return -1;
225 }
226 } else {
227 if (getComparator().compareFlatKey(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().compareFlatKey(key, offset, length, splitkey, 0,
253 splitkey.length) < 0) {
254 return -1;
255 }
256 } else {
257 if (getComparator().compareFlatKey(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 byte[] getLastKey() {
288 if (top) {
289 return super.getLastKey();
290 }
291
292 HFileScanner scanner = getScanner(true, true);
293 try {
294 if (scanner.seekBefore(this.splitkey)) {
295 return Bytes.toBytes(scanner.getKey());
296 }
297 } catch (IOException e) {
298 LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
299 }
300 return null;
301 }
302
303 @Override
304 public byte[] midkey() throws IOException {
305
306 return null;
307 }
308
309 @Override
310 public byte[] getFirstKey() {
311 if (!firstKeySeeked) {
312 HFileScanner scanner = getScanner(true, true, false);
313 try {
314 if (scanner.seekTo()) {
315 this.firstKey = Bytes.toBytes(scanner.getKey());
316 }
317 firstKeySeeked = true;
318 } catch (IOException e) {
319 LOG.warn("Failed seekTo first KV in the file", e);
320 }
321 }
322 return this.firstKey;
323 }
324
325 @Override
326 public long getEntries() {
327
328 return super.getEntries() / 2;
329 }
330
331 @Override
332 public long getFilterEntries() {
333
334 return super.getFilterEntries() / 2;
335 }
336 }