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.hfile;
20  
21  import java.io.IOException;
22  import java.nio.ByteBuffer;
23  import java.util.concurrent.atomic.AtomicInteger;
24  
25  import org.apache.hadoop.conf.Configurable;
26  import org.apache.hadoop.conf.Configuration;
27  import org.apache.hadoop.fs.Path;
28  import org.apache.hadoop.hbase.KeyValue;
29  import org.apache.hadoop.hbase.KeyValue.KVComparator;
30  import org.apache.hadoop.hbase.classification.InterfaceAudience;
31  import org.apache.hadoop.hbase.fs.HFileSystem;
32  import org.apache.hadoop.hbase.io.compress.Compression;
33  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
34  import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
35  
36  /**
37   * Common functionality needed by all versions of {@link HFile} readers.
38   */
39  @InterfaceAudience.Private
40  public abstract class AbstractHFileReader
41      implements HFile.Reader, Configurable {
42    /** Stream to read from. Does checksum verifications in file system */
43  
44    /** The file system stream of the underlying {@link HFile} that
45     * does not do checksum verification in the file system */
46  
47    /** Data block index reader keeping the root data index in memory */
48    protected HFileBlockIndex.BlockIndexReader dataBlockIndexReader;
49  
50    /** Meta block index reader -- always single level */
51    protected HFileBlockIndex.BlockIndexReader metaBlockIndexReader;
52  
53    protected final FixedFileTrailer trailer;
54  
55    /** Filled when we read in the trailer. */
56    protected final Compression.Algorithm compressAlgo;
57  
58    private boolean isPrimaryReplicaReader;
59  
60    /**
61     * What kind of data block encoding should be used while reading, writing,
62     * and handling cache.
63     */
64    protected HFileDataBlockEncoder dataBlockEncoder =
65        NoOpDataBlockEncoder.INSTANCE;
66  
67    /** Last key in the file. Filled in when we read in the file info */
68    protected byte [] lastKey = null;
69  
70    /** Average key length read from file info */
71    protected int avgKeyLen = -1;
72  
73    /** Average value length read from file info */
74    protected int avgValueLen = -1;
75  
76    /** Key comparator */
77    protected KVComparator comparator = new KVComparator();
78  
79    /** Size of this file. */
80    protected final long fileSize;
81  
82    /** Block cache configuration. */
83    protected final CacheConfig cacheConf;
84  
85    /** Path of file */
86    protected final Path path;
87  
88    /** File name to be used for block names */
89    protected final String name;
90  
91    protected FileInfo fileInfo;
92  
93    /** The filesystem used for accesing data */
94    protected HFileSystem hfs;
95  
96    protected Configuration conf;
97  
98    @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD")
99    protected AbstractHFileReader(Path path, FixedFileTrailer trailer,
100       final long fileSize, final CacheConfig cacheConf, final HFileSystem hfs,
101       final Configuration conf) {
102     this.trailer = trailer;
103     this.compressAlgo = trailer.getCompressionCodec();
104     this.cacheConf = cacheConf;
105     this.fileSize = fileSize;
106     this.path = path;
107     this.name = path.getName();
108     this.hfs = hfs; // URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD
109     this.conf = conf;
110   }
111 
112   @SuppressWarnings("serial")
113   public static class BlockIndexNotLoadedException
114       extends IllegalStateException {
115     public BlockIndexNotLoadedException() {
116       // Add a message in case anyone relies on it as opposed to class name.
117       super("Block index not loaded");
118     }
119   }
120 
121   protected String toStringFirstKey() {
122     return KeyValue.keyToString(getFirstKey());
123   }
124 
125   protected String toStringLastKey() {
126     return KeyValue.keyToString(getLastKey());
127   }
128 
129   public abstract boolean isFileInfoLoaded();
130 
131   @Override
132   public String toString() {
133     return "reader=" + path.toString() +
134         (!isFileInfoLoaded()? "":
135           ", compression=" + compressAlgo.getName() +
136           ", cacheConf=" + cacheConf +
137           ", firstKey=" + toStringFirstKey() +
138           ", lastKey=" + toStringLastKey()) +
139           ", avgKeyLen=" + avgKeyLen +
140           ", avgValueLen=" + avgValueLen +
141           ", entries=" + trailer.getEntryCount() +
142           ", length=" + fileSize;
143   }
144 
145   @Override
146   public long length() {
147     return fileSize;
148   }
149 
150   /**
151    * Create a Scanner on this file. No seeks or reads are done on creation. Call
152    * {@link HFileScanner#seekTo(byte[])} to position an start the read. There is
153    * nothing to clean up in a Scanner. Letting go of your references to the
154    * scanner is sufficient. NOTE: Do not use this overload of getScanner for
155    * compactions.
156    *
157    * @param cacheBlocks True if we should cache blocks read in by this scanner.
158    * @param pread Use positional read rather than seek+read if true (pread is
159    *          better for random reads, seek+read is better scanning).
160    * @return Scanner on this file.
161    */
162   @Override
163   public HFileScanner getScanner(boolean cacheBlocks, final boolean pread) {
164     return getScanner(cacheBlocks, pread, false);
165   }
166 
167   /**
168    * @return the first key in the file. May be null if file has no entries. Note
169    *         that this is not the first row key, but rather the byte form of the
170    *         first KeyValue.
171    */
172   @Override
173   public byte [] getFirstKey() {
174     if (dataBlockIndexReader == null) {
175       throw new BlockIndexNotLoadedException();
176     }
177     return dataBlockIndexReader.isEmpty() ? null
178         : dataBlockIndexReader.getRootBlockKey(0);
179   }
180 
181   /**
182    * TODO left from {@link HFile} version 1: move this to StoreFile after Ryan's
183    * patch goes in to eliminate {@link KeyValue} here.
184    *
185    * @return the first row key, or null if the file is empty.
186    */
187   @Override
188   public byte[] getFirstRowKey() {
189     byte[] firstKey = getFirstKey();
190     if (firstKey == null)
191       return null;
192     return KeyValue.createKeyValueFromKey(firstKey).getRow();
193   }
194 
195   /**
196    * TODO left from {@link HFile} version 1: move this to StoreFile after
197    * Ryan's patch goes in to eliminate {@link KeyValue} here.
198    *
199    * @return the last row key, or null if the file is empty.
200    */
201   @Override
202   public byte[] getLastRowKey() {
203     byte[] lastKey = getLastKey();
204     if (lastKey == null)
205       return null;
206     return KeyValue.createKeyValueFromKey(lastKey).getRow();
207   }
208 
209   /** @return number of KV entries in this HFile */
210   @Override
211   public long getEntries() {
212     return trailer.getEntryCount();
213   }
214 
215   /** @return comparator */
216   @Override
217   public KVComparator getComparator() {
218     return comparator;
219   }
220 
221   /** @return compression algorithm */
222   @Override
223   public Compression.Algorithm getCompressionAlgorithm() {
224     return compressAlgo;
225   }
226 
227   /**
228    * @return the total heap size of data and meta block indexes in bytes. Does
229    *         not take into account non-root blocks of a multilevel data index.
230    */
231   public long indexSize() {
232     return (dataBlockIndexReader != null ? dataBlockIndexReader.heapSize() : 0)
233         + ((metaBlockIndexReader != null) ? metaBlockIndexReader.heapSize()
234             : 0);
235   }
236 
237   @Override
238   public String getName() {
239     return name;
240   }
241 
242   @Override
243   public HFileBlockIndex.BlockIndexReader getDataBlockIndexReader() {
244     return dataBlockIndexReader;
245   }
246 
247   @Override
248   public FixedFileTrailer getTrailer() {
249     return trailer;
250   }
251 
252   @Override
253   public boolean isPrimaryReplicaReader() {
254     return isPrimaryReplicaReader;
255   }
256 
257   @Override
258   public void setPrimaryReplicaReader(boolean isPrimaryReplicaReader) {
259     this.isPrimaryReplicaReader = isPrimaryReplicaReader;
260   }
261 
262   public FileInfo loadFileInfo() throws IOException {
263     return fileInfo;
264   }
265 
266   /**
267    * An exception thrown when an operation requiring a scanner to be seeked
268    * is invoked on a scanner that is not seeked.
269    */
270   @SuppressWarnings("serial")
271   public static class NotSeekedException extends IllegalStateException {
272     public NotSeekedException() {
273       super("Not seeked to a key/value");
274     }
275   }
276 
277   protected static abstract class Scanner implements HFileScanner {
278     protected ByteBuffer blockBuffer;
279 
280     protected boolean cacheBlocks;
281     protected final boolean pread;
282     protected final boolean isCompaction;
283 
284     protected int currKeyLen;
285     protected int currValueLen;
286     protected int currMemstoreTSLen;
287     protected long currMemstoreTS;
288 
289     protected AtomicInteger blockFetches = new AtomicInteger();
290 
291     protected final HFile.Reader reader;
292 
293     public Scanner(final HFile.Reader reader, final boolean cacheBlocks,
294         final boolean pread, final boolean isCompaction) {
295       this.reader = reader;
296       this.cacheBlocks = cacheBlocks;
297       this.pread = pread;
298       this.isCompaction = isCompaction;
299     }
300 
301     @Override
302     public boolean isSeeked(){
303       return blockBuffer != null;
304     }
305 
306     @Override
307     public String toString() {
308       return "HFileScanner for reader " + String.valueOf(getReader());
309     }
310 
311     protected void assertSeeked() {
312       if (!isSeeked())
313         throw new NotSeekedException();
314     }
315 
316     @Override
317     public int seekTo(byte[] key) throws IOException {
318       return seekTo(key, 0, key.length);
319     }
320     
321     @Override
322     public boolean seekBefore(byte[] key) throws IOException {
323       return seekBefore(key, 0, key.length);
324     }
325     
326     @Override
327     public int reseekTo(byte[] key) throws IOException {
328       return reseekTo(key, 0, key.length);
329     }
330 
331     @Override
332     public HFile.Reader getReader() {
333       return reader;
334     }
335 
336     @Override
337     public void close() {
338       if (!pread) {
339         // For seek + pread stream socket should be closed when the scanner is closed. HBASE-9393
340         reader.unbufferStream();
341       }
342     }
343   }
344 
345   /** For testing */
346   abstract HFileBlock.FSReader getUncachedBlockReader();
347 
348   public Path getPath() {
349     return path;
350   }
351 
352   @Override
353   public DataBlockEncoding getDataBlockEncoding() {
354     return dataBlockEncoder.getDataBlockEncoding();
355   }
356 
357   public abstract int getMajorVersion();
358 
359   @Override
360   public Configuration getConf() {
361     return conf;
362   }
363 
364   @Override
365   public void setConf(Configuration conf) {
366     this.conf = conf;
367   }
368 }