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 public HalfStoreFileReader(final FileSystem fs, final Path p, final FSDataInputStreamWrapper in,
97 long size, final CacheConfig cacheConf, final Reference r,
98 final DataBlockEncoding preferredEncodingInCache) throws IOException {
99 super(fs, p, in, size, cacheConf, preferredEncodingInCache);
100
101
102
103
104
105 this.splitkey = r.getSplitKey();
106
107 this.top = Reference.isTopFileRegion(r.getFileRegion());
108 }
109
110 protected boolean isTop() {
111 return this.top;
112 }
113
114 @Override
115 public HFileScanner getScanner(final boolean cacheBlocks,
116 final boolean pread, final boolean isCompaction) {
117 final HFileScanner s = super.getScanner(cacheBlocks, pread, isCompaction);
118 return new HFileScanner() {
119 final HFileScanner delegate = s;
120 public boolean atEnd = false;
121
122 public ByteBuffer getKey() {
123 if (atEnd) return null;
124 return delegate.getKey();
125 }
126
127 public String getKeyString() {
128 if (atEnd) return null;
129
130 return delegate.getKeyString();
131 }
132
133 public ByteBuffer getValue() {
134 if (atEnd) return null;
135
136 return delegate.getValue();
137 }
138
139 public String getValueString() {
140 if (atEnd) return null;
141
142 return delegate.getValueString();
143 }
144
145 public KeyValue getKeyValue() {
146 if (atEnd) return null;
147
148 return delegate.getKeyValue();
149 }
150
151 public boolean next() throws IOException {
152 if (atEnd) return false;
153
154 boolean b = delegate.next();
155 if (!b) {
156 return b;
157 }
158
159 if (!top) {
160 ByteBuffer bb = getKey();
161 if (getComparator().compare(bb.array(), bb.arrayOffset(), bb.limit(),
162 splitkey, 0, splitkey.length) >= 0) {
163 atEnd = true;
164 return false;
165 }
166 }
167 return true;
168 }
169
170 public boolean seekBefore(byte[] key) throws IOException {
171 return seekBefore(key, 0, key.length);
172 }
173
174 public boolean seekBefore(byte [] key, int offset, int length)
175 throws IOException {
176 if (top) {
177 byte[] fk = getFirstKey();
178
179 if (fk == null) return false;
180 if (getComparator().compare(key, offset, length, fk, 0,
181 fk.length) <= 0) {
182 return false;
183 }
184 } else {
185
186 if (getComparator().compare(key, offset, length, splitkey, 0,
187 splitkey.length) >= 0) {
188 return this.delegate.seekBefore(splitkey, 0, splitkey.length);
189 }
190 }
191 return this.delegate.seekBefore(key, offset, length);
192 }
193
194 public boolean seekTo() throws IOException {
195 if (top) {
196 int r = this.delegate.seekTo(splitkey);
197 if (r < 0) {
198
199 return this.delegate.seekTo();
200 }
201 if (r > 0) {
202 return this.delegate.next();
203 }
204 return true;
205 }
206
207 boolean b = delegate.seekTo();
208 if (!b) {
209 return b;
210 }
211
212 ByteBuffer k = this.delegate.getKey();
213 return this.delegate.getReader().getComparator().
214 compare(k.array(), k.arrayOffset(), k.limit(),
215 splitkey, 0, splitkey.length) < 0;
216 }
217
218 public int seekTo(byte[] key) throws IOException {
219 return seekTo(key, 0, key.length);
220 }
221
222 public int seekTo(byte[] key, int offset, int length) throws IOException {
223 if (top) {
224 if (getComparator().compare(key, offset, length, splitkey, 0,
225 splitkey.length) < 0) {
226 return -1;
227 }
228 } else {
229 if (getComparator().compare(key, offset, length, splitkey, 0,
230 splitkey.length) >= 0) {
231
232
233 boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
234 if (!res) {
235 throw new IOException("Seeking for a key in bottom of file, but key exists in top of file, failed on seekBefore(midkey)");
236 }
237 return 1;
238 }
239 }
240 return delegate.seekTo(key, offset, length);
241 }
242
243 @Override
244 public int reseekTo(byte[] key) throws IOException {
245 return reseekTo(key, 0, key.length);
246 }
247
248 @Override
249 public int reseekTo(byte[] key, int offset, int length)
250 throws IOException {
251
252
253 if (top) {
254 if (getComparator().compare(key, offset, length, splitkey, 0,
255 splitkey.length) < 0) {
256 return -1;
257 }
258 } else {
259 if (getComparator().compare(key, offset, length, splitkey, 0,
260 splitkey.length) >= 0) {
261
262
263 boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
264 if (!res) {
265 throw new IOException("Seeking for a key in bottom of file, but" +
266 " key exists in top of file, failed on seekBefore(midkey)");
267 }
268 return 1;
269 }
270 }
271 if (atEnd) {
272
273 return 1;
274 }
275 return delegate.reseekTo(key, offset, length);
276 }
277
278 public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
279 return this.delegate.getReader();
280 }
281
282 public boolean isSeeked() {
283 return this.delegate.isSeeked();
284 }
285 };
286 }
287
288 @Override
289 public byte[] getLastKey() {
290 if (top) {
291 return super.getLastKey();
292 }
293
294 HFileScanner scanner = getScanner(true, true);
295 try {
296 if (scanner.seekBefore(this.splitkey)) {
297 return Bytes.toBytes(scanner.getKey());
298 }
299 } catch (IOException e) {
300 LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
301 }
302 return null;
303 }
304
305 @Override
306 public byte[] midkey() throws IOException {
307
308 return null;
309 }
310
311 @Override
312 public byte[] getFirstKey() {
313 if (!firstKeySeeked) {
314 HFileScanner scanner = getScanner(true, true, false);
315 try {
316 if (scanner.seekTo()) {
317 this.firstKey = Bytes.toBytes(scanner.getKey());
318 }
319 firstKeySeeked = true;
320 } catch (IOException e) {
321 LOG.warn("Failed seekTo first KV in the file", e);
322 }
323 }
324 return this.firstKey;
325 }
326
327 @Override
328 public long getEntries() {
329
330 return super.getEntries() / 2;
331 }
332
333 @Override
334 public long getFilterEntries() {
335
336 return super.getFilterEntries() / 2;
337 }
338 }