001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.regionserver.wal;
019
020import java.io.IOException;
021import java.io.OutputStream;
022import org.apache.hadoop.fs.FSDataOutputStream;
023import org.apache.hadoop.fs.FileSystem;
024import org.apache.hadoop.fs.Path;
025import org.apache.hadoop.hbase.Cell;
026import org.apache.hadoop.hbase.util.CommonFSUtils;
027import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
028import org.apache.hadoop.hbase.wal.FSHLogProvider;
029import org.apache.hadoop.hbase.wal.WAL.Entry;
030import org.apache.yetus.audience.InterfaceAudience;
031import org.slf4j.Logger;
032import org.slf4j.LoggerFactory;
033
034import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
035import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
036
037/**
038 * Writer for protobuf-based WAL.
039 */
040@InterfaceAudience.Private
041public class ProtobufLogWriter extends AbstractProtobufLogWriter
042    implements FSHLogProvider.Writer {
043
044  private static final Logger LOG = LoggerFactory.getLogger(ProtobufLogWriter.class);
045
046  protected FSDataOutputStream output;
047
048  @Override
049  public void append(Entry entry) throws IOException {
050    entry.setCompressionContext(compressionContext);
051    entry.getKey().getBuilder(compressor).
052        setFollowingKvCount(entry.getEdit().size()).build().writeDelimitedTo(output);
053    for (Cell cell : entry.getEdit().getCells()) {
054      // cellEncoder must assume little about the stream, since we write PB and cells in turn.
055      cellEncoder.write(cell);
056    }
057    length.set(output.getPos());
058  }
059
060  @Override
061  public void close() throws IOException {
062    if (this.output != null) {
063      try {
064        if (!trailerWritten) {
065          writeWALTrailer();
066        }
067        this.output.close();
068      } catch (NullPointerException npe) {
069        // Can get a NPE coming up from down in DFSClient$DFSOutputStream#close
070        LOG.warn(npe.toString(), npe);
071      }
072      this.output = null;
073    }
074  }
075
076  @Override
077  public void sync() throws IOException {
078    FSDataOutputStream fsdos = this.output;
079    if (fsdos == null) {
080      return; // Presume closed
081    }
082    fsdos.flush();
083    fsdos.hflush();
084  }
085
086  public FSDataOutputStream getStream() {
087    return this.output;
088  }
089
090  @SuppressWarnings("deprecation")
091  @Override
092  protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
093      short replication, long blockSize) throws IOException, StreamLacksCapabilityException {
094    this.output = CommonFSUtils.createForWal(fs, path, overwritable, bufferSize, replication,
095        blockSize, false);
096    // TODO Be sure to add a check for hsync if this branch includes HBASE-19024
097    if (fs.getConf().getBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, true) &&
098        !(CommonFSUtils.hasCapability(output, "hflush"))) {
099      throw new StreamLacksCapabilityException("hflush");
100    }
101  }
102
103  @Override
104  protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException {
105    output.write(magic);
106    header.writeDelimitedTo(output);
107    return output.getPos();
108  }
109
110  @Override
111  protected OutputStream getOutputStreamForCellEncoder() {
112    return this.output;
113  }
114
115  @Override
116  protected long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic) throws IOException {
117    trailer.writeTo(output);
118    output.writeInt(trailer.getSerializedSize());
119    output.write(magic);
120    return output.getPos();
121  }
122}