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.hbase.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.HBaseInterfaceAudience;
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.protobuf.generated.WALProtos.WALTrailer;
36  import org.apache.hadoop.hbase.util.FSUtils;
37  
38  /**
39   * Writer for protobuf-based WAL.
40   */
41  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
42  public class ProtobufLogWriter extends WriterBase {
43    private final Log LOG = LogFactory.getLog(this.getClass());
44    protected FSDataOutputStream output;
45    protected Codec.Encoder cellEncoder;
46    protected 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    protected WALCellCodec getCodec(Configuration conf, CompressionContext compressionContext)
58        throws IOException {
59      return WALCellCodec.create(conf, null, compressionContext);
60    }
61  
62    protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder)
63        throws IOException {
64      if (!builder.hasWriterClsName()) {
65        builder.setWriterClsName(ProtobufLogWriter.class.getSimpleName());
66      }
67      if (!builder.hasCellCodecClsName()) {
68        builder.setCellCodecClsName(WALCellCodec.getWALCellCodecClass(conf));
69      }
70      return builder.build();
71    }
72  
73    @Override
74    @SuppressWarnings("deprecation")
75    public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable) throws IOException {
76      super.init(fs, path, conf, overwritable);
77      assert this.output == null;
78      boolean doCompress = initializeCompressionContext(conf, path);
79      this.trailerWarnSize = conf.getInt(HLog.WAL_TRAILER_WARN_SIZE,
80        HLog.DEFAULT_WAL_TRAILER_WARN_SIZE);
81      int bufferSize = FSUtils.getDefaultBufferSize(fs);
82      short replication = (short)conf.getInt(
83          "hbase.regionserver.hlog.replication", FSUtils.getDefaultReplication(fs, path));
84      long blockSize = conf.getLong("hbase.regionserver.hlog.blocksize",
85          FSUtils.getDefaultBlockSize(fs, path));
86      output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize, null);
87      output.write(ProtobufLogReader.PB_WAL_MAGIC);
88      boolean doTagCompress = doCompress
89          && conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true);
90      buildWALHeader(conf,
91          WALHeader.newBuilder().setHasCompression(doCompress).setHasTagCompression(doTagCompress))
92          .writeDelimitedTo(output);
93  
94      initAfterHeader(doCompress);
95  
96      // instantiate trailer to default value.
97      trailer = WALTrailer.newBuilder().build();
98      if (LOG.isTraceEnabled()) {
99        LOG.trace("Initialized protobuf WAL=" + path + ", compression=" + doCompress);
100     }
101   }
102 
103   protected void initAfterHeader(boolean doCompress) throws IOException {
104     WALCellCodec codec = getCodec(conf, this.compressionContext);
105     this.cellEncoder = codec.getEncoder(this.output);
106     if (doCompress) {
107       this.compressor = codec.getByteStringCompressor();
108     }
109   }
110 
111   @Override
112   public void append(HLog.Entry entry) throws IOException {
113     entry.setCompressionContext(compressionContext);
114     entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size())
115       .build().writeDelimitedTo(output);
116     for (KeyValue kv : entry.getEdit().getKeyValues()) {
117       // cellEncoder must assume little about the stream, since we write PB and cells in turn.
118       cellEncoder.write(kv);
119     }
120   }
121 
122   @Override
123   public void close() throws IOException {
124     if (this.output != null) {
125       try {
126         if (!trailerWritten) writeWALTrailer();
127         this.output.close();
128       } catch (NullPointerException npe) {
129         // Can get a NPE coming up from down in DFSClient$DFSOutputStream#close
130         LOG.warn(npe);
131       }
132       this.output = null;
133     }
134   }
135 
136   protected WALTrailer buildWALTrailer(WALTrailer.Builder builder) {
137     return builder.build();
138   }
139 
140   private void writeWALTrailer() {
141     try {
142       int trailerSize = 0;
143       if (this.trailer == null) {
144         // use default trailer.
145         LOG.warn("WALTrailer is null. Continuing with default.");
146         this.trailer = buildWALTrailer(WALTrailer.newBuilder());
147         trailerSize = this.trailer.getSerializedSize();
148       } else if ((trailerSize = this.trailer.getSerializedSize()) > this.trailerWarnSize) {
149         // continue writing after warning the user.
150         LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum size : " +
151           trailerSize + " > " + this.trailerWarnSize);
152       }
153       this.trailer.writeTo(output);
154       output.writeInt(trailerSize);
155       output.write(ProtobufLogReader.PB_WAL_COMPLETE_MAGIC);
156       this.trailerWritten = true;
157     } catch (IOException ioe) {
158       LOG.error("Got IOException while writing trailer", ioe);
159     }
160   }
161 
162   @Override
163   public void sync() throws IOException {
164     try {
165       this.output.flush();
166       this.output.sync();
167     } catch (NullPointerException npe) {
168       // Concurrent close...
169       throw new IOException(npe);
170     }
171   }
172 
173   @Override
174   public long getLength() throws IOException {
175     try {
176       return this.output.getPos();
177     } catch (NullPointerException npe) {
178       // Concurrent close...
179       throw new IOException(npe);
180     }
181   }
182 
183   public FSDataOutputStream getStream() {
184     return this.output;
185   }
186 
187   @Override
188   public void setWALTrailer(WALTrailer walTrailer) {
189     this.trailer = walTrailer;
190   }
191 }