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