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  
20  package org.apache.hadoop.hbase.regionserver.wal;
21  
22  import java.io.IOException;
23  
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.classification.InterfaceAudience;
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.fs.FSDataOutputStream;
29  import org.apache.hadoop.fs.FileSystem;
30  import org.apache.hadoop.fs.Path;
31  import org.apache.hadoop.hbase.HConstants;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.codec.Codec;
34  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
35  import org.apache.hadoop.hbase.util.FSUtils;
36  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
37  
38  /**
39   * Writer for protobuf-based WAL.
40   */
41  @InterfaceAudience.Private
42  public class ProtobufLogWriter extends WriterBase {
43    private final Log LOG = LogFactory.getLog(this.getClass());
44    private FSDataOutputStream output;
45    private Codec.Encoder cellEncoder;
46    private WALCellCodec.ByteStringCompressor compressor;
47    private boolean trailerWritten;
48    private WALTrailer trailer;
49    // maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
50    // than this size, it is written/read respectively, with a WARN message in the log.
51    private int trailerWarnSize;
52  
53    public ProtobufLogWriter() {
54      super();
55    }
56  
57    @Override
58    public void init(FileSystem fs, Path path, Configuration conf) throws IOException {
59      assert this.output == null;
60      boolean doCompress = initializeCompressionContext(conf, path);
61      this.trailerWarnSize = conf.getInt(HLog.WAL_TRAILER_WARN_SIZE,
62        HLog.DEFAULT_WAL_TRAILER_WARN_SIZE);
63      int bufferSize = FSUtils.getDefaultBufferSize(fs);
64      short replication = (short)conf.getInt(
65          "hbase.regionserver.hlog.replication", FSUtils.getDefaultReplication(fs, path));
66      long blockSize = conf.getLong("hbase.regionserver.hlog.blocksize",
67          FSUtils.getDefaultBlockSize(fs, path));
68      output = fs.create(path, true, bufferSize, replication, blockSize);
69      output.write(ProtobufLogReader.PB_WAL_MAGIC);
70      WALHeader.newBuilder().setHasCompression(doCompress).build().writeDelimitedTo(output);
71  
72      WALCellCodec codec = WALCellCodec.create(conf, this.compressionContext);
73      this.cellEncoder = codec.getEncoder(this.output);
74      if (doCompress) {
75        this.compressor = codec.getByteStringCompressor();
76      }
77      // instantiate trailer to default value.
78      trailer = WALTrailer.newBuilder().build();
79      if (LOG.isTraceEnabled()) {
80        LOG.trace("Initialized protobuf WAL=" + path + ", compression=" + doCompress);
81      }
82    }
83  
84    @Override
85    public void append(HLog.Entry entry) throws IOException {
86      entry.setCompressionContext(compressionContext);
87      entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size())
88        .build().writeDelimitedTo(output);
89      for (KeyValue kv : entry.getEdit().getKeyValues()) {
90        // cellEncoder must assume little about the stream, since we write PB and cells in turn.
91        cellEncoder.write(kv);
92      }
93    }
94  
95    @Override
96    public void close() throws IOException {
97      if (this.output != null) {
98        try {
99          if (!trailerWritten) writeWALTrailer();
100         this.output.close();
101       } catch (NullPointerException npe) {
102         // Can get a NPE coming up from down in DFSClient$DFSOutputStream#close
103         LOG.warn(npe);
104       }
105       this.output = null;
106     }
107   }
108 
109   private void writeWALTrailer() {
110     try {
111       int trailerSize = 0;
112       if (this.trailer == null) {
113         // use default trailer.
114         LOG.warn("WALTrailer is null. Continuing with default.");
115         this.trailer = WALTrailer.newBuilder().build();
116         trailerSize = this.trailer.getSerializedSize();
117       } else if ((trailerSize = this.trailer.getSerializedSize()) > this.trailerWarnSize) {
118         // continue writing after warning the user.
119         LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum size : " +
120           trailerSize + " > " + this.trailerWarnSize);
121       }
122       this.trailer.writeTo(output);
123       this.output.writeInt(trailerSize);
124       this.output.write(ProtobufLogReader.PB_WAL_COMPLETE_MAGIC);
125       this.trailerWritten = true;
126     } catch (IOException ioe) {
127       LOG.error("Got IOException while writing trailer", ioe);
128     }
129   }
130 
131   @Override
132   public void sync() throws IOException {
133     try {
134       this.output.flush();
135       this.output.sync();
136     } catch (NullPointerException npe) {
137       // Concurrent close...
138       throw new IOException(npe);
139     }
140   }
141 
142   @Override
143   public long getLength() throws IOException {
144     try {
145       return this.output.getPos();
146     } catch (NullPointerException npe) {
147       // Concurrent close...
148       throw new IOException(npe);
149     }
150   }
151 
152   public FSDataOutputStream getStream() {
153     return this.output;
154   }
155 
156   @Override
157   public void setWALTrailer(WALTrailer walTrailer) {
158     this.trailer = walTrailer;
159   }
160 }