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