blob: 17b6d79e5dcee79a7506845c428065fd8fe5bdf8 [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: TestNettyChannelWritability">
<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 static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE;</span>
<span class="source-line-no">021</span><span id="line-21">import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub;</span>
<span class="source-line-no">022</span><span id="line-22">import static org.junit.Assert.assertEquals;</span>
<span class="source-line-no">023</span><span id="line-23">import static org.junit.Assert.assertNotNull;</span>
<span class="source-line-no">024</span><span id="line-24">import static org.junit.Assert.assertThrows;</span>
<span class="source-line-no">025</span><span id="line-25">import static org.junit.Assert.assertTrue;</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 java.io.IOException;</span>
<span class="source-line-no">028</span><span id="line-28">import java.net.InetSocketAddress;</span>
<span class="source-line-no">029</span><span id="line-29">import java.util.ArrayList;</span>
<span class="source-line-no">030</span><span id="line-30">import java.util.List;</span>
<span class="source-line-no">031</span><span id="line-31">import java.util.concurrent.CompletableFuture;</span>
<span class="source-line-no">032</span><span id="line-32">import java.util.concurrent.CompletionException;</span>
<span class="source-line-no">033</span><span id="line-33">import java.util.concurrent.atomic.AtomicInteger;</span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.hbase.CellScanner;</span>
<span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.hbase.CompatibilityFactory;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.hadoop.hbase.ExtendedCell;</span>
<span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.hbase.HBaseClassTestRule;</span>
<span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.hbase.HBaseConfiguration;</span>
<span class="source-line-no">040</span><span id="line-40">import org.apache.hadoop.hbase.KeyValue;</span>
<span class="source-line-no">041</span><span id="line-41">import org.apache.hadoop.hbase.PrivateCellUtil;</span>
<span class="source-line-no">042</span><span id="line-42">import org.apache.hadoop.hbase.test.MetricsAssertHelper;</span>
<span class="source-line-no">043</span><span id="line-43">import org.apache.hadoop.hbase.testclassification.MediumTests;</span>
<span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.hbase.testclassification.RPCTests;</span>
<span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">046</span><span id="line-46">import org.junit.ClassRule;</span>
<span class="source-line-no">047</span><span id="line-47">import org.junit.Test;</span>
<span class="source-line-no">048</span><span id="line-48">import org.junit.experimental.categories.Category;</span>
<span class="source-line-no">049</span><span id="line-49"></span>
<span class="source-line-no">050</span><span id="line-50">import org.apache.hbase.thirdparty.com.google.common.collect.Lists;</span>
<span class="source-line-no">051</span><span id="line-51">import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;</span>
<span class="source-line-no">052</span><span id="line-52">import org.apache.hbase.thirdparty.io.netty.channel.Channel;</span>
<span class="source-line-no">053</span><span id="line-53"></span>
<span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos;</span>
<span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos;</span>
<span class="source-line-no">056</span><span id="line-56"></span>
<span class="source-line-no">057</span><span id="line-57">@Category({ RPCTests.class, MediumTests.class })</span>
<span class="source-line-no">058</span><span id="line-58">public class TestNettyChannelWritability {</span>
<span class="source-line-no">059</span><span id="line-59"></span>
<span class="source-line-no">060</span><span id="line-60"> @ClassRule</span>
<span class="source-line-no">061</span><span id="line-61"> public static final HBaseClassTestRule CLASS_RULE =</span>
<span class="source-line-no">062</span><span id="line-62"> HBaseClassTestRule.forClass(TestNettyChannelWritability.class);</span>
<span class="source-line-no">063</span><span id="line-63"></span>
<span class="source-line-no">064</span><span id="line-64"> private static final MetricsAssertHelper METRICS_ASSERT =</span>
<span class="source-line-no">065</span><span id="line-65"> CompatibilityFactory.getInstance(MetricsAssertHelper.class);</span>
<span class="source-line-no">066</span><span id="line-66"></span>
<span class="source-line-no">067</span><span id="line-67"> private static final byte[] CELL_BYTES = Bytes.toBytes("xyz");</span>
<span class="source-line-no">068</span><span id="line-68"> private static final KeyValue CELL = new KeyValue(CELL_BYTES, CELL_BYTES, CELL_BYTES, CELL_BYTES);</span>
<span class="source-line-no">069</span><span id="line-69"></span>
<span class="source-line-no">070</span><span id="line-70"> /**</span>
<span class="source-line-no">071</span><span id="line-71"> * Test that we properly send configured watermarks to netty, and trigger setWritable when</span>
<span class="source-line-no">072</span><span id="line-72"> * necessary.</span>
<span class="source-line-no">073</span><span id="line-73"> */</span>
<span class="source-line-no">074</span><span id="line-74"> @Test</span>
<span class="source-line-no">075</span><span id="line-75"> public void testNettyWritableWatermarks() throws Exception {</span>
<span class="source-line-no">076</span><span id="line-76"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">077</span><span id="line-77"> conf.setInt(NettyRpcServer.CHANNEL_WRITABLE_LOW_WATERMARK_KEY, 1);</span>
<span class="source-line-no">078</span><span id="line-78"> conf.setInt(NettyRpcServer.CHANNEL_WRITABLE_HIGH_WATERMARK_KEY, 2);</span>
<span class="source-line-no">079</span><span id="line-79"></span>
<span class="source-line-no">080</span><span id="line-80"> NettyRpcServer rpcServer = createRpcServer(conf, 0);</span>
<span class="source-line-no">081</span><span id="line-81"> try {</span>
<span class="source-line-no">082</span><span id="line-82"> sendAndReceive(conf, rpcServer, 5);</span>
<span class="source-line-no">083</span><span id="line-83"> METRICS_ASSERT.assertCounterGt("unwritableTime_numOps", 0,</span>
<span class="source-line-no">084</span><span id="line-84"> rpcServer.metrics.getMetricsSource());</span>
<span class="source-line-no">085</span><span id="line-85"> } finally {</span>
<span class="source-line-no">086</span><span id="line-86"> rpcServer.stop();</span>
<span class="source-line-no">087</span><span id="line-87"> }</span>
<span class="source-line-no">088</span><span id="line-88"> }</span>
<span class="source-line-no">089</span><span id="line-89"></span>
<span class="source-line-no">090</span><span id="line-90"> /**</span>
<span class="source-line-no">091</span><span id="line-91"> * Test that our fatal watermark is honored, which requires artificially causing some queueing so</span>
<span class="source-line-no">092</span><span id="line-92"> * that pendingOutboundBytes increases.</span>
<span class="source-line-no">093</span><span id="line-93"> */</span>
<span class="source-line-no">094</span><span id="line-94"> @Test</span>
<span class="source-line-no">095</span><span id="line-95"> public void testNettyWritableFatalThreshold() throws Exception {</span>
<span class="source-line-no">096</span><span id="line-96"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">097</span><span id="line-97"> conf.setInt(NettyRpcServer.CHANNEL_WRITABLE_FATAL_WATERMARK_KEY, 1);</span>
<span class="source-line-no">098</span><span id="line-98"></span>
<span class="source-line-no">099</span><span id="line-99"> // flushAfter is 3 here, with requestCount 5 below. If we never flush, the WriteTasks will sit</span>
<span class="source-line-no">100</span><span id="line-100"> // in the eventloop. So we flush a few at once, which will ensure that we hit fatal threshold</span>
<span class="source-line-no">101</span><span id="line-101"> NettyRpcServer rpcServer = createRpcServer(conf, 3);</span>
<span class="source-line-no">102</span><span id="line-102"> try {</span>
<span class="source-line-no">103</span><span id="line-103"> CompletionException exception =</span>
<span class="source-line-no">104</span><span id="line-104"> assertThrows(CompletionException.class, () -&gt; sendAndReceive(conf, rpcServer, 5));</span>
<span class="source-line-no">105</span><span id="line-105"> assertTrue(exception.getCause().getCause() instanceof ServiceException);</span>
<span class="source-line-no">106</span><span id="line-106"> METRICS_ASSERT.assertCounterGt("maxOutboundBytesExceeded", 0,</span>
<span class="source-line-no">107</span><span id="line-107"> rpcServer.metrics.getMetricsSource());</span>
<span class="source-line-no">108</span><span id="line-108"> } finally {</span>
<span class="source-line-no">109</span><span id="line-109"> rpcServer.stop();</span>
<span class="source-line-no">110</span><span id="line-110"> }</span>
<span class="source-line-no">111</span><span id="line-111"> }</span>
<span class="source-line-no">112</span><span id="line-112"></span>
<span class="source-line-no">113</span><span id="line-113"> private void sendAndReceive(Configuration conf, NettyRpcServer rpcServer, int requestCount)</span>
<span class="source-line-no">114</span><span id="line-114"> throws Exception {</span>
<span class="source-line-no">115</span><span id="line-115"> List&lt;ExtendedCell&gt; cells = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">116</span><span id="line-116"> int count = 3;</span>
<span class="source-line-no">117</span><span id="line-117"> for (int i = 0; i &lt; count; i++) {</span>
<span class="source-line-no">118</span><span id="line-118"> cells.add(CELL);</span>
<span class="source-line-no">119</span><span id="line-119"> }</span>
<span class="source-line-no">120</span><span id="line-120"></span>
<span class="source-line-no">121</span><span id="line-121"> try (NettyRpcClient client = new NettyRpcClient(conf)) {</span>
<span class="source-line-no">122</span><span id="line-122"> rpcServer.start();</span>
<span class="source-line-no">123</span><span id="line-123"> TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub =</span>
<span class="source-line-no">124</span><span id="line-124"> newBlockingStub(client, rpcServer.getListenerAddress());</span>
<span class="source-line-no">125</span><span id="line-125"> CompletableFuture&lt;Void&gt;[] futures = new CompletableFuture[requestCount];</span>
<span class="source-line-no">126</span><span id="line-126"> for (int i = 0; i &lt; requestCount; i++) {</span>
<span class="source-line-no">127</span><span id="line-127"> futures[i] = CompletableFuture.runAsync(() -&gt; {</span>
<span class="source-line-no">128</span><span id="line-128"> try {</span>
<span class="source-line-no">129</span><span id="line-129"> sendMessage(cells, stub);</span>
<span class="source-line-no">130</span><span id="line-130"> } catch (Exception e) {</span>
<span class="source-line-no">131</span><span id="line-131"> throw new RuntimeException(e);</span>
<span class="source-line-no">132</span><span id="line-132"> }</span>
<span class="source-line-no">133</span><span id="line-133"> });</span>
<span class="source-line-no">134</span><span id="line-134"> }</span>
<span class="source-line-no">135</span><span id="line-135"> CompletableFuture.allOf(futures).join();</span>
<span class="source-line-no">136</span><span id="line-136"> }</span>
<span class="source-line-no">137</span><span id="line-137"> }</span>
<span class="source-line-no">138</span><span id="line-138"></span>
<span class="source-line-no">139</span><span id="line-139"> private void sendMessage(List&lt;ExtendedCell&gt; cells,</span>
<span class="source-line-no">140</span><span id="line-140"> TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub) throws Exception {</span>
<span class="source-line-no">141</span><span id="line-141"> HBaseRpcController pcrc =</span>
<span class="source-line-no">142</span><span id="line-142"> new HBaseRpcControllerImpl(PrivateCellUtil.createExtendedCellScanner(cells));</span>
<span class="source-line-no">143</span><span id="line-143"> String message = "hello";</span>
<span class="source-line-no">144</span><span id="line-144"> assertEquals(message,</span>
<span class="source-line-no">145</span><span id="line-145"> stub.echo(pcrc, TestProtos.EchoRequestProto.newBuilder().setMessage(message).build())</span>
<span class="source-line-no">146</span><span id="line-146"> .getMessage());</span>
<span class="source-line-no">147</span><span id="line-147"> int index = 0;</span>
<span class="source-line-no">148</span><span id="line-148"> CellScanner cellScanner = pcrc.cellScanner();</span>
<span class="source-line-no">149</span><span id="line-149"> assertNotNull(cellScanner);</span>
<span class="source-line-no">150</span><span id="line-150"> while (cellScanner.advance()) {</span>
<span class="source-line-no">151</span><span id="line-151"> assertEquals(CELL, cellScanner.current());</span>
<span class="source-line-no">152</span><span id="line-152"> index++;</span>
<span class="source-line-no">153</span><span id="line-153"> }</span>
<span class="source-line-no">154</span><span id="line-154"> assertEquals(cells.size(), index);</span>
<span class="source-line-no">155</span><span id="line-155"> }</span>
<span class="source-line-no">156</span><span id="line-156"></span>
<span class="source-line-no">157</span><span id="line-157"> private NettyRpcServer createRpcServer(Configuration conf, int flushAfter) throws IOException {</span>
<span class="source-line-no">158</span><span id="line-158"> String name = "testRpcServer";</span>
<span class="source-line-no">159</span><span id="line-159"> ArrayList&lt;RpcServer.BlockingServiceAndInterface&gt; services =</span>
<span class="source-line-no">160</span><span id="line-160"> Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null));</span>
<span class="source-line-no">161</span><span id="line-161"></span>
<span class="source-line-no">162</span><span id="line-162"> InetSocketAddress bindAddress = new InetSocketAddress("localhost", 0);</span>
<span class="source-line-no">163</span><span id="line-163"> FifoRpcScheduler scheduler = new FifoRpcScheduler(conf, 1);</span>
<span class="source-line-no">164</span><span id="line-164"></span>
<span class="source-line-no">165</span><span id="line-165"> AtomicInteger writeCount = new AtomicInteger(0);</span>
<span class="source-line-no">166</span><span id="line-166"></span>
<span class="source-line-no">167</span><span id="line-167"> return new NettyRpcServer(null, name, services, bindAddress, conf, scheduler, true) {</span>
<span class="source-line-no">168</span><span id="line-168"> @Override</span>
<span class="source-line-no">169</span><span id="line-169"> protected NettyServerRpcConnection createNettyServerRpcConnection(Channel channel) {</span>
<span class="source-line-no">170</span><span id="line-170"> return new NettyServerRpcConnection(this, channel) {</span>
<span class="source-line-no">171</span><span id="line-171"> @Override</span>
<span class="source-line-no">172</span><span id="line-172"> protected void doRespond(RpcResponse resp) {</span>
<span class="source-line-no">173</span><span id="line-173"> if (writeCount.incrementAndGet() &gt;= flushAfter) {</span>
<span class="source-line-no">174</span><span id="line-174"> super.doRespond(resp);</span>
<span class="source-line-no">175</span><span id="line-175"> } else {</span>
<span class="source-line-no">176</span><span id="line-176"> channel.write(resp);</span>
<span class="source-line-no">177</span><span id="line-177"> }</span>
<span class="source-line-no">178</span><span id="line-178"> }</span>
<span class="source-line-no">179</span><span id="line-179"> };</span>
<span class="source-line-no">180</span><span id="line-180"> }</span>
<span class="source-line-no">181</span><span id="line-181"> };</span>
<span class="source-line-no">182</span><span id="line-182"> }</span>
<span class="source-line-no">183</span><span id="line-183">}</span>
</pre>
</div>
</main>
</body>
</html>