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.ipc;
019
020import org.apache.yetus.audience.InterfaceAudience;
021
022import org.apache.hbase.thirdparty.io.netty.buffer.Unpooled;
023import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
024import org.apache.hbase.thirdparty.io.netty.channel.ChannelOutboundHandlerAdapter;
025import org.apache.hbase.thirdparty.io.netty.channel.ChannelPromise;
026
027/**
028 * Encoder for {@link RpcResponse}.
029 * @since 2.0.0
030 */
031@InterfaceAudience.Private
032class NettyRpcServerResponseEncoder extends ChannelOutboundHandlerAdapter {
033
034  static final String NAME = "NettyRpcServerResponseEncoder";
035
036  private final MetricsHBaseServer metrics;
037
038  NettyRpcServerResponseEncoder(MetricsHBaseServer metrics) {
039    this.metrics = metrics;
040  }
041
042  @Override
043  public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
044    throws Exception {
045    if (msg instanceof RpcResponse) {
046      RpcResponse resp = (RpcResponse) msg;
047      BufferChain buf = resp.getResponse();
048      ctx.write(Unpooled.wrappedBuffer(buf.getBuffers()), promise).addListener(f -> {
049        resp.done();
050        if (f.isSuccess()) {
051          metrics.sentBytes(buf.size());
052        }
053      });
054    } else {
055      ctx.write(msg, promise);
056    }
057  }
058}