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.hfile.BlockCache;
31 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
32 import org.apache.hadoop.hbase.regionserver.StoreFile;
33 import org.apache.hadoop.hbase.util.Bytes;
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48 public class HalfStoreFileReader extends StoreFile.Reader {
49 final Log LOG = LogFactory.getLog(HalfStoreFileReader.class);
50 final boolean top;
51
52
53 protected final byte [] splitkey;
54
55
56
57
58
59
60
61
62 public HalfStoreFileReader(final FileSystem fs, final Path p, final BlockCache c,
63 final Reference r)
64 throws IOException {
65 super(fs, p, c, false);
66
67
68
69
70
71 this.splitkey = r.getSplitKey();
72
73 this.top = Reference.isTopFileRegion(r.getFileRegion());
74 }
75
76 protected boolean isTop() {
77 return this.top;
78 }
79
80 @Override
81 public HFileScanner getScanner(final boolean cacheBlocks, final boolean pread) {
82 final HFileScanner s = super.getScanner(cacheBlocks, pread);
83 return new HFileScanner() {
84 final HFileScanner delegate = s;
85 public boolean atEnd = false;
86
87 public ByteBuffer getKey() {
88 if (atEnd) return null;
89 return delegate.getKey();
90 }
91
92 public String getKeyString() {
93 if (atEnd) return null;
94
95 return delegate.getKeyString();
96 }
97
98 public ByteBuffer getValue() {
99 if (atEnd) return null;
100
101 return delegate.getValue();
102 }
103
104 public String getValueString() {
105 if (atEnd) return null;
106
107 return delegate.getValueString();
108 }
109
110 public KeyValue getKeyValue() {
111 if (atEnd) return null;
112
113 return delegate.getKeyValue();
114 }
115
116 public boolean next() throws IOException {
117 if (atEnd) return false;
118
119 boolean b = delegate.next();
120 if (!b) {
121 return b;
122 }
123
124 if (!top) {
125 ByteBuffer bb = getKey();
126 if (getComparator().compare(bb.array(), bb.arrayOffset(), bb.limit(),
127 splitkey, 0, splitkey.length) >= 0) {
128 atEnd = true;
129 return false;
130 }
131 }
132 return true;
133 }
134
135 public boolean seekBefore(byte[] key) throws IOException {
136 return seekBefore(key, 0, key.length);
137 }
138
139 public boolean seekBefore(byte [] key, int offset, int length)
140 throws IOException {
141 if (top) {
142 if (getComparator().compare(key, offset, length, splitkey, 0,
143 splitkey.length) < 0) {
144 return false;
145 }
146 } else {
147 if (getComparator().compare(key, offset, length, splitkey, 0,
148 splitkey.length) >= 0) {
149 return seekBefore(splitkey, 0, splitkey.length);
150 }
151 }
152 return this.delegate.seekBefore(key, offset, length);
153 }
154
155 public boolean seekTo() throws IOException {
156 if (top) {
157 int r = this.delegate.seekTo(splitkey);
158 if (r < 0) {
159
160 return this.delegate.seekTo();
161 }
162 if (r > 0) {
163 return this.delegate.next();
164 }
165 return true;
166 }
167
168 boolean b = delegate.seekTo();
169 if (!b) {
170 return b;
171 }
172
173 ByteBuffer k = this.delegate.getKey();
174 return this.delegate.getReader().getComparator().
175 compare(k.array(), k.arrayOffset(), k.limit(),
176 splitkey, 0, splitkey.length) < 0;
177 }
178
179 public int seekTo(byte[] key) throws IOException {
180 return seekTo(key, 0, key.length);
181 }
182
183 public int seekTo(byte[] key, int offset, int length) throws IOException {
184 if (top) {
185 if (getComparator().compare(key, offset, length, splitkey, 0,
186 splitkey.length) < 0) {
187 return -1;
188 }
189 } else {
190 if (getComparator().compare(key, offset, length, splitkey, 0,
191 splitkey.length) >= 0) {
192
193
194 boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
195 if (!res) {
196 throw new IOException("Seeking for a key in bottom of file, but key exists in top of file, failed on seekBefore(midkey)");
197 }
198 return 1;
199 }
200 }
201 return delegate.seekTo(key, offset, length);
202 }
203
204 @Override
205 public int reseekTo(byte[] key) throws IOException {
206 return reseekTo(key, 0, key.length);
207 }
208
209 @Override
210 public int reseekTo(byte[] key, int offset, int length)
211 throws IOException {
212
213
214 if (top) {
215 if (getComparator().compare(key, offset, length, splitkey, 0,
216 splitkey.length) < 0) {
217 return -1;
218 }
219 } else {
220 if (getComparator().compare(key, offset, length, splitkey, 0,
221 splitkey.length) >= 0) {
222
223
224 boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
225 if (!res) {
226 throw new IOException("Seeking for a key in bottom of file, but" +
227 " key exists in top of file, failed on seekBefore(midkey)");
228 }
229 return 1;
230 }
231 }
232 return delegate.reseekTo(key, offset, length);
233 }
234
235 public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
236 return this.delegate.getReader();
237 }
238
239 public boolean isSeeked() {
240 return this.delegate.isSeeked();
241 }
242 };
243 }
244
245 @Override
246 public byte[] getLastKey() {
247 if (top) {
248 return super.getLastKey();
249 }
250
251 HFileScanner scanner = getScanner(true, true);
252 try {
253 if (scanner.seekBefore(this.splitkey)) {
254 return Bytes.toBytes(scanner.getKey());
255 }
256 } catch (IOException e) {
257 LOG.warn("Failed seekBefore " + Bytes.toString(this.splitkey), e);
258 }
259 return null;
260 }
261
262 @Override
263 public byte[] midkey() throws IOException {
264
265 return null;
266 }
267 }