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