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.fs.StreamCapabilities; 026import org.apache.hadoop.hbase.Cell; 027import org.apache.hadoop.hbase.util.CommonFSUtils; 028import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException; 029import org.apache.hadoop.hbase.wal.FSHLogProvider; 030import org.apache.hadoop.hbase.wal.WAL.Entry; 031import org.apache.yetus.audience.InterfaceAudience; 032import org.slf4j.Logger; 033import org.slf4j.LoggerFactory; 034 035import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader; 036import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer; 037 038/** 039 * Writer for protobuf-based WAL. 040 */ 041@InterfaceAudience.Private 042public class ProtobufLogWriter extends AbstractProtobufLogWriter 043 implements FSHLogProvider.Writer { 044 045 private static final Logger LOG = LoggerFactory.getLogger(ProtobufLogWriter.class); 046 047 protected FSDataOutputStream output; 048 049 @Override 050 public void append(Entry entry) throws IOException { 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(boolean forceSync) throws IOException { 078 FSDataOutputStream fsdos = this.output; 079 if (fsdos == null) { 080 return; // Presume closed 081 } 082 fsdos.flush(); 083 if (forceSync) { 084 fsdos.hsync(); 085 } else { 086 fsdos.hflush(); 087 } 088 } 089 090 public FSDataOutputStream getStream() { 091 return this.output; 092 } 093 094 @Override 095 protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize, 096 short replication, long blockSize) throws IOException, StreamLacksCapabilityException { 097 this.output = CommonFSUtils.createForWal(fs, path, overwritable, bufferSize, replication, 098 blockSize, false); 099 if (fs.getConf().getBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, true)) { 100 if (!output.hasCapability(StreamCapabilities.HFLUSH)) { 101 throw new StreamLacksCapabilityException(StreamCapabilities.HFLUSH); 102 } 103 if (!output.hasCapability(StreamCapabilities.HSYNC)) { 104 throw new StreamLacksCapabilityException(StreamCapabilities.HSYNC); 105 } 106 } 107 } 108 109 @Override 110 protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException { 111 output.write(magic); 112 header.writeDelimitedTo(output); 113 return output.getPos(); 114 } 115 116 @Override 117 protected OutputStream getOutputStreamForCellEncoder() { 118 return this.output; 119 } 120 121 @Override 122 protected long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic) throws IOException { 123 trailer.writeTo(output); 124 output.writeInt(trailer.getSerializedSize()); 125 output.write(magic); 126 return output.getPos(); 127 } 128}