blob: 259a2318b5898e0000fa7952672b79e6833e71d2 [file] [log] [blame]
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
<html lang="en">
<head>
<title>Source code</title>
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body>
<div class="sourceContainer">
<pre><span class="sourceLineNo">001</span>/*<a name="line.1"></a>
<span class="sourceLineNo">002</span> * Licensed to the Apache Software Foundation (ASF) under one<a name="line.2"></a>
<span class="sourceLineNo">003</span> * or more contributor license agreements. See the NOTICE file<a name="line.3"></a>
<span class="sourceLineNo">004</span> * distributed with this work for additional information<a name="line.4"></a>
<span class="sourceLineNo">005</span> * regarding copyright ownership. The ASF licenses this file<a name="line.5"></a>
<span class="sourceLineNo">006</span> * to you under the Apache License, Version 2.0 (the<a name="line.6"></a>
<span class="sourceLineNo">007</span> * "License"); you may not use this file except in compliance<a name="line.7"></a>
<span class="sourceLineNo">008</span> * with the License. You may obtain a copy of the License at<a name="line.8"></a>
<span class="sourceLineNo">009</span> *<a name="line.9"></a>
<span class="sourceLineNo">010</span> * http://www.apache.org/licenses/LICENSE-2.0<a name="line.10"></a>
<span class="sourceLineNo">011</span> *<a name="line.11"></a>
<span class="sourceLineNo">012</span> * Unless required by applicable law or agreed to in writing, software<a name="line.12"></a>
<span class="sourceLineNo">013</span> * distributed under the License is distributed on an "AS IS" BASIS,<a name="line.13"></a>
<span class="sourceLineNo">014</span> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.<a name="line.14"></a>
<span class="sourceLineNo">015</span> * See the License for the specific language governing permissions and<a name="line.15"></a>
<span class="sourceLineNo">016</span> * limitations under the License.<a name="line.16"></a>
<span class="sourceLineNo">017</span> */<a name="line.17"></a>
<span class="sourceLineNo">018</span>package org.apache.hadoop.hbase.ipc;<a name="line.18"></a>
<span class="sourceLineNo">019</span><a name="line.19"></a>
<span class="sourceLineNo">020</span>import io.opentelemetry.api.trace.Span;<a name="line.20"></a>
<span class="sourceLineNo">021</span>import io.opentelemetry.api.trace.StatusCode;<a name="line.21"></a>
<span class="sourceLineNo">022</span>import io.opentelemetry.context.Scope;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import java.net.InetSocketAddress;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import java.nio.channels.ClosedChannelException;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import org.apache.hadoop.hbase.CallDroppedException;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import org.apache.hadoop.hbase.CellScanner;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import org.apache.hadoop.hbase.HBaseInterfaceAudience;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import org.apache.hadoop.hbase.exceptions.TimeoutIOException;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import org.apache.hadoop.hbase.security.User;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import org.apache.hadoop.hbase.server.trace.IpcServerSpanBuilder;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import org.apache.hadoop.hbase.trace.TraceUtil;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import org.apache.hadoop.hbase.util.Pair;<a name="line.34"></a>
<span class="sourceLineNo">035</span>import org.apache.hadoop.util.StringUtils;<a name="line.35"></a>
<span class="sourceLineNo">036</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.36"></a>
<span class="sourceLineNo">037</span>import org.apache.yetus.audience.InterfaceStability;<a name="line.37"></a>
<span class="sourceLineNo">038</span><a name="line.38"></a>
<span class="sourceLineNo">039</span>import org.apache.hbase.thirdparty.com.google.protobuf.Message;<a name="line.39"></a>
<span class="sourceLineNo">040</span><a name="line.40"></a>
<span class="sourceLineNo">041</span>/**<a name="line.41"></a>
<span class="sourceLineNo">042</span> * The request processing logic, which is usually executed in thread pools provided by an<a name="line.42"></a>
<span class="sourceLineNo">043</span> * {@link RpcScheduler}. Call {@link #run()} to actually execute the contained RpcServer.Call<a name="line.43"></a>
<span class="sourceLineNo">044</span> */<a name="line.44"></a>
<span class="sourceLineNo">045</span>@InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX })<a name="line.45"></a>
<span class="sourceLineNo">046</span>@InterfaceStability.Evolving<a name="line.46"></a>
<span class="sourceLineNo">047</span>public class CallRunner {<a name="line.47"></a>
<span class="sourceLineNo">048</span><a name="line.48"></a>
<span class="sourceLineNo">049</span> private static final CallDroppedException CALL_DROPPED_EXCEPTION = new CallDroppedException();<a name="line.49"></a>
<span class="sourceLineNo">050</span><a name="line.50"></a>
<span class="sourceLineNo">051</span> private RpcCall call;<a name="line.51"></a>
<span class="sourceLineNo">052</span> private RpcServerInterface rpcServer;<a name="line.52"></a>
<span class="sourceLineNo">053</span> private MonitoredRPCHandler status;<a name="line.53"></a>
<span class="sourceLineNo">054</span> private final Span span;<a name="line.54"></a>
<span class="sourceLineNo">055</span> private volatile boolean successful;<a name="line.55"></a>
<span class="sourceLineNo">056</span><a name="line.56"></a>
<span class="sourceLineNo">057</span> /**<a name="line.57"></a>
<span class="sourceLineNo">058</span> * On construction, adds the size of this call to the running count of outstanding call sizes.<a name="line.58"></a>
<span class="sourceLineNo">059</span> * Presumption is that we are put on a queue while we wait on an executor to run us. During this<a name="line.59"></a>
<span class="sourceLineNo">060</span> * time we occupy heap.<a name="line.60"></a>
<span class="sourceLineNo">061</span> */<a name="line.61"></a>
<span class="sourceLineNo">062</span> // The constructor is shutdown so only RpcServer in this class can make one of these.<a name="line.62"></a>
<span class="sourceLineNo">063</span> CallRunner(final RpcServerInterface rpcServer, final RpcCall call) {<a name="line.63"></a>
<span class="sourceLineNo">064</span> this.call = call;<a name="line.64"></a>
<span class="sourceLineNo">065</span> this.rpcServer = rpcServer;<a name="line.65"></a>
<span class="sourceLineNo">066</span> this.span = Span.current();<a name="line.66"></a>
<span class="sourceLineNo">067</span> // Add size of the call to queue size.<a name="line.67"></a>
<span class="sourceLineNo">068</span> if (call != null &amp;&amp; rpcServer != null) {<a name="line.68"></a>
<span class="sourceLineNo">069</span> this.rpcServer.addCallSize(call.getSize());<a name="line.69"></a>
<span class="sourceLineNo">070</span> }<a name="line.70"></a>
<span class="sourceLineNo">071</span> }<a name="line.71"></a>
<span class="sourceLineNo">072</span><a name="line.72"></a>
<span class="sourceLineNo">073</span> public RpcCall getRpcCall() {<a name="line.73"></a>
<span class="sourceLineNo">074</span> return call;<a name="line.74"></a>
<span class="sourceLineNo">075</span> }<a name="line.75"></a>
<span class="sourceLineNo">076</span><a name="line.76"></a>
<span class="sourceLineNo">077</span> public void setStatus(MonitoredRPCHandler status) {<a name="line.77"></a>
<span class="sourceLineNo">078</span> this.status = status;<a name="line.78"></a>
<span class="sourceLineNo">079</span> }<a name="line.79"></a>
<span class="sourceLineNo">080</span><a name="line.80"></a>
<span class="sourceLineNo">081</span> /**<a name="line.81"></a>
<span class="sourceLineNo">082</span> * Cleanup after ourselves... let go of references.<a name="line.82"></a>
<span class="sourceLineNo">083</span> */<a name="line.83"></a>
<span class="sourceLineNo">084</span> private void cleanup() {<a name="line.84"></a>
<span class="sourceLineNo">085</span> this.call.cleanup();<a name="line.85"></a>
<span class="sourceLineNo">086</span> this.call = null;<a name="line.86"></a>
<span class="sourceLineNo">087</span> this.rpcServer = null;<a name="line.87"></a>
<span class="sourceLineNo">088</span> }<a name="line.88"></a>
<span class="sourceLineNo">089</span><a name="line.89"></a>
<span class="sourceLineNo">090</span> public void run() {<a name="line.90"></a>
<span class="sourceLineNo">091</span> try (Scope ignored = span.makeCurrent()) {<a name="line.91"></a>
<span class="sourceLineNo">092</span> if (call.disconnectSince() &gt;= 0) {<a name="line.92"></a>
<span class="sourceLineNo">093</span> RpcServer.LOG.debug("{}: skipped {}", Thread.currentThread().getName(), call);<a name="line.93"></a>
<span class="sourceLineNo">094</span> span.addEvent("Client disconnect detected");<a name="line.94"></a>
<span class="sourceLineNo">095</span> span.setStatus(StatusCode.OK);<a name="line.95"></a>
<span class="sourceLineNo">096</span> return;<a name="line.96"></a>
<span class="sourceLineNo">097</span> }<a name="line.97"></a>
<span class="sourceLineNo">098</span> call.setStartTime(EnvironmentEdgeManager.currentTime());<a name="line.98"></a>
<span class="sourceLineNo">099</span> if (call.getStartTime() &gt; call.getDeadline()) {<a name="line.99"></a>
<span class="sourceLineNo">100</span> RpcServer.LOG.warn("Dropping timed out call: {}", call);<a name="line.100"></a>
<span class="sourceLineNo">101</span> this.rpcServer.getMetrics().callTimedOut();<a name="line.101"></a>
<span class="sourceLineNo">102</span> span.addEvent("Call deadline exceeded");<a name="line.102"></a>
<span class="sourceLineNo">103</span> span.setStatus(StatusCode.OK);<a name="line.103"></a>
<span class="sourceLineNo">104</span> return;<a name="line.104"></a>
<span class="sourceLineNo">105</span> }<a name="line.105"></a>
<span class="sourceLineNo">106</span> this.status.setStatus("Setting up call");<a name="line.106"></a>
<span class="sourceLineNo">107</span> this.status.setConnection(call.getRemoteAddress().getHostAddress(), call.getRemotePort());<a name="line.107"></a>
<span class="sourceLineNo">108</span> if (RpcServer.LOG.isTraceEnabled()) {<a name="line.108"></a>
<span class="sourceLineNo">109</span> RpcServer.LOG.trace("{} executing as {}", call.toShortString(),<a name="line.109"></a>
<span class="sourceLineNo">110</span> call.getRequestUser().map(User::getName).orElse("NULL principal"));<a name="line.110"></a>
<span class="sourceLineNo">111</span> }<a name="line.111"></a>
<span class="sourceLineNo">112</span> Throwable errorThrowable = null;<a name="line.112"></a>
<span class="sourceLineNo">113</span> String error = null;<a name="line.113"></a>
<span class="sourceLineNo">114</span> Pair&lt;Message, CellScanner&gt; resultPair = null;<a name="line.114"></a>
<span class="sourceLineNo">115</span> RpcServer.CurCall.set(call);<a name="line.115"></a>
<span class="sourceLineNo">116</span> final Span ipcServerSpan = new IpcServerSpanBuilder(call).build();<a name="line.116"></a>
<span class="sourceLineNo">117</span> try (Scope ignored1 = ipcServerSpan.makeCurrent()) {<a name="line.117"></a>
<span class="sourceLineNo">118</span> if (!this.rpcServer.isStarted()) {<a name="line.118"></a>
<span class="sourceLineNo">119</span> InetSocketAddress address = rpcServer.getListenerAddress();<a name="line.119"></a>
<span class="sourceLineNo">120</span> throw new ServerNotRunningYetException(<a name="line.120"></a>
<span class="sourceLineNo">121</span> "Server " + (address != null ? address : "(channel closed)") + " is not running yet");<a name="line.121"></a>
<span class="sourceLineNo">122</span> }<a name="line.122"></a>
<span class="sourceLineNo">123</span> // make the call<a name="line.123"></a>
<span class="sourceLineNo">124</span> resultPair = this.rpcServer.call(call, this.status);<a name="line.124"></a>
<span class="sourceLineNo">125</span> } catch (TimeoutIOException e) {<a name="line.125"></a>
<span class="sourceLineNo">126</span> RpcServer.LOG.warn("Can not complete this request in time, drop it: {}", call);<a name="line.126"></a>
<span class="sourceLineNo">127</span> TraceUtil.setError(ipcServerSpan, e);<a name="line.127"></a>
<span class="sourceLineNo">128</span> return;<a name="line.128"></a>
<span class="sourceLineNo">129</span> } catch (Throwable e) {<a name="line.129"></a>
<span class="sourceLineNo">130</span> TraceUtil.setError(ipcServerSpan, e);<a name="line.130"></a>
<span class="sourceLineNo">131</span> if (e instanceof ServerNotRunningYetException) {<a name="line.131"></a>
<span class="sourceLineNo">132</span> // If ServerNotRunningYetException, don't spew stack trace.<a name="line.132"></a>
<span class="sourceLineNo">133</span> if (RpcServer.LOG.isTraceEnabled()) {<a name="line.133"></a>
<span class="sourceLineNo">134</span> RpcServer.LOG.trace(call.toShortString(), e);<a name="line.134"></a>
<span class="sourceLineNo">135</span> }<a name="line.135"></a>
<span class="sourceLineNo">136</span> } else {<a name="line.136"></a>
<span class="sourceLineNo">137</span> // Don't dump full exception.. just String version<a name="line.137"></a>
<span class="sourceLineNo">138</span> RpcServer.LOG.debug("{}, exception={}", call.toShortString(), e);<a name="line.138"></a>
<span class="sourceLineNo">139</span> }<a name="line.139"></a>
<span class="sourceLineNo">140</span> errorThrowable = e;<a name="line.140"></a>
<span class="sourceLineNo">141</span> error = StringUtils.stringifyException(e);<a name="line.141"></a>
<span class="sourceLineNo">142</span> if (e instanceof Error) {<a name="line.142"></a>
<span class="sourceLineNo">143</span> throw (Error) e;<a name="line.143"></a>
<span class="sourceLineNo">144</span> }<a name="line.144"></a>
<span class="sourceLineNo">145</span> } finally {<a name="line.145"></a>
<span class="sourceLineNo">146</span> RpcServer.CurCall.set(null);<a name="line.146"></a>
<span class="sourceLineNo">147</span> if (resultPair != null) {<a name="line.147"></a>
<span class="sourceLineNo">148</span> this.rpcServer.addCallSize(call.getSize() * -1);<a name="line.148"></a>
<span class="sourceLineNo">149</span> ipcServerSpan.setStatus(StatusCode.OK);<a name="line.149"></a>
<span class="sourceLineNo">150</span> successful = true;<a name="line.150"></a>
<span class="sourceLineNo">151</span> }<a name="line.151"></a>
<span class="sourceLineNo">152</span> ipcServerSpan.end();<a name="line.152"></a>
<span class="sourceLineNo">153</span> }<a name="line.153"></a>
<span class="sourceLineNo">154</span> this.status.markComplete("To send response");<a name="line.154"></a>
<span class="sourceLineNo">155</span> // return the RPC request read BB we can do here. It is done by now.<a name="line.155"></a>
<span class="sourceLineNo">156</span> call.cleanup();<a name="line.156"></a>
<span class="sourceLineNo">157</span> // Set the response<a name="line.157"></a>
<span class="sourceLineNo">158</span> Message param = resultPair != null ? resultPair.getFirst() : null;<a name="line.158"></a>
<span class="sourceLineNo">159</span> CellScanner cells = resultPair != null ? resultPair.getSecond() : null;<a name="line.159"></a>
<span class="sourceLineNo">160</span> call.setResponse(param, cells, errorThrowable, error);<a name="line.160"></a>
<span class="sourceLineNo">161</span> call.sendResponseIfReady();<a name="line.161"></a>
<span class="sourceLineNo">162</span> // don't touch `span` here because its status and `end()` are managed in `call#setResponse()`<a name="line.162"></a>
<span class="sourceLineNo">163</span> } catch (OutOfMemoryError e) {<a name="line.163"></a>
<span class="sourceLineNo">164</span> TraceUtil.setError(span, e);<a name="line.164"></a>
<span class="sourceLineNo">165</span> if (<a name="line.165"></a>
<span class="sourceLineNo">166</span> this.rpcServer.getErrorHandler() != null &amp;&amp; this.rpcServer.getErrorHandler().checkOOME(e)<a name="line.166"></a>
<span class="sourceLineNo">167</span> ) {<a name="line.167"></a>
<span class="sourceLineNo">168</span> RpcServer.LOG.info("{}: exiting on OutOfMemoryError", Thread.currentThread().getName());<a name="line.168"></a>
<span class="sourceLineNo">169</span> // exception intentionally swallowed<a name="line.169"></a>
<span class="sourceLineNo">170</span> } else {<a name="line.170"></a>
<span class="sourceLineNo">171</span> // rethrow if no handler<a name="line.171"></a>
<span class="sourceLineNo">172</span> throw e;<a name="line.172"></a>
<span class="sourceLineNo">173</span> }<a name="line.173"></a>
<span class="sourceLineNo">174</span> } catch (ClosedChannelException cce) {<a name="line.174"></a>
<span class="sourceLineNo">175</span> InetSocketAddress address = rpcServer.getListenerAddress();<a name="line.175"></a>
<span class="sourceLineNo">176</span> RpcServer.LOG.warn(<a name="line.176"></a>
<span class="sourceLineNo">177</span> "{}: caught a ClosedChannelException, " + "this means that the server "<a name="line.177"></a>
<span class="sourceLineNo">178</span> + (address != null ? address : "(channel closed)")<a name="line.178"></a>
<span class="sourceLineNo">179</span> + " was processing a request but the client went away. The error message was: {}",<a name="line.179"></a>
<span class="sourceLineNo">180</span> Thread.currentThread().getName(), cce.getMessage());<a name="line.180"></a>
<span class="sourceLineNo">181</span> TraceUtil.setError(span, cce);<a name="line.181"></a>
<span class="sourceLineNo">182</span> } catch (Exception e) {<a name="line.182"></a>
<span class="sourceLineNo">183</span> RpcServer.LOG.warn("{}: caught: {}", Thread.currentThread().getName(),<a name="line.183"></a>
<span class="sourceLineNo">184</span> StringUtils.stringifyException(e));<a name="line.184"></a>
<span class="sourceLineNo">185</span> TraceUtil.setError(span, e);<a name="line.185"></a>
<span class="sourceLineNo">186</span> } finally {<a name="line.186"></a>
<span class="sourceLineNo">187</span> if (!successful) {<a name="line.187"></a>
<span class="sourceLineNo">188</span> this.rpcServer.addCallSize(call.getSize() * -1);<a name="line.188"></a>
<span class="sourceLineNo">189</span> }<a name="line.189"></a>
<span class="sourceLineNo">190</span><a name="line.190"></a>
<span class="sourceLineNo">191</span> if (this.status.isRPCRunning()) {<a name="line.191"></a>
<span class="sourceLineNo">192</span> this.status.markComplete("Call error");<a name="line.192"></a>
<span class="sourceLineNo">193</span> }<a name="line.193"></a>
<span class="sourceLineNo">194</span> this.status.pause("Waiting for a call");<a name="line.194"></a>
<span class="sourceLineNo">195</span> cleanup();<a name="line.195"></a>
<span class="sourceLineNo">196</span> span.end();<a name="line.196"></a>
<span class="sourceLineNo">197</span> }<a name="line.197"></a>
<span class="sourceLineNo">198</span> }<a name="line.198"></a>
<span class="sourceLineNo">199</span><a name="line.199"></a>
<span class="sourceLineNo">200</span> /**<a name="line.200"></a>
<span class="sourceLineNo">201</span> * When we want to drop this call because of server is overloaded.<a name="line.201"></a>
<span class="sourceLineNo">202</span> */<a name="line.202"></a>
<span class="sourceLineNo">203</span> public void drop() {<a name="line.203"></a>
<span class="sourceLineNo">204</span> try (Scope ignored = span.makeCurrent()) {<a name="line.204"></a>
<span class="sourceLineNo">205</span> if (call.disconnectSince() &gt;= 0) {<a name="line.205"></a>
<span class="sourceLineNo">206</span> RpcServer.LOG.debug("{}: skipped {}", Thread.currentThread().getName(), call);<a name="line.206"></a>
<span class="sourceLineNo">207</span> span.addEvent("Client disconnect detected");<a name="line.207"></a>
<span class="sourceLineNo">208</span> span.setStatus(StatusCode.OK);<a name="line.208"></a>
<span class="sourceLineNo">209</span> return;<a name="line.209"></a>
<span class="sourceLineNo">210</span> }<a name="line.210"></a>
<span class="sourceLineNo">211</span><a name="line.211"></a>
<span class="sourceLineNo">212</span> // Set the response<a name="line.212"></a>
<span class="sourceLineNo">213</span> InetSocketAddress address = rpcServer.getListenerAddress();<a name="line.213"></a>
<span class="sourceLineNo">214</span> call.setResponse(null, null, CALL_DROPPED_EXCEPTION, "Call dropped, server "<a name="line.214"></a>
<span class="sourceLineNo">215</span> + (address != null ? address : "(channel closed)") + " is overloaded, please retry.");<a name="line.215"></a>
<span class="sourceLineNo">216</span> TraceUtil.setError(span, CALL_DROPPED_EXCEPTION);<a name="line.216"></a>
<span class="sourceLineNo">217</span> call.sendResponseIfReady();<a name="line.217"></a>
<span class="sourceLineNo">218</span> this.rpcServer.getMetrics().exception(CALL_DROPPED_EXCEPTION);<a name="line.218"></a>
<span class="sourceLineNo">219</span> } catch (ClosedChannelException cce) {<a name="line.219"></a>
<span class="sourceLineNo">220</span> InetSocketAddress address = rpcServer.getListenerAddress();<a name="line.220"></a>
<span class="sourceLineNo">221</span> RpcServer.LOG.warn(<a name="line.221"></a>
<span class="sourceLineNo">222</span> "{}: caught a ClosedChannelException, " + "this means that the server "<a name="line.222"></a>
<span class="sourceLineNo">223</span> + (address != null ? address : "(channel closed)")<a name="line.223"></a>
<span class="sourceLineNo">224</span> + " was processing a request but the client went away. The error message was: {}",<a name="line.224"></a>
<span class="sourceLineNo">225</span> Thread.currentThread().getName(), cce.getMessage());<a name="line.225"></a>
<span class="sourceLineNo">226</span> TraceUtil.setError(span, cce);<a name="line.226"></a>
<span class="sourceLineNo">227</span> } catch (Exception e) {<a name="line.227"></a>
<span class="sourceLineNo">228</span> RpcServer.LOG.warn("{}: caught: {}", Thread.currentThread().getName(),<a name="line.228"></a>
<span class="sourceLineNo">229</span> StringUtils.stringifyException(e));<a name="line.229"></a>
<span class="sourceLineNo">230</span> TraceUtil.setError(span, e);<a name="line.230"></a>
<span class="sourceLineNo">231</span> } finally {<a name="line.231"></a>
<span class="sourceLineNo">232</span> if (!successful) {<a name="line.232"></a>
<span class="sourceLineNo">233</span> this.rpcServer.addCallSize(call.getSize() * -1);<a name="line.233"></a>
<span class="sourceLineNo">234</span> }<a name="line.234"></a>
<span class="sourceLineNo">235</span> cleanup();<a name="line.235"></a>
<span class="sourceLineNo">236</span> span.end();<a name="line.236"></a>
<span class="sourceLineNo">237</span> }<a name="line.237"></a>
<span class="sourceLineNo">238</span> }<a name="line.238"></a>
<span class="sourceLineNo">239</span>}<a name="line.239"></a>
</pre>
</div>
</body>
</html>