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 java.util.concurrent.atomic.AtomicLong;
023
024import org.apache.hadoop.fs.FSDataOutputStream;
025import org.apache.hadoop.fs.FileSystem;
026import org.apache.hadoop.fs.Path;
027import org.apache.hadoop.fs.StreamCapabilities;
028import org.apache.hadoop.hbase.Cell;
029import org.apache.hadoop.hbase.util.AtomicUtils;
030import org.apache.hadoop.hbase.util.CommonFSUtils;
031import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
032import org.apache.hadoop.hbase.wal.FSHLogProvider;
033import org.apache.hadoop.hbase.wal.WAL.Entry;
034import org.apache.yetus.audience.InterfaceAudience;
035import org.slf4j.Logger;
036import org.slf4j.LoggerFactory;
037
038import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
039import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
040
041/**
042 * Writer for protobuf-based WAL.
043 */
044@InterfaceAudience.Private
045public class ProtobufLogWriter extends AbstractProtobufLogWriter
046    implements FSHLogProvider.Writer {
047
048  private static final Logger LOG = LoggerFactory.getLogger(ProtobufLogWriter.class);
049
050  protected FSDataOutputStream output;
051
052  private final AtomicLong syncedLength = new AtomicLong(0);
053
054  @Override
055  public void append(Entry entry) throws IOException {
056    entry.getKey().getBuilder(compressor).
057        setFollowingKvCount(entry.getEdit().size()).build().writeDelimitedTo(output);
058    for (Cell cell : entry.getEdit().getCells()) {
059      // cellEncoder must assume little about the stream, since we write PB and cells in turn.
060      cellEncoder.write(cell);
061    }
062    length.set(output.getPos());
063  }
064
065  @Override
066  public void close() throws IOException {
067    if (this.output != null) {
068      try {
069        if (!trailerWritten) {
070          writeWALTrailer();
071        }
072        this.output.close();
073      } catch (NullPointerException npe) {
074        // Can get a NPE coming up from down in DFSClient$DFSOutputStream#close
075        LOG.warn(npe.toString(), npe);
076      }
077      this.output = null;
078    }
079  }
080
081  @Override
082  public void sync(boolean forceSync) throws IOException {
083    FSDataOutputStream fsdos = this.output;
084    if (fsdos == null) {
085      return; // Presume closed
086    }
087    fsdos.flush();
088    if (forceSync) {
089      fsdos.hsync();
090    } else {
091      fsdos.hflush();
092    }
093    AtomicUtils.updateMax(this.syncedLength, fsdos.getPos());
094  }
095
096  @Override
097  public long getSyncedLength() {
098    return this.syncedLength.get();
099  }
100
101  public FSDataOutputStream getStream() {
102    return this.output;
103  }
104
105  @Override
106  protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
107      short replication, long blockSize) throws IOException, StreamLacksCapabilityException {
108    this.output = CommonFSUtils.createForWal(fs, path, overwritable, bufferSize, replication,
109        blockSize, false);
110    if (fs.getConf().getBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, true)) {
111      if (!output.hasCapability(StreamCapabilities.HFLUSH)) {
112        throw new StreamLacksCapabilityException(StreamCapabilities.HFLUSH);
113      }
114      if (!output.hasCapability(StreamCapabilities.HSYNC)) {
115        throw new StreamLacksCapabilityException(StreamCapabilities.HSYNC);
116      }
117    }
118  }
119
120  @Override
121  protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException {
122    output.write(magic);
123    header.writeDelimitedTo(output);
124    return output.getPos();
125  }
126
127  @Override
128  protected OutputStream getOutputStreamForCellEncoder() {
129    return this.output;
130  }
131
132  @Override
133  protected long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic) throws IOException {
134    trailer.writeTo(output);
135    output.writeInt(trailer.getSerializedSize());
136    output.write(magic);
137    return output.getPos();
138  }
139}