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.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   * 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     * @param preferredEncodingInCache
69     * @throws IOException
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      // This is not actual midkey for this half-file; its just border
76      // around which we split top and bottom.  Have to look in files to find
77      // actual last and first keys for bottom and top halves.  Half-files don't
78      // have an actual midkey themselves. No midkey is how we indicate file is
79      // not splittable.
80      this.splitkey = r.getSplitKey();
81      // Is it top or bottom half?
82      this.top = Reference.isTopFileRegion(r.getFileRegion());
83    }
84  
85    /**
86     * Creates a half file reader for a hfile referred to by an hfilelink.
87     * @param fs fileystem to read from
88     * @param p path to hfile
89     * @param in {@link FSDataInputStreamWrapper}
90     * @param size Full size of the hfile file
91     * @param cacheConf
92     * @param r original reference file (contains top or bottom)
93     * @param preferredEncodingInCache
94     * @throws IOException
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     // This is not actual midkey for this half-file; its just border
101     // around which we split top and bottom.  Have to look in files to find
102     // actual last and first keys for bottom and top halves.  Half-files don't
103     // have an actual midkey themselves. No midkey is how we indicate file is
104     // not splittable.
105     this.splitkey = r.getSplitKey();
106     // Is it top or bottom half?
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         // constrain the bottom.
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           // This will be null when the file is empty in which we can not seekBefore to any key
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           // The equals sign isn't strictly necessary just here to be consistent with seekTo
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             // midkey is < first key in file
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         // Check key.
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             // we would place the scanner in the second half.
232             // it might be an error to return false here ever...
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         //This function is identical to the corresponding seekTo function except
252         //that we call reseekTo (and not seekTo) on the delegate.
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             // we would place the scanner in the second half.
262             // it might be an error to return false here ever...
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           // skip the 'reseek' and just return 1.
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     // Get a scanner that caches the block and that uses pread.
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     // Returns null to indicate file is not splitable.
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     // Estimate the number of entries as half the original file; this may be wildly inaccurate.
330     return super.getEntries() / 2;
331   }
332 
333   @Override
334   public long getFilterEntries() {
335     // Estimate the number of entries as half the original file; this may be wildly inaccurate.
336     return super.getFilterEntries() / 2;
337   }
338 }