View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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.client.Scan;
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   * A facade for a {@link org.apache.hadoop.hbase.io.hfile.HFile.Reader} that serves up
39   * either the top or bottom half of a HFile where 'bottom' is the first half
40   * of the file containing the keys that sort lowest and 'top' is the second half
41   * of the file with keys that sort greater than those of the bottom half.
42   * The top includes the split files midkey, of the key that follows if it does
43   * not exist in the file.
44   *
45   * <p>This type works in tandem with the {@link Reference} type.  This class
46   * is used reading while Reference is used writing.
47   *
48   * <p>This file is not splitable.  Calls to {@link #midkey()} return null.
49   */
50  @InterfaceAudience.Private
51  public class HalfStoreFileReader extends StoreFile.Reader {
52    final Log LOG = LogFactory.getLog(HalfStoreFileReader.class);
53    final boolean top;
54    // This is the key we split around.  Its the first possible entry on a row:
55    // i.e. empty column and a timestamp of LATEST_TIMESTAMP.
56    protected final byte [] splitkey;
57    
58    private byte[] firstKey = null;
59    
60    private boolean firstKeySeeked = false;
61  
62    /**
63     * Creates a half file reader for a normal hfile.
64     * @param fs fileystem to read from
65     * @param p path to hfile
66     * @param cacheConf
67     * @param r original reference file (contains top or bottom)
68     * @throws IOException
69     */
70    public HalfStoreFileReader(final FileSystem fs, final Path p,
71        final CacheConfig cacheConf, final Reference r) throws IOException {
72      super(fs, p, cacheConf);
73      // This is not actual midkey for this half-file; its just border
74      // around which we split top and bottom.  Have to look in files to find
75      // actual last and first keys for bottom and top halves.  Half-files don't
76      // have an actual midkey themselves. No midkey is how we indicate file is
77      // not splittable.
78      this.splitkey = r.getSplitKey();
79      // Is it top or bottom half?
80      this.top = Reference.isTopFileRegion(r.getFileRegion());
81    }
82  
83    /**
84     * Creates a half file reader for a hfile referred to by an hfilelink.
85     * @param fs fileystem to read from
86     * @param p path to hfile
87     * @param in {@link FSDataInputStreamWrapper}
88     * @param size Full size of the hfile file
89     * @param cacheConf
90     * @param r original reference file (contains top or bottom)
91     * @throws IOException
92     */
93    public HalfStoreFileReader(final FileSystem fs, final Path p, final FSDataInputStreamWrapper in,
94        long size, final CacheConfig cacheConf,  final Reference r) throws IOException {
95      super(fs, p, in, size, cacheConf);
96      // This is not actual midkey for this half-file; its just border
97      // around which we split top and bottom.  Have to look in files to find
98      // actual last and first keys for bottom and top halves.  Half-files don't
99      // have an actual midkey themselves. No midkey is how we indicate file is
100     // not splittable.
101     this.splitkey = r.getSplitKey();
102     // Is it top or bottom half?
103     this.top = Reference.isTopFileRegion(r.getFileRegion());
104   }
105 
106   protected boolean isTop() {
107     return this.top;
108   }
109 
110   @Override
111   public HFileScanner getScanner(final boolean cacheBlocks,
112       final boolean pread, final boolean isCompaction) {
113     final HFileScanner s = super.getScanner(cacheBlocks, pread, isCompaction);
114     return new HFileScanner() {
115       final HFileScanner delegate = s;
116       public boolean atEnd = false;
117 
118       public ByteBuffer getKey() {
119         if (atEnd) return null;
120         return delegate.getKey();
121       }
122 
123       public String getKeyString() {
124         if (atEnd) return null;
125 
126         return delegate.getKeyString();
127       }
128 
129       public ByteBuffer getValue() {
130         if (atEnd) return null;
131 
132         return delegate.getValue();
133       }
134 
135       public String getValueString() {
136         if (atEnd) return null;
137 
138         return delegate.getValueString();
139       }
140 
141       public KeyValue getKeyValue() {
142         if (atEnd) return null;
143 
144         return delegate.getKeyValue();
145       }
146 
147       public boolean next() throws IOException {
148         if (atEnd) return false;
149 
150         boolean b = delegate.next();
151         if (!b) {
152           return b;
153         }
154         // constrain the bottom.
155         if (!top) {
156           ByteBuffer bb = getKey();
157           if (getComparator().compareFlatKey(bb.array(), bb.arrayOffset(), bb.limit(),
158               splitkey, 0, splitkey.length) >= 0) {
159             atEnd = true;
160             return false;
161           }
162         }
163         return true;
164       }
165 
166       public boolean seekBefore(byte[] key) throws IOException {
167         return seekBefore(key, 0, key.length);
168       }
169 
170       public boolean seekBefore(byte [] key, int offset, int length)
171       throws IOException {
172         if (top) {
173           byte[] fk = getFirstKey();
174           // This will be null when the file is empty in which we can not seekBefore to any key
175           if (fk == null) return false;
176           if (getComparator().compareFlatKey(key, offset, length, fk, 0,
177               fk.length) <= 0) {
178             return false;
179           }
180         } else {
181           // The equals sign isn't strictly necessary just here to be consistent with seekTo
182           if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
183               splitkey.length) >= 0) {
184             return this.delegate.seekBefore(splitkey, 0, splitkey.length);
185           }
186         }
187         return this.delegate.seekBefore(key, offset, length);
188       }
189 
190       public boolean seekTo() throws IOException {
191         if (top) {
192           int r = this.delegate.seekTo(splitkey);
193           if (r == HConstants.INDEX_KEY_MAGIC) {
194             return true;
195           }
196           if (r < 0) {
197             // midkey is < first key in file
198             return this.delegate.seekTo();
199           }
200           if (r > 0) {
201             return this.delegate.next();
202           }
203           return true;
204         }
205 
206         boolean b = delegate.seekTo();
207         if (!b) {
208           return b;
209         }
210         // Check key.
211         ByteBuffer k = this.delegate.getKey();
212         return this.delegate.getReader().getComparator().
213           compareFlatKey(k.array(), k.arrayOffset(), k.limit(),
214             splitkey, 0, splitkey.length) < 0;
215       }
216 
217       public int seekTo(byte[] key) throws IOException {
218         return seekTo(key, 0, key.length);
219       }
220 
221       public int seekTo(byte[] key, int offset, int length) throws IOException {
222         if (top) {
223           if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
224               splitkey.length) < 0) {
225             return -1;
226           }
227         } else {
228           if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
229               splitkey.length) >= 0) {
230             // we would place the scanner in the second half.
231             // it might be an error to return false here ever...
232             boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
233             if (!res) {
234               throw new IOException("Seeking for a key in bottom of file, but key exists in top of file, failed on seekBefore(midkey)");
235             }
236             return 1;
237           }
238         }
239         return delegate.seekTo(key, offset, length);
240       }
241 
242       @Override
243       public int reseekTo(byte[] key) throws IOException {
244         return reseekTo(key, 0, key.length);
245       }
246 
247       @Override
248       public int reseekTo(byte[] key, int offset, int length)
249       throws IOException {
250         //This function is identical to the corresponding seekTo function except
251         //that we call reseekTo (and not seekTo) on the delegate.
252         if (top) {
253           if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
254               splitkey.length) < 0) {
255             return -1;
256           }
257         } else {
258           if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
259               splitkey.length) >= 0) {
260             // we would place the scanner in the second half.
261             // it might be an error to return false here ever...
262             boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
263             if (!res) {
264               throw new IOException("Seeking for a key in bottom of file, but" +
265                   " key exists in top of file, failed on seekBefore(midkey)");
266             }
267             return 1;
268           }
269         }
270         if (atEnd) {
271           // skip the 'reseek' and just return 1.
272           return 1;
273         }
274         return delegate.reseekTo(key, offset, length);
275       }
276 
277       public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
278         return this.delegate.getReader();
279       }
280 
281       public boolean isSeeked() {
282         return this.delegate.isSeeked();
283       }
284     };
285   }
286   
287   @Override
288   public boolean passesKeyRangeFilter(Scan scan) {
289     return true;
290   }
291   
292   @Override
293   public byte[] getLastKey() {
294     if (top) {
295       return super.getLastKey();
296     }
297     // Get a scanner that caches the block and that uses pread.
298     HFileScanner scanner = getScanner(true, true);
299     try {
300       if (scanner.seekBefore(this.splitkey)) {
301         return Bytes.toBytes(scanner.getKey());
302       }
303     } catch (IOException e) {
304       LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
305     }
306     return null;
307   }
308 
309   @Override
310   public byte[] midkey() throws IOException {
311     // Returns null to indicate file is not splitable.
312     return null;
313   }
314   
315   @Override
316   public byte[] getFirstKey() {
317     if (!firstKeySeeked) {
318       HFileScanner scanner = getScanner(true, true, false);
319       try {
320         if (scanner.seekTo()) {
321           this.firstKey = Bytes.toBytes(scanner.getKey());
322         }
323         firstKeySeeked = true;
324       } catch (IOException e) {
325         LOG.warn("Failed seekTo first KV in the file", e);
326       }
327     }
328     return this.firstKey;
329   }
330 
331   @Override
332   public long getEntries() {
333     // Estimate the number of entries as half the original file; this may be wildly inaccurate.
334     return super.getEntries() / 2;
335   }
336 
337   @Override
338   public long getFilterEntries() {
339     // Estimate the number of entries as half the original file; this may be wildly inaccurate.
340     return super.getFilterEntries() / 2;
341   }
342 }