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.InterruptedIOException;
022import java.io.OutputStream;
023import java.nio.ByteBuffer;
024import java.util.concurrent.CompletableFuture;
025import java.util.concurrent.ExecutionException;
026import java.util.function.Consumer;
027import org.apache.hadoop.fs.FileSystem;
028import org.apache.hadoop.fs.Path;
029import org.apache.hadoop.hbase.Cell;
030import org.apache.hadoop.hbase.io.ByteBufferWriter;
031import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput;
032import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutputHelper;
033import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
034import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
035import org.apache.hadoop.hbase.wal.WAL.Entry;
036import org.apache.yetus.audience.InterfaceAudience;
037import org.slf4j.Logger;
038import org.slf4j.LoggerFactory;
039
040import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
041import org.apache.hbase.thirdparty.io.netty.channel.Channel;
042import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
043
044import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
045import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
046
047/**
048 * AsyncWriter for protobuf-based WAL.
049 */
050@InterfaceAudience.Private
051public class AsyncProtobufLogWriter extends AbstractProtobufLogWriter
052    implements AsyncFSWALProvider.AsyncWriter {
053
054  private static final Logger LOG = LoggerFactory.getLogger(AsyncProtobufLogWriter.class);
055
056  private final EventLoopGroup eventLoopGroup;
057
058  private final Class<? extends Channel> channelClass;
059
060  private AsyncFSOutput output;
061
062  private static final class OutputStreamWrapper extends OutputStream
063      implements ByteBufferWriter {
064
065    private final AsyncFSOutput out;
066
067    private final byte[] oneByteBuf = new byte[1];
068
069    @Override
070    public void write(int b) throws IOException {
071      oneByteBuf[0] = (byte) b;
072      write(oneByteBuf);
073    }
074
075    public OutputStreamWrapper(AsyncFSOutput out) {
076      this.out = out;
077    }
078
079    @Override
080    public void write(ByteBuffer b, int off, int len) throws IOException {
081      ByteBuffer bb = b.duplicate();
082      bb.position(off);
083      bb.limit(off + len);
084      out.write(bb);
085    }
086
087    @Override
088    public void writeInt(int i) throws IOException {
089      out.writeInt(i);
090    }
091
092    @Override
093    public void write(byte[] b, int off, int len) throws IOException {
094      out.write(b, off, len);
095    }
096
097    @Override
098    public void close() throws IOException {
099      out.close();
100    }
101  }
102
103  private OutputStream asyncOutputWrapper;
104
105  public AsyncProtobufLogWriter(EventLoopGroup eventLoopGroup,
106      Class<? extends Channel> channelClass) {
107    this.eventLoopGroup = eventLoopGroup;
108    this.channelClass = channelClass;
109  }
110
111  /*
112   * @return class name which is recognized by hbase-1.x to avoid ProtobufLogReader throwing error:
113   *   IOException: Got unknown writer class: AsyncProtobufLogWriter
114   */
115  @Override
116  protected String getWriterClassName() {
117    return "ProtobufLogWriter";
118  }
119
120  @Override
121  public void append(Entry entry) {
122    int buffered = output.buffered();
123    try {
124      entry.getKey().
125        getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
126          .writeDelimitedTo(asyncOutputWrapper);
127    } catch (IOException e) {
128      throw new AssertionError("should not happen", e);
129    }
130    try {
131      for (Cell cell : entry.getEdit().getCells()) {
132        cellEncoder.write(cell);
133      }
134    } catch (IOException e) {
135      throw new AssertionError("should not happen", e);
136    }
137    length.addAndGet(output.buffered() - buffered);
138  }
139
140  @Override
141  public CompletableFuture<Long> sync() {
142    return output.flush(false);
143  }
144
145  @Override
146  public synchronized void close() throws IOException {
147    if (this.output == null) {
148      return;
149    }
150    try {
151      writeWALTrailer();
152      output.close();
153    } catch (Exception e) {
154      LOG.warn("normal close failed, try recover", e);
155      output.recoverAndClose(null);
156    }
157    this.output = null;
158  }
159
160  public AsyncFSOutput getOutput() {
161    return this.output;
162  }
163
164  @Override
165  protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
166      short replication, long blockSize) throws IOException, StreamLacksCapabilityException {
167    this.output = AsyncFSOutputHelper.createOutput(fs, path, overwritable, false, replication,
168        blockSize, eventLoopGroup, channelClass);
169    this.asyncOutputWrapper = new OutputStreamWrapper(output);
170  }
171
172  private long write(Consumer<CompletableFuture<Long>> action) throws IOException {
173    CompletableFuture<Long> future = new CompletableFuture<>();
174    action.accept(future);
175    try {
176      return future.get().longValue();
177    } catch (InterruptedException e) {
178      InterruptedIOException ioe = new InterruptedIOException();
179      ioe.initCause(e);
180      throw ioe;
181    } catch (ExecutionException e) {
182      Throwables.propagateIfPossible(e.getCause(), IOException.class);
183      throw new RuntimeException(e.getCause());
184    }
185  }
186
187  @Override
188  protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException {
189    return write(future -> {
190      output.write(magic);
191      try {
192        header.writeDelimitedTo(asyncOutputWrapper);
193      } catch (IOException e) {
194        // should not happen
195        throw new AssertionError(e);
196      }
197      output.flush(false).whenComplete((len, error) -> {
198        if (error != null) {
199          future.completeExceptionally(error);
200        } else {
201          future.complete(len);
202        }
203      });
204    });
205  }
206
207  @Override
208  protected long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic) throws IOException {
209    return write(future -> {
210      try {
211        trailer.writeTo(asyncOutputWrapper);
212      } catch (IOException e) {
213        // should not happen
214        throw new AssertionError(e);
215      }
216      output.writeInt(trailer.getSerializedSize());
217      output.write(magic);
218      output.flush(false).whenComplete((len, error) -> {
219        if (error != null) {
220          future.completeExceptionally(error);
221        } else {
222          future.complete(len);
223        }
224      });
225    });
226  }
227
228  @Override
229  protected OutputStream getOutputStreamForCellEncoder() {
230    return asyncOutputWrapper;
231  }
232}