View Javadoc

1   /*
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.io.hfile;
22  
23  import java.io.DataOutput;
24  import java.io.DataOutputStream;
25  import java.io.IOException;
26  import java.util.ArrayList;
27  import java.util.List;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.fs.FSDataOutputStream;
33  import org.apache.hadoop.fs.FileSystem;
34  import org.apache.hadoop.fs.Path;
35  import org.apache.hadoop.hbase.HConstants;
36  import org.apache.hadoop.hbase.KeyValue;
37  import org.apache.hadoop.hbase.KeyValue.KeyComparator;
38  import org.apache.hadoop.hbase.fs.HFileSystem;
39  import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
40  import org.apache.hadoop.hbase.io.hfile.HFileBlock.BlockWritable;
41  import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
42  import org.apache.hadoop.hbase.util.ChecksumType;
43  import org.apache.hadoop.hbase.util.BloomFilterWriter;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.io.Writable;
46  import org.apache.hadoop.io.WritableUtils;
47  
48  /**
49   * Writes HFile format version 2.
50   */
51  public class HFileWriterV2 extends AbstractHFileWriter {
52    static final Log LOG = LogFactory.getLog(HFileWriterV2.class);
53  
54    /** Max memstore (mvcc) timestamp in FileInfo */
55    public static final byte [] MAX_MEMSTORE_TS_KEY =
56        Bytes.toBytes("MAX_MEMSTORE_TS_KEY");
57  
58    /** KeyValue version in FileInfo */
59    public static final byte [] KEY_VALUE_VERSION =
60        Bytes.toBytes("KEY_VALUE_VERSION");
61  
62    /** Version for KeyValue which includes memstore timestamp */
63    public static final int KEY_VALUE_VER_WITH_MEMSTORE = 1;
64  
65    /** Inline block writers for multi-level block index and compound Blooms. */
66    private List<InlineBlockWriter> inlineBlockWriters =
67        new ArrayList<InlineBlockWriter>();
68  
69    /** Unified version 2 block writer */
70    private HFileBlock.Writer fsBlockWriter;
71  
72    private HFileBlockIndex.BlockIndexWriter dataBlockIndexWriter;
73    private HFileBlockIndex.BlockIndexWriter metaBlockIndexWriter;
74  
75    /** The offset of the first data block or -1 if the file is empty. */
76    private long firstDataBlockOffset = -1;
77  
78    /** The offset of the last data block or 0 if the file is empty. */
79    private long lastDataBlockOffset;
80  
81    /** Additional data items to be written to the "load-on-open" section. */
82    private List<BlockWritable> additionalLoadOnOpenData =
83      new ArrayList<BlockWritable>();
84  
85    /** Checksum related settings */
86    private ChecksumType checksumType = HFile.DEFAULT_CHECKSUM_TYPE;
87    private int bytesPerChecksum = HFile.DEFAULT_BYTES_PER_CHECKSUM;
88  
89    private final boolean includeMemstoreTS = true;
90    private long maxMemstoreTS = 0;
91  
92    private int minorVersion = HFileReaderV2.MAX_MINOR_VERSION;
93  
94    static class WriterFactoryV2 extends HFile.WriterFactory {
95      WriterFactoryV2(Configuration conf, CacheConfig cacheConf) {
96        super(conf, cacheConf);
97      }
98  
99      @Override
100     public Writer createWriter(FileSystem fs, Path path,
101         FSDataOutputStream ostream, int blockSize,
102         Compression.Algorithm compress, HFileDataBlockEncoder blockEncoder,
103         final KeyComparator comparator, final ChecksumType checksumType,
104         final int bytesPerChecksum) throws IOException {
105       return new HFileWriterV2(conf, cacheConf, fs, path, ostream, blockSize,
106           compress, blockEncoder, comparator, checksumType, bytesPerChecksum);
107     }
108   }
109 
110   /** Constructor that takes a path, creates and closes the output stream. */
111   public HFileWriterV2(Configuration conf, CacheConfig cacheConf,
112       FileSystem fs, Path path, FSDataOutputStream ostream, int blockSize,
113       Compression.Algorithm compressAlgo, HFileDataBlockEncoder blockEncoder,
114       final KeyComparator comparator, final ChecksumType checksumType,
115       final int bytesPerChecksum) throws IOException {
116     super(cacheConf,
117         ostream == null ? createOutputStream(conf, fs, path) : ostream,
118         path, blockSize, compressAlgo, blockEncoder, comparator);
119     SchemaMetrics.configureGlobally(conf);
120     this.checksumType = checksumType;
121     this.bytesPerChecksum = bytesPerChecksum;
122     if (!conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, false)) {
123       this.minorVersion = 0;
124     }
125     finishInit(conf);
126   }
127 
128   /** Additional initialization steps */
129   private void finishInit(final Configuration conf) {
130     if (fsBlockWriter != null)
131       throw new IllegalStateException("finishInit called twice");
132 
133     // HFile filesystem-level (non-caching) block writer
134     fsBlockWriter = new HFileBlock.Writer(compressAlgo, blockEncoder,
135         includeMemstoreTS, minorVersion, checksumType, bytesPerChecksum);
136 
137     // Data block index writer
138     boolean cacheIndexesOnWrite = cacheConf.shouldCacheIndexesOnWrite();
139     dataBlockIndexWriter = new HFileBlockIndex.BlockIndexWriter(fsBlockWriter,
140         cacheIndexesOnWrite ? cacheConf.getBlockCache(): null,
141         cacheIndexesOnWrite ? name : null);
142     dataBlockIndexWriter.setMaxChunkSize(
143         HFileBlockIndex.getMaxChunkSize(conf));
144     inlineBlockWriters.add(dataBlockIndexWriter);
145 
146     // Meta data block index writer
147     metaBlockIndexWriter = new HFileBlockIndex.BlockIndexWriter();
148     LOG.debug("Initialized with " + cacheConf);
149 
150     if (isSchemaConfigured()) {
151       schemaConfigurationChanged();
152     }
153   }
154 
155   @Override
156   protected void schemaConfigurationChanged() {
157     passSchemaMetricsTo(dataBlockIndexWriter);
158     passSchemaMetricsTo(metaBlockIndexWriter);
159   }
160 
161   /**
162    * At a block boundary, write all the inline blocks and opens new block.
163    *
164    * @throws IOException
165    */
166   private void checkBlockBoundary() throws IOException {
167     if (fsBlockWriter.blockSizeWritten() < blockSize)
168       return;
169 
170     finishBlock();
171     writeInlineBlocks(false);
172     newBlock();
173   }
174 
175   /** Clean up the current block */
176   private void finishBlock() throws IOException {
177     if (!fsBlockWriter.isWriting() || fsBlockWriter.blockSizeWritten() == 0)
178       return;
179 
180     long startTimeNs = System.nanoTime();
181 
182     // Update the first data block offset for scanning.
183     if (firstDataBlockOffset == -1) {
184       firstDataBlockOffset = outputStream.getPos();
185     }
186 
187     // Update the last data block offset
188     lastDataBlockOffset = outputStream.getPos();
189 
190     fsBlockWriter.writeHeaderAndData(outputStream);
191 
192     int onDiskSize = fsBlockWriter.getOnDiskSizeWithHeader();
193     dataBlockIndexWriter.addEntry(firstKeyInBlock, lastDataBlockOffset,
194         onDiskSize);
195     totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
196 
197     HFile.offerWriteLatency(System.nanoTime() - startTimeNs);
198     
199     if (cacheConf.shouldCacheDataOnWrite()) {
200       doCacheOnWrite(lastDataBlockOffset);
201     }
202   }
203 
204   /** Gives inline block writers an opportunity to contribute blocks. */
205   private void writeInlineBlocks(boolean closing) throws IOException {
206     for (InlineBlockWriter ibw : inlineBlockWriters) {
207       while (ibw.shouldWriteBlock(closing)) {
208         long offset = outputStream.getPos();
209         boolean cacheThisBlock = ibw.cacheOnWrite();
210         ibw.writeInlineBlock(fsBlockWriter.startWriting(
211             ibw.getInlineBlockType()));
212         fsBlockWriter.writeHeaderAndData(outputStream);
213         ibw.blockWritten(offset, fsBlockWriter.getOnDiskSizeWithHeader(),
214             fsBlockWriter.getUncompressedSizeWithoutHeader());
215         totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
216 
217         if (cacheThisBlock) {
218           doCacheOnWrite(offset);
219         }
220       }
221     }
222   }
223 
224   /**
225    * Caches the last written HFile block.
226    * @param offset the offset of the block we want to cache. Used to determine
227    *          the cache key.
228    */
229   private void doCacheOnWrite(long offset) {
230     // We don't cache-on-write data blocks on compaction, so assume this is not
231     // a compaction.
232     final boolean isCompaction = false;
233     HFileBlock cacheFormatBlock = blockEncoder.diskToCacheFormat(
234         fsBlockWriter.getBlockForCaching(), isCompaction);
235     passSchemaMetricsTo(cacheFormatBlock);
236     cacheConf.getBlockCache().cacheBlock(
237         new BlockCacheKey(name, offset, blockEncoder.getEncodingInCache(),
238             cacheFormatBlock.getBlockType()), cacheFormatBlock);
239   }
240 
241   /**
242    * Ready a new block for writing.
243    *
244    * @throws IOException
245    */
246   private void newBlock() throws IOException {
247     // This is where the next block begins.
248     fsBlockWriter.startWriting(BlockType.DATA);
249     firstKeyInBlock = null;
250   }
251 
252   /**
253    * Add a meta block to the end of the file. Call before close(). Metadata
254    * blocks are expensive. Fill one with a bunch of serialized data rather than
255    * do a metadata block per metadata instance. If metadata is small, consider
256    * adding to file info using {@link #appendFileInfo(byte[], byte[])}
257    *
258    * @param metaBlockName
259    *          name of the block
260    * @param content
261    *          will call readFields to get data later (DO NOT REUSE)
262    */
263   @Override
264   public void appendMetaBlock(String metaBlockName, Writable content) {
265     byte[] key = Bytes.toBytes(metaBlockName);
266     int i;
267     for (i = 0; i < metaNames.size(); ++i) {
268       // stop when the current key is greater than our own
269       byte[] cur = metaNames.get(i);
270       if (Bytes.BYTES_RAWCOMPARATOR.compare(cur, 0, cur.length, key, 0,
271           key.length) > 0) {
272         break;
273       }
274     }
275     metaNames.add(i, key);
276     metaData.add(i, content);
277   }
278 
279   /**
280    * Add key/value to file. Keys must be added in an order that agrees with the
281    * Comparator passed on construction.
282    *
283    * @param kv
284    *          KeyValue to add. Cannot be empty nor null.
285    * @throws IOException
286    */
287   @Override
288   public void append(final KeyValue kv) throws IOException {
289     append(kv.getMemstoreTS(), kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(),
290         kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
291     this.maxMemstoreTS = Math.max(this.maxMemstoreTS, kv.getMemstoreTS());
292   }
293 
294   /**
295    * Add key/value to file. Keys must be added in an order that agrees with the
296    * Comparator passed on construction.
297    *
298    * @param key
299    *          Key to add. Cannot be empty nor null.
300    * @param value
301    *          Value to add. Cannot be empty nor null.
302    * @throws IOException
303    */
304   @Override
305   public void append(final byte[] key, final byte[] value) throws IOException {
306     append(0, key, 0, key.length, value, 0, value.length);
307   }
308 
309   /**
310    * Add key/value to file. Keys must be added in an order that agrees with the
311    * Comparator passed on construction.
312    *
313    * @param key
314    * @param koffset
315    * @param klength
316    * @param value
317    * @param voffset
318    * @param vlength
319    * @throws IOException
320    */
321   private void append(final long memstoreTS, final byte[] key, final int koffset, final int klength,
322       final byte[] value, final int voffset, final int vlength)
323       throws IOException {
324     boolean dupKey = checkKey(key, koffset, klength);
325     checkValue(value, voffset, vlength);
326     if (!dupKey) {
327       checkBlockBoundary();
328     }
329 
330     if (!fsBlockWriter.isWriting())
331       newBlock();
332 
333     // Write length of key and value and then actual key and value bytes.
334     // Additionally, we may also write down the memstoreTS.
335     {
336       DataOutputStream out = fsBlockWriter.getUserDataStream();
337       out.writeInt(klength);
338       totalKeyLength += klength;
339       out.writeInt(vlength);
340       totalValueLength += vlength;
341       out.write(key, koffset, klength);
342       out.write(value, voffset, vlength);
343       if (this.includeMemstoreTS) {
344         WritableUtils.writeVLong(out, memstoreTS);
345       }
346     }
347 
348     // Are we the first key in this block?
349     if (firstKeyInBlock == null) {
350       // Copy the key.
351       firstKeyInBlock = new byte[klength];
352       System.arraycopy(key, koffset, firstKeyInBlock, 0, klength);
353     }
354 
355     lastKeyBuffer = key;
356     lastKeyOffset = koffset;
357     lastKeyLength = klength;
358     entryCount++;
359   }
360 
361   @Override
362   public void close() throws IOException {
363     if (outputStream == null) {
364       return;
365     }
366     // Save data block encoder metadata in the file info.
367     blockEncoder.saveMetadata(this);
368     // Write out the end of the data blocks, then write meta data blocks.
369     // followed by fileinfo, data block index and meta block index.
370 
371     finishBlock();
372     writeInlineBlocks(true);
373 
374     FixedFileTrailer trailer = new FixedFileTrailer(2, minorVersion);
375 
376     // Write out the metadata blocks if any.
377     if (!metaNames.isEmpty()) {
378       for (int i = 0; i < metaNames.size(); ++i) {
379         // store the beginning offset
380         long offset = outputStream.getPos();
381         // write the metadata content
382         DataOutputStream dos = fsBlockWriter.startWriting(BlockType.META);
383         metaData.get(i).write(dos);
384 
385         fsBlockWriter.writeHeaderAndData(outputStream);
386         totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
387 
388         // Add the new meta block to the meta index.
389         metaBlockIndexWriter.addEntry(metaNames.get(i), offset,
390             fsBlockWriter.getOnDiskSizeWithHeader());
391       }
392     }
393 
394     // Load-on-open section.
395 
396     // Data block index.
397     //
398     // In version 2, this section of the file starts with the root level data
399     // block index. We call a function that writes intermediate-level blocks
400     // first, then root level, and returns the offset of the root level block
401     // index.
402 
403     long rootIndexOffset = dataBlockIndexWriter.writeIndexBlocks(outputStream);
404     trailer.setLoadOnOpenOffset(rootIndexOffset);
405 
406     // Meta block index.
407     metaBlockIndexWriter.writeSingleLevelIndex(fsBlockWriter.startWriting(
408         BlockType.ROOT_INDEX), "meta");
409     fsBlockWriter.writeHeaderAndData(outputStream);
410     totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
411 
412     if (this.includeMemstoreTS) {
413       appendFileInfo(MAX_MEMSTORE_TS_KEY, Bytes.toBytes(maxMemstoreTS));
414       appendFileInfo(KEY_VALUE_VERSION, Bytes.toBytes(KEY_VALUE_VER_WITH_MEMSTORE));
415     }
416 
417     // File info
418     writeFileInfo(trailer, fsBlockWriter.startWriting(BlockType.FILE_INFO));
419     fsBlockWriter.writeHeaderAndData(outputStream);
420     totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
421 
422     // Load-on-open data supplied by higher levels, e.g. Bloom filters.
423     for (BlockWritable w : additionalLoadOnOpenData){
424       fsBlockWriter.writeBlock(w, outputStream);
425       totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
426     }
427 
428     // Now finish off the trailer.
429     trailer.setNumDataIndexLevels(dataBlockIndexWriter.getNumLevels());
430     trailer.setUncompressedDataIndexSize(
431         dataBlockIndexWriter.getTotalUncompressedSize());
432     trailer.setFirstDataBlockOffset(firstDataBlockOffset);
433     trailer.setLastDataBlockOffset(lastDataBlockOffset);
434     trailer.setComparatorClass(comparator.getClass());
435     trailer.setDataIndexCount(dataBlockIndexWriter.getNumRootEntries());
436 
437 
438     finishClose(trailer);
439 
440     fsBlockWriter.releaseCompressor();
441   }
442 
443   @Override
444   public void addInlineBlockWriter(InlineBlockWriter ibw) {
445     inlineBlockWriters.add(ibw);
446   }
447 
448   @Override
449   public void addGeneralBloomFilter(final BloomFilterWriter bfw) {
450     this.addBloomFilter(bfw, BlockType.GENERAL_BLOOM_META);
451   }
452 
453   @Override
454   public void addDeleteFamilyBloomFilter(final BloomFilterWriter bfw) {
455     this.addBloomFilter(bfw, BlockType.DELETE_FAMILY_BLOOM_META);
456   }
457 
458   private void addBloomFilter(final BloomFilterWriter bfw,
459       final BlockType blockType) {
460     if (bfw.getKeyCount() <= 0)
461       return;
462 
463     if (blockType != BlockType.GENERAL_BLOOM_META &&
464         blockType != BlockType.DELETE_FAMILY_BLOOM_META) {
465       throw new RuntimeException("Block Type: " + blockType.toString() +
466           "is not supported");
467     }
468     additionalLoadOnOpenData.add(new BlockWritable() {
469       @Override
470       public BlockType getBlockType() {
471         return blockType;
472       }
473 
474       @Override
475       public void writeToBlock(DataOutput out) throws IOException {
476         bfw.getMetaWriter().write(out);
477         Writable dataWriter = bfw.getDataWriter();
478         if (dataWriter != null)
479           dataWriter.write(out);
480       }
481     });
482   }
483 
484 }