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.io.asyncfs;
019
020import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.HEART_BEAT_SEQNO;
021import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.READ_TIMEOUT;
022import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.completeFile;
023import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.endFileLease;
024import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.getStatus;
025import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
026import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.READER_IDLE;
027import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.WRITER_IDLE;
028
029import java.io.IOException;
030import java.io.InterruptedIOException;
031import java.nio.ByteBuffer;
032import java.util.Collection;
033import java.util.Collections;
034import java.util.Iterator;
035import java.util.List;
036import java.util.Set;
037import java.util.concurrent.CompletableFuture;
038import java.util.concurrent.ConcurrentHashMap;
039import java.util.concurrent.ConcurrentLinkedDeque;
040import java.util.concurrent.ExecutionException;
041import java.util.concurrent.TimeUnit;
042import java.util.function.Supplier;
043import org.apache.hadoop.conf.Configuration;
044import org.apache.hadoop.crypto.Encryptor;
045import org.apache.hadoop.fs.Path;
046import org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.CancelOnClose;
047import org.apache.hadoop.hbase.util.CancelableProgressable;
048import org.apache.hadoop.hbase.util.FSUtils;
049import org.apache.hadoop.hdfs.DFSClient;
050import org.apache.hadoop.hdfs.DistributedFileSystem;
051import org.apache.hadoop.hdfs.protocol.ClientProtocol;
052import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
053import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
054import org.apache.hadoop.hdfs.protocol.LocatedBlock;
055import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
056import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
057import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;
058import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
059import org.apache.hadoop.util.DataChecksum;
060import org.apache.yetus.audience.InterfaceAudience;
061
062import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
063import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
064import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
065import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufAllocator;
066import org.apache.hbase.thirdparty.io.netty.channel.Channel;
067import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandler.Sharable;
068import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
069import org.apache.hbase.thirdparty.io.netty.channel.ChannelId;
070import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler;
071import org.apache.hbase.thirdparty.io.netty.handler.codec.protobuf.ProtobufDecoder;
072import org.apache.hbase.thirdparty.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
073import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateEvent;
074import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler;
075
076/**
077 * An asynchronous HDFS output stream implementation which fans out data to datanode and only
078 * supports writing file with only one block.
079 * <p>
080 * Use the createOutput method in {@link FanOutOneBlockAsyncDFSOutputHelper} to create. The mainly
081 * usage of this class is implementing WAL, so we only expose a little HDFS configurations in the
082 * method. And we place it here under io package because we want to make it independent of WAL
083 * implementation thus easier to move it to HDFS project finally.
084 * <p>
085 * Note that, although we support pipelined flush, i.e, write new data and then flush before the
086 * previous flush succeeds, the implementation is not thread safe, so you should not call its
087 * methods concurrently.
088 * <p>
089 * Advantages compare to DFSOutputStream:
090 * <ol>
091 * <li>The fan out mechanism. This will reduce the latency.</li>
092 * <li>Fail-fast when connection to datanode error. The WAL implementation could open new writer
093 * ASAP.</li>
094 * <li>We could benefit from netty's ByteBuf management mechanism.</li>
095 * </ol>
096 */
097@InterfaceAudience.Private
098public class FanOutOneBlockAsyncDFSOutput implements AsyncFSOutput {
099
100  // The MAX_PACKET_SIZE is 16MB but it include the header size and checksum size. So here we set a
101  // smaller limit for data size.
102  private static final int MAX_DATA_LEN = 12 * 1024 * 1024;
103
104  private final Configuration conf;
105
106  private final FSUtils fsUtils;
107
108  private final DistributedFileSystem dfs;
109
110  private final DFSClient client;
111
112  private final ClientProtocol namenode;
113
114  private final String clientName;
115
116  private final String src;
117
118  private final long fileId;
119
120  private final ExtendedBlock block;
121
122  private final DatanodeInfo[] locations;
123
124  private final Encryptor encryptor;
125
126  private final List<Channel> datanodeList;
127
128  private final DataChecksum summer;
129
130  private final int maxDataLen;
131
132  private final ByteBufAllocator alloc;
133
134  private static final class Callback {
135
136    private final CompletableFuture<Long> future;
137
138    private final long ackedLength;
139
140    // should be backed by a thread safe collection
141    private final Set<ChannelId> unfinishedReplicas;
142
143    public Callback(CompletableFuture<Long> future, long ackedLength,
144        Collection<Channel> replicas) {
145      this.future = future;
146      this.ackedLength = ackedLength;
147      if (replicas.isEmpty()) {
148        this.unfinishedReplicas = Collections.emptySet();
149      } else {
150        this.unfinishedReplicas =
151          Collections.newSetFromMap(new ConcurrentHashMap<ChannelId, Boolean>(replicas.size()));
152        replicas.stream().map(c -> c.id()).forEachOrdered(unfinishedReplicas::add);
153      }
154    }
155  }
156
157  private final ConcurrentLinkedDeque<Callback> waitingAckQueue = new ConcurrentLinkedDeque<>();
158
159  private volatile long ackedBlockLength = 0L;
160
161  // this could be different from acked block length because a packet can not start at the middle of
162  // a chunk.
163  private long nextPacketOffsetInBlock = 0L;
164
165  // the length of the trailing partial chunk, this is because the packet start offset must be
166  // aligned with the length of checksum chunk so we need to resend the same data.
167  private int trailingPartialChunkLength = 0;
168
169  private long nextPacketSeqno = 0L;
170
171  private ByteBuf buf;
172
173  private final SendBufSizePredictor sendBufSizePRedictor = new SendBufSizePredictor();
174
175  // State for connections to DN
176  private enum State {
177    STREAMING, CLOSING, BROKEN, CLOSED
178  }
179
180  private volatile State state;
181
182  // all lock-free to make it run faster
183  private void completed(Channel channel) {
184    for (Iterator<Callback> iter = waitingAckQueue.iterator(); iter.hasNext();) {
185      Callback c = iter.next();
186      // if the current unfinished replicas does not contain us then it means that we have already
187      // acked this one, let's iterate to find the one we have not acked yet.
188      if (c.unfinishedReplicas.remove(channel.id())) {
189        if (c.unfinishedReplicas.isEmpty()) {
190          // we need to remove first before complete the future. It is possible that after we
191          // complete the future the upper layer will call close immediately before we remove the
192          // entry from waitingAckQueue and lead to an IllegalStateException. And also set the
193          // ackedBlockLength first otherwise we may use a wrong length to commit the block. This
194          // may lead to multiple remove and assign but is OK. The semantic of iter.remove is
195          // removing the entry returned by calling previous next, so if the entry has already been
196          // removed then it is a no-op, and for the assign, the values are the same so no problem.
197          iter.remove();
198          ackedBlockLength = c.ackedLength;
199          // the future.complete check is to confirm that we are the only one who grabbed the work,
200          // otherwise just give up and return.
201          if (c.future.complete(c.ackedLength)) {
202            // also wake up flush requests which have the same length.
203            while (iter.hasNext()) {
204              Callback maybeDummyCb = iter.next();
205              if (maybeDummyCb.ackedLength == c.ackedLength) {
206                iter.remove();
207                maybeDummyCb.future.complete(c.ackedLength);
208              } else {
209                break;
210              }
211            }
212          }
213        }
214        return;
215      }
216    }
217  }
218
219  // this usually does not happen which means it is not on the critical path so make it synchronized
220  // so that the implementation will not burn up our brain as there are multiple state changes and
221  // checks.
222  private synchronized void failed(Channel channel, Supplier<Throwable> errorSupplier) {
223    if (state == State.BROKEN || state == State.CLOSED) {
224      return;
225    }
226    if (state == State.CLOSING) {
227      Callback c = waitingAckQueue.peekFirst();
228      if (c == null || !c.unfinishedReplicas.contains(channel.id())) {
229        // nothing, the endBlock request has already finished.
230        return;
231      }
232    }
233    // disable further write, and fail all pending ack.
234    state = State.BROKEN;
235    Throwable error = errorSupplier.get();
236    for (Iterator<Callback> iter = waitingAckQueue.iterator(); iter.hasNext();) {
237      Callback c = iter.next();
238      // find the first sync request which we have not acked yet and fail all the request after it.
239      if (!c.unfinishedReplicas.contains(channel.id())) {
240        continue;
241      }
242      for (;;) {
243        c.future.completeExceptionally(error);
244        if (!iter.hasNext()) {
245          break;
246        }
247        c = iter.next();
248      }
249      break;
250    }
251    datanodeList.forEach(ch -> ch.close());
252  }
253
254  @Sharable
255  private final class AckHandler extends SimpleChannelInboundHandler<PipelineAckProto> {
256
257    private final int timeoutMs;
258
259    public AckHandler(int timeoutMs) {
260      this.timeoutMs = timeoutMs;
261    }
262
263    @Override
264    protected void channelRead0(ChannelHandlerContext ctx, PipelineAckProto ack) throws Exception {
265      Status reply = getStatus(ack);
266      if (reply != Status.SUCCESS) {
267        failed(ctx.channel(), () -> new IOException("Bad response " + reply + " for block " +
268          block + " from datanode " + ctx.channel().remoteAddress()));
269        return;
270      }
271      if (PipelineAck.isRestartOOBStatus(reply)) {
272        failed(ctx.channel(), () -> new IOException("Restart response " + reply + " for block " +
273          block + " from datanode " + ctx.channel().remoteAddress()));
274        return;
275      }
276      if (ack.getSeqno() == HEART_BEAT_SEQNO) {
277        return;
278      }
279      completed(ctx.channel());
280    }
281
282    @Override
283    public void channelInactive(ChannelHandlerContext ctx) throws Exception {
284      if (state == State.CLOSED) {
285        return;
286      }
287      failed(ctx.channel(),
288        () -> new IOException("Connection to " + ctx.channel().remoteAddress() + " closed"));
289    }
290
291    @Override
292    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
293      failed(ctx.channel(), () -> cause);
294    }
295
296    @Override
297    public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
298      if (evt instanceof IdleStateEvent) {
299        IdleStateEvent e = (IdleStateEvent) evt;
300        if (e.state() == READER_IDLE) {
301          failed(ctx.channel(),
302            () -> new IOException("Timeout(" + timeoutMs + "ms) waiting for response"));
303        } else if (e.state() == WRITER_IDLE) {
304          PacketHeader heartbeat = new PacketHeader(4, 0, HEART_BEAT_SEQNO, false, 0, false);
305          int len = heartbeat.getSerializedSize();
306          ByteBuf buf = alloc.buffer(len);
307          heartbeat.putInBuffer(buf.nioBuffer(0, len));
308          buf.writerIndex(len);
309          ctx.channel().writeAndFlush(buf);
310        }
311        return;
312      }
313      super.userEventTriggered(ctx, evt);
314    }
315  }
316
317  private void setupReceiver(int timeoutMs) {
318    AckHandler ackHandler = new AckHandler(timeoutMs);
319    for (Channel ch : datanodeList) {
320      ch.pipeline().addLast(
321        new IdleStateHandler(timeoutMs, timeoutMs / 2, 0, TimeUnit.MILLISECONDS),
322        new ProtobufVarint32FrameDecoder(),
323        new ProtobufDecoder(PipelineAckProto.getDefaultInstance()), ackHandler);
324      ch.config().setAutoRead(true);
325    }
326  }
327
328  FanOutOneBlockAsyncDFSOutput(Configuration conf, FSUtils fsUtils, DistributedFileSystem dfs,
329      DFSClient client, ClientProtocol namenode, String clientName, String src, long fileId,
330      LocatedBlock locatedBlock, Encryptor encryptor, List<Channel> datanodeList,
331      DataChecksum summer, ByteBufAllocator alloc) {
332    this.conf = conf;
333    this.fsUtils = fsUtils;
334    this.dfs = dfs;
335    this.client = client;
336    this.namenode = namenode;
337    this.fileId = fileId;
338    this.clientName = clientName;
339    this.src = src;
340    this.block = locatedBlock.getBlock();
341    this.locations = locatedBlock.getLocations();
342    this.encryptor = encryptor;
343    this.datanodeList = datanodeList;
344    this.summer = summer;
345    this.maxDataLen = MAX_DATA_LEN - (MAX_DATA_LEN % summer.getBytesPerChecksum());
346    this.alloc = alloc;
347    this.buf = alloc.directBuffer(sendBufSizePRedictor.initialSize());
348    this.state = State.STREAMING;
349    setupReceiver(conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT));
350  }
351
352  @Override
353  public void writeInt(int i) {
354    buf.ensureWritable(4);
355    buf.writeInt(i);
356  }
357
358  @Override
359  public void write(ByteBuffer bb) {
360    buf.ensureWritable(bb.remaining());
361    buf.writeBytes(bb);
362  }
363
364  @Override
365  public void write(byte[] b) {
366    write(b, 0, b.length);
367  }
368
369  @Override
370  public void write(byte[] b, int off, int len) {
371    buf.ensureWritable(len);
372    buf.writeBytes(b, off, len);
373  }
374
375  @Override
376  public int buffered() {
377    return buf.readableBytes();
378  }
379
380  @Override
381  public DatanodeInfo[] getPipeline() {
382    return locations;
383  }
384
385  private void flushBuffer(CompletableFuture<Long> future, ByteBuf dataBuf,
386      long nextPacketOffsetInBlock, boolean syncBlock) {
387    int dataLen = dataBuf.readableBytes();
388    int chunkLen = summer.getBytesPerChecksum();
389    int trailingPartialChunkLen = dataLen % chunkLen;
390    int numChecks = dataLen / chunkLen + (trailingPartialChunkLen != 0 ? 1 : 0);
391    int checksumLen = numChecks * summer.getChecksumSize();
392    ByteBuf checksumBuf = alloc.directBuffer(checksumLen);
393    summer.calculateChunkedSums(dataBuf.nioBuffer(), checksumBuf.nioBuffer(0, checksumLen));
394    checksumBuf.writerIndex(checksumLen);
395    PacketHeader header = new PacketHeader(4 + checksumLen + dataLen, nextPacketOffsetInBlock,
396        nextPacketSeqno, false, dataLen, syncBlock);
397    int headerLen = header.getSerializedSize();
398    ByteBuf headerBuf = alloc.buffer(headerLen);
399    header.putInBuffer(headerBuf.nioBuffer(0, headerLen));
400    headerBuf.writerIndex(headerLen);
401    Callback c = new Callback(future, nextPacketOffsetInBlock + dataLen, datanodeList);
402    waitingAckQueue.addLast(c);
403    // recheck again after we pushed the callback to queue
404    if (state != State.STREAMING && waitingAckQueue.peekFirst() == c) {
405      future.completeExceptionally(new IOException("stream already broken"));
406      // it's the one we have just pushed or just a no-op
407      waitingAckQueue.removeFirst();
408      return;
409    }
410    // TODO: we should perhaps measure time taken per DN here;
411    //       we could collect statistics per DN, and/or exclude bad nodes in createOutput.
412    datanodeList.forEach(ch -> {
413      ch.write(headerBuf.retainedDuplicate());
414      ch.write(checksumBuf.retainedDuplicate());
415      ch.writeAndFlush(dataBuf.retainedDuplicate());
416    });
417    checksumBuf.release();
418    headerBuf.release();
419    dataBuf.release();
420    nextPacketSeqno++;
421  }
422
423  private void flush0(CompletableFuture<Long> future, boolean syncBlock) {
424    if (state != State.STREAMING) {
425      future.completeExceptionally(new IOException("stream already broken"));
426      return;
427    }
428    int dataLen = buf.readableBytes();
429    if (dataLen == trailingPartialChunkLength) {
430      // no new data
431      long lengthAfterFlush = nextPacketOffsetInBlock + dataLen;
432      Callback lastFlush = waitingAckQueue.peekLast();
433      if (lastFlush != null) {
434        Callback c = new Callback(future, lengthAfterFlush, Collections.emptyList());
435        waitingAckQueue.addLast(c);
436        // recheck here if we have already removed the previous callback from the queue
437        if (waitingAckQueue.peekFirst() == c) {
438          // all previous callbacks have been removed
439          // notice that this does mean we will always win here because the background thread may
440          // have already started to mark the future here as completed in the completed or failed
441          // methods but haven't removed it from the queue yet. That's also why the removeFirst
442          // call below may be a no-op.
443          if (state != State.STREAMING) {
444            future.completeExceptionally(new IOException("stream already broken"));
445          } else {
446            future.complete(lengthAfterFlush);
447          }
448          // it's the one we have just pushed or just a no-op
449          waitingAckQueue.removeFirst();
450        }
451      } else {
452        // we must have acked all the data so the ackedBlockLength must be same with
453        // lengthAfterFlush
454        future.complete(lengthAfterFlush);
455      }
456      return;
457    }
458
459    if (encryptor != null) {
460      ByteBuf encryptBuf = alloc.directBuffer(dataLen);
461      buf.readBytes(encryptBuf, trailingPartialChunkLength);
462      int toEncryptLength = dataLen - trailingPartialChunkLength;
463      try {
464        encryptor.encrypt(buf.nioBuffer(trailingPartialChunkLength, toEncryptLength),
465          encryptBuf.nioBuffer(trailingPartialChunkLength, toEncryptLength));
466      } catch (IOException e) {
467        encryptBuf.release();
468        future.completeExceptionally(e);
469        return;
470      }
471      encryptBuf.writerIndex(dataLen);
472      buf.release();
473      buf = encryptBuf;
474    }
475
476    if (dataLen > maxDataLen) {
477      // We need to write out the data by multiple packets as the max packet allowed is 16M.
478      long nextSubPacketOffsetInBlock = nextPacketOffsetInBlock;
479      for (int remaining = dataLen;;) {
480        if (remaining < maxDataLen) {
481          flushBuffer(future, buf.readRetainedSlice(remaining), nextSubPacketOffsetInBlock,
482            syncBlock);
483          break;
484        } else {
485          flushBuffer(new CompletableFuture<>(), buf.readRetainedSlice(maxDataLen),
486            nextSubPacketOffsetInBlock, syncBlock);
487          remaining -= maxDataLen;
488          nextSubPacketOffsetInBlock += maxDataLen;
489        }
490      }
491    } else {
492      flushBuffer(future, buf.retain(), nextPacketOffsetInBlock, syncBlock);
493    }
494    trailingPartialChunkLength = dataLen % summer.getBytesPerChecksum();
495    ByteBuf newBuf = alloc.directBuffer(sendBufSizePRedictor.guess(dataLen))
496        .ensureWritable(trailingPartialChunkLength);
497    if (trailingPartialChunkLength != 0) {
498      buf.readerIndex(dataLen - trailingPartialChunkLength).readBytes(newBuf,
499        trailingPartialChunkLength);
500    }
501    buf.release();
502    this.buf = newBuf;
503    nextPacketOffsetInBlock += dataLen - trailingPartialChunkLength;
504  }
505
506  /**
507   * Flush the buffer out to datanodes.
508   * @param syncBlock will call hsync if true, otherwise hflush.
509   * @return A CompletableFuture that hold the acked length after flushing.
510   */
511  @Override
512  public CompletableFuture<Long> flush(boolean syncBlock) {
513    CompletableFuture<Long> future = new CompletableFuture<>();
514    flush0(future, syncBlock);
515    return future;
516  }
517
518  private void endBlock() throws IOException {
519    Preconditions.checkState(waitingAckQueue.isEmpty(),
520      "should call flush first before calling close");
521    if (state != State.STREAMING) {
522      throw new IOException("stream already broken");
523    }
524    state = State.CLOSING;
525    long finalizedLength = ackedBlockLength;
526    PacketHeader header = new PacketHeader(4, finalizedLength, nextPacketSeqno, true, 0, false);
527    buf.release();
528    buf = null;
529    int headerLen = header.getSerializedSize();
530    ByteBuf headerBuf = alloc.directBuffer(headerLen);
531    header.putInBuffer(headerBuf.nioBuffer(0, headerLen));
532    headerBuf.writerIndex(headerLen);
533    CompletableFuture<Long> future = new CompletableFuture<>();
534    waitingAckQueue.add(new Callback(future, finalizedLength, datanodeList));
535    datanodeList.forEach(ch -> ch.writeAndFlush(headerBuf.retainedDuplicate()));
536    headerBuf.release();
537    try {
538      future.get();
539    } catch (InterruptedException e) {
540      throw (IOException) new InterruptedIOException().initCause(e);
541    } catch (ExecutionException e) {
542      Throwable cause = e.getCause();
543      Throwables.propagateIfPossible(cause, IOException.class);
544      throw new IOException(cause);
545    }
546  }
547
548  /**
549   * The close method when error occurred. Now we just call recoverFileLease.
550   */
551  @Override
552  public void recoverAndClose(CancelableProgressable reporter) throws IOException {
553    datanodeList.forEach(ch -> ch.close());
554    datanodeList.forEach(ch -> ch.closeFuture().awaitUninterruptibly());
555    endFileLease(client, fileId);
556    fsUtils.recoverFileLease(dfs, new Path(src), conf,
557      reporter == null ? new CancelOnClose(client) : reporter);
558  }
559
560  /**
561   * End the current block and complete file at namenode. You should call
562   * {@link #recoverAndClose(CancelableProgressable)} if this method throws an exception.
563   */
564  @Override
565  public void close() throws IOException {
566    endBlock();
567    state = State.CLOSED;
568    datanodeList.forEach(ch -> ch.close());
569    datanodeList.forEach(ch -> ch.closeFuture().awaitUninterruptibly());
570    block.setNumBytes(ackedBlockLength);
571    completeFile(client, namenode, src, clientName, block, fileId);
572  }
573
574  @Override
575  public boolean isBroken() {
576    return state == State.BROKEN;
577  }
578}