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