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.conf.Configuration;
28  import org.apache.hadoop.fs.FileSystem;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.HConstants;
31  import org.apache.hadoop.hbase.KeyValue;
32  import org.apache.hadoop.hbase.client.Scan;
33  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
34  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
35  import org.apache.hadoop.hbase.regionserver.StoreFile;
36  import org.apache.hadoop.hbase.util.Bytes;
37  
38  /**
39   * A facade for a {@link org.apache.hadoop.hbase.io.hfile.HFile.Reader} that serves up
40   * either the top or bottom half of a HFile where 'bottom' is the first half
41   * of the file containing the keys that sort lowest and 'top' is the second half
42   * of the file with keys that sort greater than those of the bottom half.
43   * The top includes the split files midkey, of the key that follows if it does
44   * not exist in the file.
45   *
46   * <p>This type works in tandem with the {@link Reference} type.  This class
47   * is used reading while Reference is used writing.
48   *
49   * <p>This file is not splitable.  Calls to {@link #midkey()} return null.
50   */
51  @InterfaceAudience.Private
52  public class HalfStoreFileReader extends StoreFile.Reader {
53    final Log LOG = LogFactory.getLog(HalfStoreFileReader.class);
54    final boolean top;
55    // This is the key we split around.  Its the first possible entry on a row:
56    // i.e. empty column and a timestamp of LATEST_TIMESTAMP.
57    protected final byte [] splitkey;
58    
59    private byte[] firstKey = null;
60    
61    private boolean firstKeySeeked = false;
62  
63    /**
64     * Creates a half file reader for a normal hfile.
65     * @param fs fileystem to read from
66     * @param p path to hfile
67     * @param cacheConf
68     * @param r original reference file (contains top or bottom)
69     * @param conf Configuration
70     * @throws IOException
71     */
72    public HalfStoreFileReader(final FileSystem fs, final Path p,
73        final CacheConfig cacheConf, final Reference r, final Configuration conf)
74        throws IOException {
75      super(fs, p, cacheConf, conf);
76      // This is not actual midkey for this half-file; its just border
77      // around which we split top and bottom.  Have to look in files to find
78      // actual last and first keys for bottom and top halves.  Half-files don't
79      // have an actual midkey themselves. No midkey is how we indicate file is
80      // not splittable.
81      this.splitkey = r.getSplitKey();
82      // Is it top or bottom half?
83      this.top = Reference.isTopFileRegion(r.getFileRegion());
84    }
85  
86    /**
87     * Creates a half file reader for a hfile referred to by an hfilelink.
88     * @param fs fileystem to read from
89     * @param p path to hfile
90     * @param in {@link FSDataInputStreamWrapper}
91     * @param size Full size of the hfile file
92     * @param cacheConf
93     * @param r original reference file (contains top or bottom)
94     * @param conf Configuration
95     * @throws IOException
96     */
97    public HalfStoreFileReader(final FileSystem fs, final Path p, final FSDataInputStreamWrapper in,
98        long size, final CacheConfig cacheConf,  final Reference r, final Configuration conf)
99        throws IOException {
100     super(fs, p, in, size, cacheConf, conf);
101     // This is not actual midkey for this half-file; its just border
102     // around which we split top and bottom.  Have to look in files to find
103     // actual last and first keys for bottom and top halves.  Half-files don't
104     // have an actual midkey themselves. No midkey is how we indicate file is
105     // not splittable.
106     this.splitkey = r.getSplitKey();
107     // Is it top or bottom half?
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         // constrain the bottom.
160         if (!top) {
161           ByteBuffer bb = getKey();
162           if (getComparator().compareFlatKey(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           // This will be null when the file is empty in which we can not seekBefore to any key
180           if (fk == null) return false;
181           if (getComparator().compareFlatKey(key, offset, length, fk, 0,
182               fk.length) <= 0) {
183             return false;
184           }
185         } else {
186           // The equals sign isn't strictly necessary just here to be consistent with seekTo
187           if (getComparator().compareFlatKey(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 == HConstants.INDEX_KEY_MAGIC) {
199             return true;
200           }
201           if (r < 0) {
202             // midkey is < first key in file
203             return this.delegate.seekTo();
204           }
205           if (r > 0) {
206             return this.delegate.next();
207           }
208           return true;
209         }
210 
211         boolean b = delegate.seekTo();
212         if (!b) {
213           return b;
214         }
215         // Check key.
216         ByteBuffer k = this.delegate.getKey();
217         return this.delegate.getReader().getComparator().
218           compareFlatKey(k.array(), k.arrayOffset(), k.limit(),
219             splitkey, 0, splitkey.length) < 0;
220       }
221 
222       public int seekTo(byte[] key) throws IOException {
223         return seekTo(key, 0, key.length);
224       }
225 
226       public int seekTo(byte[] key, int offset, int length) throws IOException {
227         if (top) {
228           if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
229               splitkey.length) < 0) {
230             return -1;
231           }
232         } else {
233           if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
234               splitkey.length) >= 0) {
235             // we would place the scanner in the second half.
236             // it might be an error to return false here ever...
237             boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
238             if (!res) {
239               throw new IOException("Seeking for a key in bottom of file, but key exists in top of file, failed on seekBefore(midkey)");
240             }
241             return 1;
242           }
243         }
244         return delegate.seekTo(key, offset, length);
245       }
246 
247       @Override
248       public int reseekTo(byte[] key) throws IOException {
249         return reseekTo(key, 0, key.length);
250       }
251 
252       @Override
253       public int reseekTo(byte[] key, int offset, int length)
254       throws IOException {
255         //This function is identical to the corresponding seekTo function except
256         //that we call reseekTo (and not seekTo) on the delegate.
257         if (top) {
258           if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
259               splitkey.length) < 0) {
260             return -1;
261           }
262         } else {
263           if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
264               splitkey.length) >= 0) {
265             // we would place the scanner in the second half.
266             // it might be an error to return false here ever...
267             boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
268             if (!res) {
269               throw new IOException("Seeking for a key in bottom of file, but" +
270                   " key exists in top of file, failed on seekBefore(midkey)");
271             }
272             return 1;
273           }
274         }
275         if (atEnd) {
276           // skip the 'reseek' and just return 1.
277           return 1;
278         }
279         return delegate.reseekTo(key, offset, length);
280       }
281 
282       public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
283         return this.delegate.getReader();
284       }
285 
286       public boolean isSeeked() {
287         return this.delegate.isSeeked();
288       }
289     };
290   }
291   
292   @Override
293   public boolean passesKeyRangeFilter(Scan scan) {
294     return true;
295   }
296   
297   @Override
298   public byte[] getLastKey() {
299     if (top) {
300       return super.getLastKey();
301     }
302     // Get a scanner that caches the block and that uses pread.
303     HFileScanner scanner = getScanner(true, true);
304     try {
305       if (scanner.seekBefore(this.splitkey)) {
306         return Bytes.toBytes(scanner.getKey());
307       }
308     } catch (IOException e) {
309       LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
310     }
311     return null;
312   }
313 
314   @Override
315   public byte[] midkey() throws IOException {
316     // Returns null to indicate file is not splitable.
317     return null;
318   }
319   
320   @Override
321   public byte[] getFirstKey() {
322     if (!firstKeySeeked) {
323       HFileScanner scanner = getScanner(true, true, false);
324       try {
325         if (scanner.seekTo()) {
326           this.firstKey = Bytes.toBytes(scanner.getKey());
327         }
328         firstKeySeeked = true;
329       } catch (IOException e) {
330         LOG.warn("Failed seekTo first KV in the file", e);
331       }
332     }
333     return this.firstKey;
334   }
335 
336   @Override
337   public long getEntries() {
338     // Estimate the number of entries as half the original file; this may be wildly inaccurate.
339     return super.getEntries() / 2;
340   }
341 
342   @Override
343   public long getFilterEntries() {
344     // Estimate the number of entries as half the original file; this may be wildly inaccurate.
345     return super.getFilterEntries() / 2;
346   }
347 }