blob: bc266d01b0e47985d11a80ba5561110134050d02 [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.ipc, class: NettyRpcServerChannelWritabilityHandler">
<meta name="generator" content="javadoc/SourceToHTMLConverter">
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body class="source-page">
<main role="main">
<div class="source-container">
<pre><span class="source-line-no">001</span><span id="line-1">/*</span>
<span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span>
<span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span>
<span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span>
<span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span>
<span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span>
<span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span>
<span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span>
<span class="source-line-no">009</span><span id="line-9"> *</span>
<span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="source-line-no">011</span><span id="line-11"> *</span>
<span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span>
<span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span>
<span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span>
<span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span>
<span class="source-line-no">017</span><span id="line-17"> */</span>
<span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.ipc;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import java.util.function.BooleanSupplier;</span>
<span class="source-line-no">021</span><span id="line-21">import java.util.function.IntSupplier;</span>
<span class="source-line-no">022</span><span id="line-22">import org.apache.hadoop.hbase.exceptions.ConnectionClosedException;</span>
<span class="source-line-no">023</span><span id="line-23">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span>
<span class="source-line-no">024</span><span id="line-24">import org.apache.hadoop.hbase.util.NettyUnsafeUtils;</span>
<span class="source-line-no">025</span><span id="line-25">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">026</span><span id="line-26"></span>
<span class="source-line-no">027</span><span id="line-27">import org.apache.hbase.thirdparty.io.netty.channel.Channel;</span>
<span class="source-line-no">028</span><span id="line-28">import org.apache.hbase.thirdparty.io.netty.channel.ChannelDuplexHandler;</span>
<span class="source-line-no">029</span><span id="line-29">import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;</span>
<span class="source-line-no">030</span><span id="line-30">import org.apache.hbase.thirdparty.io.netty.channel.ChannelPromise;</span>
<span class="source-line-no">031</span><span id="line-31">import org.apache.hbase.thirdparty.io.netty.util.ReferenceCountUtil;</span>
<span class="source-line-no">032</span><span id="line-32"></span>
<span class="source-line-no">033</span><span id="line-33">/**</span>
<span class="source-line-no">034</span><span id="line-34"> * Handler to enforce writability protections on our server channels: &lt;br&gt;</span>
<span class="source-line-no">035</span><span id="line-35"> * - Responds to channel writability events, which are triggered when the total pending bytes for a</span>
<span class="source-line-no">036</span><span id="line-36"> * channel passes configured high and low watermarks. When high watermark is exceeded, the channel</span>
<span class="source-line-no">037</span><span id="line-37"> * is setAutoRead(false). This way, we won't accept new requests from the client until some pending</span>
<span class="source-line-no">038</span><span id="line-38"> * outbound bytes are successfully received by the client.&lt;br&gt;</span>
<span class="source-line-no">039</span><span id="line-39"> * - Pre-processes any channel write requests. If the total pending outbound bytes exceeds a fatal</span>
<span class="source-line-no">040</span><span id="line-40"> * threshold, the channel is forcefully closed and the write is set to failed. This handler should</span>
<span class="source-line-no">041</span><span id="line-41"> * be the last handler in the pipeline so that it's the first handler to receive any messages sent</span>
<span class="source-line-no">042</span><span id="line-42"> * to channel.write() or channel.writeAndFlush().</span>
<span class="source-line-no">043</span><span id="line-43"> */</span>
<span class="source-line-no">044</span><span id="line-44">@InterfaceAudience.Private</span>
<span class="source-line-no">045</span><span id="line-45">public class NettyRpcServerChannelWritabilityHandler extends ChannelDuplexHandler {</span>
<span class="source-line-no">046</span><span id="line-46"></span>
<span class="source-line-no">047</span><span id="line-47"> static final String NAME = "NettyRpcServerChannelWritabilityHandler";</span>
<span class="source-line-no">048</span><span id="line-48"></span>
<span class="source-line-no">049</span><span id="line-49"> private final MetricsHBaseServer metrics;</span>
<span class="source-line-no">050</span><span id="line-50"> private final IntSupplier pendingBytesFatalThreshold;</span>
<span class="source-line-no">051</span><span id="line-51"> private final BooleanSupplier isWritabilityBackpressureEnabled;</span>
<span class="source-line-no">052</span><span id="line-52"></span>
<span class="source-line-no">053</span><span id="line-53"> private boolean writable = true;</span>
<span class="source-line-no">054</span><span id="line-54"> private long unwritableStartTime;</span>
<span class="source-line-no">055</span><span id="line-55"></span>
<span class="source-line-no">056</span><span id="line-56"> NettyRpcServerChannelWritabilityHandler(MetricsHBaseServer metrics,</span>
<span class="source-line-no">057</span><span id="line-57"> IntSupplier pendingBytesFatalThreshold, BooleanSupplier isWritabilityBackpressureEnabled) {</span>
<span class="source-line-no">058</span><span id="line-58"> this.metrics = metrics;</span>
<span class="source-line-no">059</span><span id="line-59"> this.pendingBytesFatalThreshold = pendingBytesFatalThreshold;</span>
<span class="source-line-no">060</span><span id="line-60"> this.isWritabilityBackpressureEnabled = isWritabilityBackpressureEnabled;</span>
<span class="source-line-no">061</span><span id="line-61"> }</span>
<span class="source-line-no">062</span><span id="line-62"></span>
<span class="source-line-no">063</span><span id="line-63"> @Override</span>
<span class="source-line-no">064</span><span id="line-64"> public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)</span>
<span class="source-line-no">065</span><span id="line-65"> throws Exception {</span>
<span class="source-line-no">066</span><span id="line-66"> if (handleFatalThreshold(ctx)) {</span>
<span class="source-line-no">067</span><span id="line-67"> promise.setFailure(</span>
<span class="source-line-no">068</span><span id="line-68"> new ConnectionClosedException("Channel outbound bytes exceeded fatal threshold"));</span>
<span class="source-line-no">069</span><span id="line-69"> if (msg instanceof RpcResponse) {</span>
<span class="source-line-no">070</span><span id="line-70"> ((RpcResponse) msg).done();</span>
<span class="source-line-no">071</span><span id="line-71"> } else {</span>
<span class="source-line-no">072</span><span id="line-72"> ReferenceCountUtil.release(msg);</span>
<span class="source-line-no">073</span><span id="line-73"> }</span>
<span class="source-line-no">074</span><span id="line-74"> return;</span>
<span class="source-line-no">075</span><span id="line-75"> }</span>
<span class="source-line-no">076</span><span id="line-76"> ctx.write(msg, promise);</span>
<span class="source-line-no">077</span><span id="line-77"> }</span>
<span class="source-line-no">078</span><span id="line-78"></span>
<span class="source-line-no">079</span><span id="line-79"> @Override</span>
<span class="source-line-no">080</span><span id="line-80"> public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {</span>
<span class="source-line-no">081</span><span id="line-81"> if (isWritabilityBackpressureEnabled.getAsBoolean()) {</span>
<span class="source-line-no">082</span><span id="line-82"> handleWritabilityChanged(ctx);</span>
<span class="source-line-no">083</span><span id="line-83"> }</span>
<span class="source-line-no">084</span><span id="line-84"> ctx.fireChannelWritabilityChanged();</span>
<span class="source-line-no">085</span><span id="line-85"> }</span>
<span class="source-line-no">086</span><span id="line-86"></span>
<span class="source-line-no">087</span><span id="line-87"> private boolean handleFatalThreshold(ChannelHandlerContext ctx) {</span>
<span class="source-line-no">088</span><span id="line-88"> int fatalThreshold = pendingBytesFatalThreshold.getAsInt();</span>
<span class="source-line-no">089</span><span id="line-89"> if (fatalThreshold &lt;= 0) {</span>
<span class="source-line-no">090</span><span id="line-90"> return false;</span>
<span class="source-line-no">091</span><span id="line-91"> }</span>
<span class="source-line-no">092</span><span id="line-92"></span>
<span class="source-line-no">093</span><span id="line-93"> Channel channel = ctx.channel();</span>
<span class="source-line-no">094</span><span id="line-94"> long outboundBytes = NettyUnsafeUtils.getTotalPendingOutboundBytes(channel);</span>
<span class="source-line-no">095</span><span id="line-95"> if (outboundBytes &lt; fatalThreshold) {</span>
<span class="source-line-no">096</span><span id="line-96"> return false;</span>
<span class="source-line-no">097</span><span id="line-97"> }</span>
<span class="source-line-no">098</span><span id="line-98"></span>
<span class="source-line-no">099</span><span id="line-99"> if (channel.isOpen()) {</span>
<span class="source-line-no">100</span><span id="line-100"> metrics.maxOutboundBytesExceeded();</span>
<span class="source-line-no">101</span><span id="line-101"> RpcServer.LOG.warn(</span>
<span class="source-line-no">102</span><span id="line-102"> "{}: Closing connection because outbound buffer size of {} exceeds fatal threshold of {}",</span>
<span class="source-line-no">103</span><span id="line-103"> channel.remoteAddress(), outboundBytes, fatalThreshold);</span>
<span class="source-line-no">104</span><span id="line-104"> NettyUnsafeUtils.closeImmediately(channel);</span>
<span class="source-line-no">105</span><span id="line-105"> }</span>
<span class="source-line-no">106</span><span id="line-106"></span>
<span class="source-line-no">107</span><span id="line-107"> return true;</span>
<span class="source-line-no">108</span><span id="line-108"> }</span>
<span class="source-line-no">109</span><span id="line-109"></span>
<span class="source-line-no">110</span><span id="line-110"> private void handleWritabilityChanged(ChannelHandlerContext ctx) {</span>
<span class="source-line-no">111</span><span id="line-111"> boolean oldWritableValue = this.writable;</span>
<span class="source-line-no">112</span><span id="line-112"></span>
<span class="source-line-no">113</span><span id="line-113"> this.writable = ctx.channel().isWritable();</span>
<span class="source-line-no">114</span><span id="line-114"> ctx.channel().config().setAutoRead(this.writable);</span>
<span class="source-line-no">115</span><span id="line-115"></span>
<span class="source-line-no">116</span><span id="line-116"> if (!oldWritableValue &amp;&amp; this.writable) {</span>
<span class="source-line-no">117</span><span id="line-117"> // changing from not writable to writable, update metrics</span>
<span class="source-line-no">118</span><span id="line-118"> metrics.unwritableTime(EnvironmentEdgeManager.currentTime() - unwritableStartTime);</span>
<span class="source-line-no">119</span><span id="line-119"> unwritableStartTime = 0;</span>
<span class="source-line-no">120</span><span id="line-120"> } else if (oldWritableValue &amp;&amp; !this.writable) {</span>
<span class="source-line-no">121</span><span id="line-121"> // changing from writable to non-writable, set start time</span>
<span class="source-line-no">122</span><span id="line-122"> unwritableStartTime = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">123</span><span id="line-123"> }</span>
<span class="source-line-no">124</span><span id="line-124"> }</span>
<span class="source-line-no">125</span><span id="line-125">}</span>
</pre>
</div>
</main>
</body>
</html>