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.FSDataInputStream;
28 import org.apache.hadoop.fs.FileSystem;
29 import org.apache.hadoop.fs.Path;
30 import org.apache.hadoop.hbase.KeyValue;
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 @InterfaceAudience.Private
51 public class HalfStoreFileReader extends StoreFile.Reader {
52 final Log LOG = LogFactory.getLog(HalfStoreFileReader.class);
53 final boolean top;
54
55
56 protected final byte [] splitkey;
57
58 private byte[] firstKey = null;
59
60 private boolean firstKeySeeked = false;
61
62
63
64
65
66
67
68
69
70
71 public HalfStoreFileReader(final FileSystem fs, final Path p,
72 final CacheConfig cacheConf, final Reference r,
73 DataBlockEncoding preferredEncodingInCache) throws IOException {
74 super(fs, p, cacheConf, preferredEncodingInCache);
75
76
77
78
79
80 this.splitkey = r.getSplitKey();
81
82 this.top = Reference.isTopFileRegion(r.getFileRegion());
83 }
84
85
86
87
88
89
90
91
92
93
94
95
96
97 public HalfStoreFileReader(final FileSystem fs, final Path p, final FSDataInputStream in,
98 final FSDataInputStream inNoChecksum, long size, final CacheConfig cacheConf,
99 final Reference r, final DataBlockEncoding preferredEncodingInCache) throws IOException {
100 super(fs, p, in, inNoChecksum, size, cacheConf, preferredEncodingInCache, true);
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().compare(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().compare(key, offset, length, fk, 0,
182 fk.length) <= 0) {
183 return false;
184 }
185 } else {
186
187 if (getComparator().compare(key, offset, length, splitkey, 0,
188 splitkey.length) >= 0) {
189 return this.delegate.seekBefore(splitkey, 0, splitkey.length);
190 }
191 }
192 return this.delegate.seekBefore(key, offset, length);
193 }
194
195 public boolean seekTo() throws IOException {
196 if (top) {
197 int r = this.delegate.seekTo(splitkey);
198 if (r < 0) {
199
200 return this.delegate.seekTo();
201 }
202 if (r > 0) {
203 return this.delegate.next();
204 }
205 return true;
206 }
207
208 boolean b = delegate.seekTo();
209 if (!b) {
210 return b;
211 }
212
213 ByteBuffer k = this.delegate.getKey();
214 return this.delegate.getReader().getComparator().
215 compare(k.array(), k.arrayOffset(), k.limit(),
216 splitkey, 0, splitkey.length) < 0;
217 }
218
219 public int seekTo(byte[] key) throws IOException {
220 return seekTo(key, 0, key.length);
221 }
222
223 public int seekTo(byte[] key, int offset, int length) throws IOException {
224 if (top) {
225 if (getComparator().compare(key, offset, length, splitkey, 0,
226 splitkey.length) < 0) {
227 return -1;
228 }
229 } else {
230 if (getComparator().compare(key, offset, length, splitkey, 0,
231 splitkey.length) >= 0) {
232
233
234 boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
235 if (!res) {
236 throw new IOException("Seeking for a key in bottom of file, but key exists in top of file, failed on seekBefore(midkey)");
237 }
238 return 1;
239 }
240 }
241 return delegate.seekTo(key, offset, length);
242 }
243
244 @Override
245 public int reseekTo(byte[] key) throws IOException {
246 return reseekTo(key, 0, key.length);
247 }
248
249 @Override
250 public int reseekTo(byte[] key, int offset, int length)
251 throws IOException {
252
253
254 if (top) {
255 if (getComparator().compare(key, offset, length, splitkey, 0,
256 splitkey.length) < 0) {
257 return -1;
258 }
259 } else {
260 if (getComparator().compare(key, offset, length, splitkey, 0,
261 splitkey.length) >= 0) {
262
263
264 boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
265 if (!res) {
266 throw new IOException("Seeking for a key in bottom of file, but" +
267 " key exists in top of file, failed on seekBefore(midkey)");
268 }
269 return 1;
270 }
271 }
272 if (atEnd) {
273
274 return 1;
275 }
276 return delegate.reseekTo(key, offset, length);
277 }
278
279 public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
280 return this.delegate.getReader();
281 }
282
283 public boolean isSeeked() {
284 return this.delegate.isSeeked();
285 }
286 };
287 }
288
289 @Override
290 public byte[] getLastKey() {
291 if (top) {
292 return super.getLastKey();
293 }
294
295 HFileScanner scanner = getScanner(true, true);
296 try {
297 if (scanner.seekBefore(this.splitkey)) {
298 return Bytes.toBytes(scanner.getKey());
299 }
300 } catch (IOException e) {
301 LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
302 }
303 return null;
304 }
305
306 @Override
307 public byte[] midkey() throws IOException {
308
309 return null;
310 }
311
312 @Override
313 public byte[] getFirstKey() {
314 if (!firstKeySeeked) {
315 HFileScanner scanner = getScanner(true, true, false);
316 try {
317 if (scanner.seekTo()) {
318 this.firstKey = Bytes.toBytes(scanner.getKey());
319 }
320 firstKeySeeked = true;
321 } catch (IOException e) {
322 LOG.warn("Failed seekTo first KV in the file", e);
323 }
324 }
325 return this.firstKey;
326 }
327 }