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  package org.apache.hadoop.hbase.io.hfile;
21  
22  import java.io.ByteArrayOutputStream;
23  import java.io.DataOutputStream;
24  import java.io.IOException;
25  import java.io.OutputStream;
26  import java.nio.ByteBuffer;
27  import java.util.ArrayList;
28  import java.util.List;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.fs.FSDataOutputStream;
34  import org.apache.hadoop.fs.FileSystem;
35  import org.apache.hadoop.fs.Path;
36  import org.apache.hadoop.hbase.KeyValue;
37  import org.apache.hadoop.hbase.KeyValue.KeyComparator;
38  import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
39  import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
40  import org.apache.hadoop.hbase.util.BloomFilterWriter;
41  import org.apache.hadoop.hbase.util.Bytes;
42  import org.apache.hadoop.io.Writable;
43  import org.apache.hadoop.io.compress.Compressor;
44  
45  /**
46   * Writes version 1 HFiles. Mainly used for testing backwards-compatibilty.
47   */
48  public class HFileWriterV1 extends AbstractHFileWriter {
49  
50    /** Meta data block name for bloom filter parameters. */
51    static final String BLOOM_FILTER_META_KEY = "BLOOM_FILTER_META";
52  
53    /** Meta data block name for bloom filter bits. */
54    public static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA";
55  
56    private static final Log LOG = LogFactory.getLog(HFileWriterV1.class);
57  
58    // A stream made per block written.
59    private DataOutputStream out;
60  
61    // Offset where the current block began.
62    private long blockBegin;
63  
64    // First keys of every block.
65    private ArrayList<byte[]> blockKeys = new ArrayList<byte[]>();
66  
67    // Block offset in backing stream.
68    private ArrayList<Long> blockOffsets = new ArrayList<Long>();
69  
70    // Raw (decompressed) data size.
71    private ArrayList<Integer> blockDataSizes = new ArrayList<Integer>();
72  
73    private Compressor compressor;
74  
75    // Additional byte array output stream used to fill block cache
76    private ByteArrayOutputStream baos;
77    private DataOutputStream baosDos;
78    private int blockNumber = 0;
79  
80    static class WriterFactoryV1 extends HFile.WriterFactory {
81  
82      WriterFactoryV1(Configuration conf, CacheConfig cacheConf) {
83        super(conf, cacheConf);
84      }
85  
86      @Override
87      public Writer createWriter(FileSystem fs, Path path) throws IOException {
88        return new HFileWriterV1(conf, cacheConf, fs, path);
89      }
90  
91      @Override
92      public Writer createWriter(FileSystem fs, Path path, int blockSize,
93          Compression.Algorithm compressAlgo, final KeyComparator comparator)
94          throws IOException {
95        return new HFileWriterV1(conf, cacheConf, fs, path, blockSize,
96            compressAlgo, comparator);
97      }
98  
99      @Override
100     public Writer createWriter(FileSystem fs, Path path, int blockSize,
101         String compressAlgoName,
102         final KeyComparator comparator) throws IOException {
103       return new HFileWriterV1(conf, cacheConf, fs, path, blockSize,
104           compressAlgoName, comparator);
105     }
106 
107     @Override
108     public Writer createWriter(final FSDataOutputStream ostream,
109         final int blockSize, final String compress,
110         final KeyComparator comparator) throws IOException {
111       return new HFileWriterV1(cacheConf, ostream, blockSize, compress,
112           comparator);
113     }
114 
115     @Override
116     public Writer createWriter(final FSDataOutputStream ostream,
117         final int blockSize, final Compression.Algorithm compress,
118         final KeyComparator c) throws IOException {
119       return new HFileWriterV1(cacheConf, ostream, blockSize, compress, c);
120     }
121   }
122 
123   /** Constructor that uses all defaults for compression and block size. */
124   public HFileWriterV1(Configuration conf, CacheConfig cacheConf,
125       FileSystem fs, Path path)
126       throws IOException {
127     this(conf, cacheConf, fs, path, HFile.DEFAULT_BLOCKSIZE,
128         HFile.DEFAULT_COMPRESSION_ALGORITHM,
129         null);
130   }
131 
132   /**
133    * Constructor that takes a path, creates and closes the output stream. Takes
134    * compression algorithm name as string.
135    */
136   public HFileWriterV1(Configuration conf, CacheConfig cacheConf, FileSystem fs,
137       Path path, int blockSize, String compressAlgoName,
138       final KeyComparator comparator) throws IOException {
139     this(conf, cacheConf, fs, path, blockSize,
140         compressionByName(compressAlgoName), comparator);
141   }
142 
143   /** Constructor that takes a path, creates and closes the output stream. */
144   public HFileWriterV1(Configuration conf, CacheConfig cacheConf, FileSystem fs,
145       Path path, int blockSize, Compression.Algorithm compress,
146       final KeyComparator comparator) throws IOException {
147     super(cacheConf, createOutputStream(conf, fs, path), path,
148         blockSize, compress, comparator);
149   }
150 
151   /** Constructor that takes a stream. */
152   public HFileWriterV1(CacheConfig cacheConf,
153       final FSDataOutputStream outputStream, final int blockSize,
154       final String compressAlgoName, final KeyComparator comparator)
155       throws IOException {
156     this(cacheConf, outputStream, blockSize,
157         Compression.getCompressionAlgorithmByName(compressAlgoName),
158         comparator);
159   }
160 
161   /** Constructor that takes a stream. */
162   public HFileWriterV1(CacheConfig cacheConf,
163       final FSDataOutputStream outputStream, final int blockSize,
164       final Compression.Algorithm compress, final KeyComparator comparator)
165       throws IOException {
166     super(cacheConf, outputStream, null, blockSize, compress, comparator);
167   }
168 
169   /**
170    * If at block boundary, opens new block.
171    *
172    * @throws IOException
173    */
174   private void checkBlockBoundary() throws IOException {
175     if (this.out != null && this.out.size() < blockSize)
176       return;
177     finishBlock();
178     newBlock();
179   }
180 
181   /**
182    * Do the cleanup if a current block.
183    *
184    * @throws IOException
185    */
186   private void finishBlock() throws IOException {
187     if (this.out == null)
188       return;
189     long startTimeNs = System.nanoTime();
190 
191     int size = releaseCompressingStream(this.out);
192     this.out = null;
193     blockKeys.add(firstKeyInBlock);
194     blockOffsets.add(Long.valueOf(blockBegin));
195     blockDataSizes.add(Integer.valueOf(size));
196     this.totalUncompressedBytes += size;
197 
198     HFile.writeTimeNano.addAndGet(System.nanoTime() - startTimeNs);
199     HFile.writeOps.incrementAndGet();
200 
201     if (cacheConf.shouldCacheDataOnWrite()) {
202       baosDos.flush();
203       byte[] bytes = baos.toByteArray();
204       cacheConf.getBlockCache().cacheBlock(
205           HFile.getBlockCacheKey(name, blockBegin),
206           new HFileBlock(BlockType.DATA,
207               (int) (outputStream.getPos() - blockBegin), bytes.length, -1,
208               ByteBuffer.wrap(bytes, 0, bytes.length), true, blockBegin));
209       baosDos.close();
210     }
211     blockNumber++;
212   }
213 
214   /**
215    * Ready a new block for writing.
216    *
217    * @throws IOException
218    */
219   private void newBlock() throws IOException {
220     // This is where the next block begins.
221     blockBegin = outputStream.getPos();
222     this.out = getCompressingStream();
223     BlockType.DATA.write(out);
224     firstKeyInBlock = null;
225     if (cacheConf.shouldCacheDataOnWrite()) {
226       this.baos = new ByteArrayOutputStream();
227       this.baosDos = new DataOutputStream(baos);
228       baosDos.write(HFileBlock.DUMMY_HEADER);
229     }
230   }
231 
232   /**
233    * Sets up a compressor and creates a compression stream on top of
234    * this.outputStream. Get one per block written.
235    *
236    * @return A compressing stream; if 'none' compression, returned stream does
237    * not compress.
238    *
239    * @throws IOException
240    *
241    * @see {@link #releaseCompressingStream(DataOutputStream)}
242    */
243   private DataOutputStream getCompressingStream() throws IOException {
244     this.compressor = compressAlgo.getCompressor();
245     // Get new DOS compression stream. In tfile, the DOS, is not closed,
246     // just finished, and that seems to be fine over there. TODO: Check
247     // no memory retention of the DOS. Should I disable the 'flush' on the
248     // DOS as the BCFile over in tfile does? It wants to make it so flushes
249     // don't go through to the underlying compressed stream. Flush on the
250     // compressed downstream should be only when done. I was going to but
251     // looks like when we call flush in here, its legitimate flush that
252     // should go through to the compressor.
253     OutputStream os = this.compressAlgo.createCompressionStream(
254         this.outputStream, this.compressor, 0);
255     return new DataOutputStream(os);
256   }
257 
258   /**
259    * Let go of block compressor and compressing stream gotten in call {@link
260    * #getCompressingStream}.
261    *
262    * @param dos
263    *
264    * @return How much was written on this stream since it was taken out.
265    *
266    * @see #getCompressingStream()
267    *
268    * @throws IOException
269    */
270   private int releaseCompressingStream(final DataOutputStream dos)
271       throws IOException {
272     dos.flush();
273     this.compressAlgo.returnCompressor(this.compressor);
274     this.compressor = null;
275     return dos.size();
276   }
277 
278   /**
279    * Add a meta block to the end of the file. Call before close(). Metadata
280    * blocks are expensive. Fill one with a bunch of serialized data rather than
281    * do a metadata block per metadata instance. If metadata is small, consider
282    * adding to file info using {@link #appendFileInfo(byte[], byte[])}
283    *
284    * @param metaBlockName
285    *          name of the block
286    * @param content
287    *          will call readFields to get data later (DO NOT REUSE)
288    */
289   public void appendMetaBlock(String metaBlockName, Writable content) {
290     byte[] key = Bytes.toBytes(metaBlockName);
291     int i;
292     for (i = 0; i < metaNames.size(); ++i) {
293       // stop when the current key is greater than our own
294       byte[] cur = metaNames.get(i);
295       if (Bytes.BYTES_RAWCOMPARATOR.compare(cur, 0, cur.length, key, 0,
296           key.length) > 0) {
297         break;
298       }
299     }
300     metaNames.add(i, key);
301     metaData.add(i, content);
302   }
303 
304   /**
305    * Add key/value to file. Keys must be added in an order that agrees with the
306    * Comparator passed on construction.
307    *
308    * @param kv
309    *          KeyValue to add. Cannot be empty nor null.
310    * @throws IOException
311    */
312   public void append(final KeyValue kv) throws IOException {
313     append(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(),
314         kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
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   public void append(final byte[] key, final byte[] value) throws IOException {
328     append(key, 0, key.length, value, 0, value.length);
329   }
330 
331   /**
332    * Add key/value to file. Keys must be added in an order that agrees with the
333    * Comparator passed on construction.
334    *
335    * @param key
336    * @param koffset
337    * @param klength
338    * @param value
339    * @param voffset
340    * @param vlength
341    * @throws IOException
342    */
343   private void append(final byte[] key, final int koffset, final int klength,
344       final byte[] value, final int voffset, final int vlength)
345       throws IOException {
346     boolean dupKey = checkKey(key, koffset, klength);
347     checkValue(value, voffset, vlength);
348     if (!dupKey) {
349       checkBlockBoundary();
350     }
351     // Write length of key and value and then actual key and value bytes.
352     this.out.writeInt(klength);
353     totalKeyLength += klength;
354     this.out.writeInt(vlength);
355     totalValueLength += vlength;
356     this.out.write(key, koffset, klength);
357     this.out.write(value, voffset, vlength);
358     // Are we the first key in this block?
359     if (this.firstKeyInBlock == null) {
360       // Copy the key.
361       this.firstKeyInBlock = new byte[klength];
362       System.arraycopy(key, koffset, this.firstKeyInBlock, 0, klength);
363     }
364     this.lastKeyBuffer = key;
365     this.lastKeyOffset = koffset;
366     this.lastKeyLength = klength;
367     this.entryCount++;
368     // If we are pre-caching blocks on write, fill byte array stream
369     if (cacheConf.shouldCacheDataOnWrite()) {
370       this.baosDos.writeInt(klength);
371       this.baosDos.writeInt(vlength);
372       this.baosDos.write(key, koffset, klength);
373       this.baosDos.write(value, voffset, vlength);
374     }
375   }
376 
377   public void close() throws IOException {
378     if (this.outputStream == null) {
379       return;
380     }
381     // Write out the end of the data blocks, then write meta data blocks.
382     // followed by fileinfo, data block index and meta block index.
383 
384     finishBlock();
385 
386     FixedFileTrailer trailer = new FixedFileTrailer(1);
387 
388     // Write out the metadata blocks if any.
389     ArrayList<Long> metaOffsets = null;
390     ArrayList<Integer> metaDataSizes = null;
391     if (metaNames.size() > 0) {
392       metaOffsets = new ArrayList<Long>(metaNames.size());
393       metaDataSizes = new ArrayList<Integer>(metaNames.size());
394       for (int i = 0; i < metaNames.size(); ++i) {
395         // store the beginning offset
396         long curPos = outputStream.getPos();
397         metaOffsets.add(curPos);
398         // write the metadata content
399         DataOutputStream dos = getCompressingStream();
400         BlockType.META.write(dos);
401         metaData.get(i).write(dos);
402         int size = releaseCompressingStream(dos);
403         // store the metadata size
404         metaDataSizes.add(size);
405       }
406     }
407 
408     writeFileInfo(trailer, outputStream);
409 
410     // Write the data block index.
411     trailer.setLoadOnOpenOffset(writeBlockIndex(this.outputStream,
412         this.blockKeys, this.blockOffsets, this.blockDataSizes));
413     LOG.info("Wrote a version 1 block index with " + this.blockKeys.size()
414         + " keys");
415 
416     if (metaNames.size() > 0) {
417       // Write the meta index.
418       writeBlockIndex(this.outputStream, metaNames, metaOffsets, metaDataSizes);
419     }
420 
421     // Now finish off the trailer.
422     trailer.setDataIndexCount(blockKeys.size());
423 
424     finishClose(trailer);
425   }
426 
427   @Override
428   protected void finishFileInfo() throws IOException {
429     super.finishFileInfo();
430 
431     // In version 1, we store comparator name in the file info.
432     fileInfo.append(FileInfo.COMPARATOR,
433         Bytes.toBytes(comparator.getClass().getName()), false);
434   }
435 
436   @Override
437   public void addInlineBlockWriter(InlineBlockWriter bloomWriter) {
438     // Inline blocks only exist in HFile format version 2.
439     throw new UnsupportedOperationException();
440   }
441 
442   /**
443    * Version 1 Bloom filters are stored in two meta blocks with two different
444    * keys.
445    */
446   @Override
447   public void addBloomFilter(BloomFilterWriter bfw) {
448     appendMetaBlock(BLOOM_FILTER_META_KEY,
449         bfw.getMetaWriter());
450     Writable dataWriter = bfw.getDataWriter();
451     if (dataWriter != null) {
452       appendMetaBlock(BLOOM_FILTER_DATA_KEY, dataWriter);
453     }
454   }
455 
456   /**
457    * Write out the index in the version 1 format. This conforms to the legacy
458    * version 1 format, but can still be read by
459    * {@link HFileBlockIndex.BlockIndexReader#readRootIndex(java.io.DataInputStream,
460    * int)}.
461    *
462    * @param out the stream to write to
463    * @param keys
464    * @param offsets
465    * @param uncompressedSizes in contrast with a version 2 root index format,
466    *          the sizes stored in the version 1 are uncompressed sizes
467    * @return
468    * @throws IOException
469    */
470   private static long writeBlockIndex(final FSDataOutputStream out,
471       final List<byte[]> keys, final List<Long> offsets,
472       final List<Integer> uncompressedSizes) throws IOException {
473     long pos = out.getPos();
474     // Don't write an index if nothing in the index.
475     if (keys.size() > 0) {
476       BlockType.INDEX_V1.write(out);
477       // Write the index.
478       for (int i = 0; i < keys.size(); ++i) {
479         out.writeLong(offsets.get(i).longValue());
480         out.writeInt(uncompressedSizes.get(i).intValue());
481         byte[] key = keys.get(i);
482         Bytes.writeByteArray(out, key);
483       }
484     }
485     return pos;
486   }
487 
488 }