blob: 8d5b8c4b283f3e086fbd3df281232c73d228ef6e [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.client;<a name="line.18"></a>
<span class="sourceLineNo">019</span><a name="line.19"></a>
<span class="sourceLineNo">020</span>import static java.util.stream.Collectors.toList;<a name="line.20"></a>
<span class="sourceLineNo">021</span>import static org.apache.hadoop.hbase.HConstants.EMPTY_END_ROW;<a name="line.21"></a>
<span class="sourceLineNo">022</span>import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import static org.apache.hadoop.hbase.util.FutureUtils.addListener;<a name="line.23"></a>
<span class="sourceLineNo">024</span><a name="line.24"></a>
<span class="sourceLineNo">025</span>import java.io.IOException;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import java.lang.reflect.UndeclaredThrowableException;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import java.net.UnknownHostException;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import java.util.Arrays;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import java.util.List;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import java.util.Optional;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import java.util.concurrent.CompletableFuture;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import java.util.concurrent.ExecutorService;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import java.util.concurrent.ThreadLocalRandom;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import java.util.concurrent.TimeUnit;<a name="line.34"></a>
<span class="sourceLineNo">035</span>import java.util.concurrent.atomic.AtomicReference;<a name="line.35"></a>
<span class="sourceLineNo">036</span>import java.util.function.Function;<a name="line.36"></a>
<span class="sourceLineNo">037</span>import java.util.function.Predicate;<a name="line.37"></a>
<span class="sourceLineNo">038</span>import java.util.function.Supplier;<a name="line.38"></a>
<span class="sourceLineNo">039</span>import org.apache.hadoop.conf.Configuration;<a name="line.39"></a>
<span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.Cell;<a name="line.40"></a>
<span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.CellComparator;<a name="line.41"></a>
<span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.HConstants;<a name="line.42"></a>
<span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.HRegionLocation;<a name="line.43"></a>
<span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.PrivateCellUtil;<a name="line.44"></a>
<span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.RegionLocations;<a name="line.45"></a>
<span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.ServerName;<a name="line.46"></a>
<span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.TableName;<a name="line.47"></a>
<span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.client.metrics.ScanMetrics;<a name="line.48"></a>
<span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.ipc.HBaseRpcController;<a name="line.49"></a>
<span class="sourceLineNo">050</span>import org.apache.hadoop.hbase.ipc.ServerRpcController;<a name="line.50"></a>
<span class="sourceLineNo">051</span>import org.apache.hadoop.hbase.util.Bytes;<a name="line.51"></a>
<span class="sourceLineNo">052</span>import org.apache.hadoop.hbase.util.ReflectionUtils;<a name="line.52"></a>
<span class="sourceLineNo">053</span>import org.apache.hadoop.ipc.RemoteException;<a name="line.53"></a>
<span class="sourceLineNo">054</span>import org.apache.hadoop.net.DNS;<a name="line.54"></a>
<span class="sourceLineNo">055</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.55"></a>
<span class="sourceLineNo">056</span>import org.slf4j.Logger;<a name="line.56"></a>
<span class="sourceLineNo">057</span>import org.slf4j.LoggerFactory;<a name="line.57"></a>
<span class="sourceLineNo">058</span><a name="line.58"></a>
<span class="sourceLineNo">059</span>import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;<a name="line.59"></a>
<span class="sourceLineNo">060</span>import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;<a name="line.60"></a>
<span class="sourceLineNo">061</span>import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;<a name="line.61"></a>
<span class="sourceLineNo">062</span>import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;<a name="line.62"></a>
<span class="sourceLineNo">063</span>import org.apache.hbase.thirdparty.io.netty.util.Timer;<a name="line.63"></a>
<span class="sourceLineNo">064</span><a name="line.64"></a>
<span class="sourceLineNo">065</span>import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;<a name="line.65"></a>
<span class="sourceLineNo">066</span>import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;<a name="line.66"></a>
<span class="sourceLineNo">067</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;<a name="line.67"></a>
<span class="sourceLineNo">068</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;<a name="line.68"></a>
<span class="sourceLineNo">069</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;<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> * Utility used by client connections.<a name="line.72"></a>
<span class="sourceLineNo">073</span> */<a name="line.73"></a>
<span class="sourceLineNo">074</span>@InterfaceAudience.Private<a name="line.74"></a>
<span class="sourceLineNo">075</span>public final class ConnectionUtils {<a name="line.75"></a>
<span class="sourceLineNo">076</span><a name="line.76"></a>
<span class="sourceLineNo">077</span> private static final Logger LOG = LoggerFactory.getLogger(ConnectionUtils.class);<a name="line.77"></a>
<span class="sourceLineNo">078</span><a name="line.78"></a>
<span class="sourceLineNo">079</span> /**<a name="line.79"></a>
<span class="sourceLineNo">080</span> * Key for configuration in Configuration whose value is the class we implement making a new<a name="line.80"></a>
<span class="sourceLineNo">081</span> * Connection instance.<a name="line.81"></a>
<span class="sourceLineNo">082</span> */<a name="line.82"></a>
<span class="sourceLineNo">083</span> public static final String HBASE_CLIENT_CONNECTION_IMPL = "hbase.client.connection.impl";<a name="line.83"></a>
<span class="sourceLineNo">084</span><a name="line.84"></a>
<span class="sourceLineNo">085</span> private ConnectionUtils() {<a name="line.85"></a>
<span class="sourceLineNo">086</span> }<a name="line.86"></a>
<span class="sourceLineNo">087</span><a name="line.87"></a>
<span class="sourceLineNo">088</span> /**<a name="line.88"></a>
<span class="sourceLineNo">089</span> * Calculate pause time. Built on {@link HConstants#RETRY_BACKOFF}.<a name="line.89"></a>
<span class="sourceLineNo">090</span> * @param pause time to pause<a name="line.90"></a>
<span class="sourceLineNo">091</span> * @param tries amount of tries<a name="line.91"></a>
<span class="sourceLineNo">092</span> * @return How long to wait after &lt;code&gt;tries&lt;/code&gt; retries<a name="line.92"></a>
<span class="sourceLineNo">093</span> */<a name="line.93"></a>
<span class="sourceLineNo">094</span> public static long getPauseTime(final long pause, final int tries) {<a name="line.94"></a>
<span class="sourceLineNo">095</span> int ntries = tries;<a name="line.95"></a>
<span class="sourceLineNo">096</span> if (ntries &gt;= HConstants.RETRY_BACKOFF.length) {<a name="line.96"></a>
<span class="sourceLineNo">097</span> ntries = HConstants.RETRY_BACKOFF.length - 1;<a name="line.97"></a>
<span class="sourceLineNo">098</span> }<a name="line.98"></a>
<span class="sourceLineNo">099</span> if (ntries &lt; 0) {<a name="line.99"></a>
<span class="sourceLineNo">100</span> ntries = 0;<a name="line.100"></a>
<span class="sourceLineNo">101</span> }<a name="line.101"></a>
<span class="sourceLineNo">102</span><a name="line.102"></a>
<span class="sourceLineNo">103</span> long normalPause = pause * HConstants.RETRY_BACKOFF[ntries];<a name="line.103"></a>
<span class="sourceLineNo">104</span> // 1% possible jitter<a name="line.104"></a>
<span class="sourceLineNo">105</span> long jitter = (long) (normalPause * ThreadLocalRandom.current().nextFloat() * 0.01f);<a name="line.105"></a>
<span class="sourceLineNo">106</span> return normalPause + jitter;<a name="line.106"></a>
<span class="sourceLineNo">107</span> }<a name="line.107"></a>
<span class="sourceLineNo">108</span><a name="line.108"></a>
<span class="sourceLineNo">109</span> /**<a name="line.109"></a>
<span class="sourceLineNo">110</span> * Changes the configuration to set the number of retries needed when using Connection internally,<a name="line.110"></a>
<span class="sourceLineNo">111</span> * e.g. for updating catalog tables, etc. Call this method before we create any Connections.<a name="line.111"></a>
<span class="sourceLineNo">112</span> * @param c The Configuration instance to set the retries into.<a name="line.112"></a>
<span class="sourceLineNo">113</span> * @param log Used to log what we set in here.<a name="line.113"></a>
<span class="sourceLineNo">114</span> */<a name="line.114"></a>
<span class="sourceLineNo">115</span> public static void setServerSideHConnectionRetriesConfig(final Configuration c, final String sn,<a name="line.115"></a>
<span class="sourceLineNo">116</span> final Logger log) {<a name="line.116"></a>
<span class="sourceLineNo">117</span> // TODO: Fix this. Not all connections from server side should have 10 times the retries.<a name="line.117"></a>
<span class="sourceLineNo">118</span> int hcRetries = c.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,<a name="line.118"></a>
<span class="sourceLineNo">119</span> HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);<a name="line.119"></a>
<span class="sourceLineNo">120</span> // Go big. Multiply by 10. If we can't get to meta after this many retries<a name="line.120"></a>
<span class="sourceLineNo">121</span> // then something seriously wrong.<a name="line.121"></a>
<span class="sourceLineNo">122</span> int serversideMultiplier = c.getInt(HConstants.HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER,<a name="line.122"></a>
<span class="sourceLineNo">123</span> HConstants.DEFAULT_HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER);<a name="line.123"></a>
<span class="sourceLineNo">124</span> int retries = hcRetries * serversideMultiplier;<a name="line.124"></a>
<span class="sourceLineNo">125</span> c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);<a name="line.125"></a>
<span class="sourceLineNo">126</span> log.info(sn + " server-side Connection retries=" + retries);<a name="line.126"></a>
<span class="sourceLineNo">127</span> }<a name="line.127"></a>
<span class="sourceLineNo">128</span><a name="line.128"></a>
<span class="sourceLineNo">129</span> /**<a name="line.129"></a>
<span class="sourceLineNo">130</span> * Get a unique key for the rpc stub to the given server.<a name="line.130"></a>
<span class="sourceLineNo">131</span> */<a name="line.131"></a>
<span class="sourceLineNo">132</span> static String getStubKey(String serviceName, ServerName serverName) {<a name="line.132"></a>
<span class="sourceLineNo">133</span> return String.format("%s@%s", serviceName, serverName);<a name="line.133"></a>
<span class="sourceLineNo">134</span> }<a name="line.134"></a>
<span class="sourceLineNo">135</span><a name="line.135"></a>
<span class="sourceLineNo">136</span> /**<a name="line.136"></a>
<span class="sourceLineNo">137</span> * Return retires + 1. The returned value will be in range [1, Integer.MAX_VALUE].<a name="line.137"></a>
<span class="sourceLineNo">138</span> */<a name="line.138"></a>
<span class="sourceLineNo">139</span> static int retries2Attempts(int retries) {<a name="line.139"></a>
<span class="sourceLineNo">140</span> return Math.max(1, retries == Integer.MAX_VALUE ? Integer.MAX_VALUE : retries + 1);<a name="line.140"></a>
<span class="sourceLineNo">141</span> }<a name="line.141"></a>
<span class="sourceLineNo">142</span><a name="line.142"></a>
<span class="sourceLineNo">143</span> static void checkHasFamilies(Mutation mutation) {<a name="line.143"></a>
<span class="sourceLineNo">144</span> Preconditions.checkArgument(mutation.numFamilies() &gt; 0,<a name="line.144"></a>
<span class="sourceLineNo">145</span> "Invalid arguments to %s, zero columns specified", mutation.toString());<a name="line.145"></a>
<span class="sourceLineNo">146</span> }<a name="line.146"></a>
<span class="sourceLineNo">147</span><a name="line.147"></a>
<span class="sourceLineNo">148</span> /** Dummy nonce generator for disabled nonces. */<a name="line.148"></a>
<span class="sourceLineNo">149</span> static final NonceGenerator NO_NONCE_GENERATOR = new NonceGenerator() {<a name="line.149"></a>
<span class="sourceLineNo">150</span><a name="line.150"></a>
<span class="sourceLineNo">151</span> @Override<a name="line.151"></a>
<span class="sourceLineNo">152</span> public long newNonce() {<a name="line.152"></a>
<span class="sourceLineNo">153</span> return HConstants.NO_NONCE;<a name="line.153"></a>
<span class="sourceLineNo">154</span> }<a name="line.154"></a>
<span class="sourceLineNo">155</span><a name="line.155"></a>
<span class="sourceLineNo">156</span> @Override<a name="line.156"></a>
<span class="sourceLineNo">157</span> public long getNonceGroup() {<a name="line.157"></a>
<span class="sourceLineNo">158</span> return HConstants.NO_NONCE;<a name="line.158"></a>
<span class="sourceLineNo">159</span> }<a name="line.159"></a>
<span class="sourceLineNo">160</span> };<a name="line.160"></a>
<span class="sourceLineNo">161</span><a name="line.161"></a>
<span class="sourceLineNo">162</span> // A byte array in which all elements are the max byte, and it is used to<a name="line.162"></a>
<span class="sourceLineNo">163</span> // construct closest front row<a name="line.163"></a>
<span class="sourceLineNo">164</span> static final byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);<a name="line.164"></a>
<span class="sourceLineNo">165</span><a name="line.165"></a>
<span class="sourceLineNo">166</span> /**<a name="line.166"></a>
<span class="sourceLineNo">167</span> * Create the closest row after the specified row<a name="line.167"></a>
<span class="sourceLineNo">168</span> */<a name="line.168"></a>
<span class="sourceLineNo">169</span> static byte[] createClosestRowAfter(byte[] row) {<a name="line.169"></a>
<span class="sourceLineNo">170</span> return Arrays.copyOf(row, row.length + 1);<a name="line.170"></a>
<span class="sourceLineNo">171</span> }<a name="line.171"></a>
<span class="sourceLineNo">172</span><a name="line.172"></a>
<span class="sourceLineNo">173</span> /**<a name="line.173"></a>
<span class="sourceLineNo">174</span> * Create a row before the specified row and very close to the specified row.<a name="line.174"></a>
<span class="sourceLineNo">175</span> */<a name="line.175"></a>
<span class="sourceLineNo">176</span> static byte[] createCloseRowBefore(byte[] row) {<a name="line.176"></a>
<span class="sourceLineNo">177</span> if (row.length == 0) {<a name="line.177"></a>
<span class="sourceLineNo">178</span> return MAX_BYTE_ARRAY;<a name="line.178"></a>
<span class="sourceLineNo">179</span> }<a name="line.179"></a>
<span class="sourceLineNo">180</span> if (row[row.length - 1] == 0) {<a name="line.180"></a>
<span class="sourceLineNo">181</span> return Arrays.copyOf(row, row.length - 1);<a name="line.181"></a>
<span class="sourceLineNo">182</span> } else {<a name="line.182"></a>
<span class="sourceLineNo">183</span> byte[] nextRow = new byte[row.length + MAX_BYTE_ARRAY.length];<a name="line.183"></a>
<span class="sourceLineNo">184</span> System.arraycopy(row, 0, nextRow, 0, row.length - 1);<a name="line.184"></a>
<span class="sourceLineNo">185</span> nextRow[row.length - 1] = (byte) ((row[row.length - 1] &amp; 0xFF) - 1);<a name="line.185"></a>
<span class="sourceLineNo">186</span> System.arraycopy(MAX_BYTE_ARRAY, 0, nextRow, row.length, MAX_BYTE_ARRAY.length);<a name="line.186"></a>
<span class="sourceLineNo">187</span> return nextRow;<a name="line.187"></a>
<span class="sourceLineNo">188</span> }<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> static boolean isEmptyStartRow(byte[] row) {<a name="line.191"></a>
<span class="sourceLineNo">192</span> return Bytes.equals(row, EMPTY_START_ROW);<a name="line.192"></a>
<span class="sourceLineNo">193</span> }<a name="line.193"></a>
<span class="sourceLineNo">194</span><a name="line.194"></a>
<span class="sourceLineNo">195</span> static boolean isEmptyStopRow(byte[] row) {<a name="line.195"></a>
<span class="sourceLineNo">196</span> return Bytes.equals(row, EMPTY_END_ROW);<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> static void resetController(HBaseRpcController controller, long timeoutNs, int priority) {<a name="line.199"></a>
<span class="sourceLineNo">200</span> controller.reset();<a name="line.200"></a>
<span class="sourceLineNo">201</span> if (timeoutNs &gt;= 0) {<a name="line.201"></a>
<span class="sourceLineNo">202</span> controller.setCallTimeout(<a name="line.202"></a>
<span class="sourceLineNo">203</span> (int) Math.min(Integer.MAX_VALUE, TimeUnit.NANOSECONDS.toMillis(timeoutNs)));<a name="line.203"></a>
<span class="sourceLineNo">204</span> }<a name="line.204"></a>
<span class="sourceLineNo">205</span> controller.setPriority(priority);<a name="line.205"></a>
<span class="sourceLineNo">206</span> }<a name="line.206"></a>
<span class="sourceLineNo">207</span><a name="line.207"></a>
<span class="sourceLineNo">208</span> static Throwable translateException(Throwable t) {<a name="line.208"></a>
<span class="sourceLineNo">209</span> if (t instanceof UndeclaredThrowableException &amp;&amp; t.getCause() != null) {<a name="line.209"></a>
<span class="sourceLineNo">210</span> t = t.getCause();<a name="line.210"></a>
<span class="sourceLineNo">211</span> }<a name="line.211"></a>
<span class="sourceLineNo">212</span> if (t instanceof RemoteException) {<a name="line.212"></a>
<span class="sourceLineNo">213</span> t = ((RemoteException) t).unwrapRemoteException();<a name="line.213"></a>
<span class="sourceLineNo">214</span> }<a name="line.214"></a>
<span class="sourceLineNo">215</span> if (t instanceof ServiceException &amp;&amp; t.getCause() != null) {<a name="line.215"></a>
<span class="sourceLineNo">216</span> t = translateException(t.getCause());<a name="line.216"></a>
<span class="sourceLineNo">217</span> }<a name="line.217"></a>
<span class="sourceLineNo">218</span> return t;<a name="line.218"></a>
<span class="sourceLineNo">219</span> }<a name="line.219"></a>
<span class="sourceLineNo">220</span><a name="line.220"></a>
<span class="sourceLineNo">221</span> static long calcEstimatedSize(Result rs) {<a name="line.221"></a>
<span class="sourceLineNo">222</span> long estimatedHeapSizeOfResult = 0;<a name="line.222"></a>
<span class="sourceLineNo">223</span> // We don't make Iterator here<a name="line.223"></a>
<span class="sourceLineNo">224</span> for (Cell cell : rs.rawCells()) {<a name="line.224"></a>
<span class="sourceLineNo">225</span> estimatedHeapSizeOfResult += cell.heapSize();<a name="line.225"></a>
<span class="sourceLineNo">226</span> }<a name="line.226"></a>
<span class="sourceLineNo">227</span> return estimatedHeapSizeOfResult;<a name="line.227"></a>
<span class="sourceLineNo">228</span> }<a name="line.228"></a>
<span class="sourceLineNo">229</span><a name="line.229"></a>
<span class="sourceLineNo">230</span> static Result filterCells(Result result, Cell keepCellsAfter) {<a name="line.230"></a>
<span class="sourceLineNo">231</span> if (keepCellsAfter == null) {<a name="line.231"></a>
<span class="sourceLineNo">232</span> // do not need to filter<a name="line.232"></a>
<span class="sourceLineNo">233</span> return result;<a name="line.233"></a>
<span class="sourceLineNo">234</span> }<a name="line.234"></a>
<span class="sourceLineNo">235</span> // not the same row<a name="line.235"></a>
<span class="sourceLineNo">236</span> if (!PrivateCellUtil.matchingRows(keepCellsAfter, result.getRow(), 0, result.getRow().length)) {<a name="line.236"></a>
<span class="sourceLineNo">237</span> return result;<a name="line.237"></a>
<span class="sourceLineNo">238</span> }<a name="line.238"></a>
<span class="sourceLineNo">239</span> Cell[] rawCells = result.rawCells();<a name="line.239"></a>
<span class="sourceLineNo">240</span> int index = Arrays.binarySearch(rawCells, keepCellsAfter,<a name="line.240"></a>
<span class="sourceLineNo">241</span> CellComparator.getInstance()::compareWithoutRow);<a name="line.241"></a>
<span class="sourceLineNo">242</span> if (index &lt; 0) {<a name="line.242"></a>
<span class="sourceLineNo">243</span> index = -index - 1;<a name="line.243"></a>
<span class="sourceLineNo">244</span> } else {<a name="line.244"></a>
<span class="sourceLineNo">245</span> index++;<a name="line.245"></a>
<span class="sourceLineNo">246</span> }<a name="line.246"></a>
<span class="sourceLineNo">247</span> if (index == 0) {<a name="line.247"></a>
<span class="sourceLineNo">248</span> return result;<a name="line.248"></a>
<span class="sourceLineNo">249</span> }<a name="line.249"></a>
<span class="sourceLineNo">250</span> if (index == rawCells.length) {<a name="line.250"></a>
<span class="sourceLineNo">251</span> return null;<a name="line.251"></a>
<span class="sourceLineNo">252</span> }<a name="line.252"></a>
<span class="sourceLineNo">253</span> return Result.create(Arrays.copyOfRange(rawCells, index, rawCells.length), null,<a name="line.253"></a>
<span class="sourceLineNo">254</span> result.isStale(), result.mayHaveMoreCellsInRow());<a name="line.254"></a>
<span class="sourceLineNo">255</span> }<a name="line.255"></a>
<span class="sourceLineNo">256</span><a name="line.256"></a>
<span class="sourceLineNo">257</span> // Add a delta to avoid timeout immediately after a retry sleeping.<a name="line.257"></a>
<span class="sourceLineNo">258</span> static final long SLEEP_DELTA_NS = TimeUnit.MILLISECONDS.toNanos(1);<a name="line.258"></a>
<span class="sourceLineNo">259</span><a name="line.259"></a>
<span class="sourceLineNo">260</span> static Get toCheckExistenceOnly(Get get) {<a name="line.260"></a>
<span class="sourceLineNo">261</span> if (get.isCheckExistenceOnly()) {<a name="line.261"></a>
<span class="sourceLineNo">262</span> return get;<a name="line.262"></a>
<span class="sourceLineNo">263</span> }<a name="line.263"></a>
<span class="sourceLineNo">264</span> return ReflectionUtils.newInstance(get.getClass(), get).setCheckExistenceOnly(true);<a name="line.264"></a>
<span class="sourceLineNo">265</span> }<a name="line.265"></a>
<span class="sourceLineNo">266</span><a name="line.266"></a>
<span class="sourceLineNo">267</span> static List&lt;Get&gt; toCheckExistenceOnly(List&lt;Get&gt; gets) {<a name="line.267"></a>
<span class="sourceLineNo">268</span> return gets.stream().map(ConnectionUtils::toCheckExistenceOnly).collect(toList());<a name="line.268"></a>
<span class="sourceLineNo">269</span> }<a name="line.269"></a>
<span class="sourceLineNo">270</span><a name="line.270"></a>
<span class="sourceLineNo">271</span> static RegionLocateType getLocateType(Scan scan) {<a name="line.271"></a>
<span class="sourceLineNo">272</span> if (scan.isReversed()) {<a name="line.272"></a>
<span class="sourceLineNo">273</span> if (isEmptyStartRow(scan.getStartRow())) {<a name="line.273"></a>
<span class="sourceLineNo">274</span> return RegionLocateType.BEFORE;<a name="line.274"></a>
<span class="sourceLineNo">275</span> } else {<a name="line.275"></a>
<span class="sourceLineNo">276</span> return scan.includeStartRow() ? RegionLocateType.CURRENT : RegionLocateType.BEFORE;<a name="line.276"></a>
<span class="sourceLineNo">277</span> }<a name="line.277"></a>
<span class="sourceLineNo">278</span> } else {<a name="line.278"></a>
<span class="sourceLineNo">279</span> return scan.includeStartRow() ? RegionLocateType.CURRENT : RegionLocateType.AFTER;<a name="line.279"></a>
<span class="sourceLineNo">280</span> }<a name="line.280"></a>
<span class="sourceLineNo">281</span> }<a name="line.281"></a>
<span class="sourceLineNo">282</span><a name="line.282"></a>
<span class="sourceLineNo">283</span> static boolean noMoreResultsForScan(Scan scan, RegionInfo info) {<a name="line.283"></a>
<span class="sourceLineNo">284</span> if (isEmptyStopRow(info.getEndKey())) {<a name="line.284"></a>
<span class="sourceLineNo">285</span> return true;<a name="line.285"></a>
<span class="sourceLineNo">286</span> }<a name="line.286"></a>
<span class="sourceLineNo">287</span> if (isEmptyStopRow(scan.getStopRow())) {<a name="line.287"></a>
<span class="sourceLineNo">288</span> return false;<a name="line.288"></a>
<span class="sourceLineNo">289</span> }<a name="line.289"></a>
<span class="sourceLineNo">290</span> int c = Bytes.compareTo(info.getEndKey(), scan.getStopRow());<a name="line.290"></a>
<span class="sourceLineNo">291</span> // 1. if our stop row is less than the endKey of the region<a name="line.291"></a>
<span class="sourceLineNo">292</span> // 2. if our stop row is equal to the endKey of the region and we do not include the stop row<a name="line.292"></a>
<span class="sourceLineNo">293</span> // for scan.<a name="line.293"></a>
<span class="sourceLineNo">294</span> return c &gt; 0 || (c == 0 &amp;&amp; !scan.includeStopRow());<a name="line.294"></a>
<span class="sourceLineNo">295</span> }<a name="line.295"></a>
<span class="sourceLineNo">296</span><a name="line.296"></a>
<span class="sourceLineNo">297</span> static boolean noMoreResultsForReverseScan(Scan scan, RegionInfo info) {<a name="line.297"></a>
<span class="sourceLineNo">298</span> if (isEmptyStartRow(info.getStartKey())) {<a name="line.298"></a>
<span class="sourceLineNo">299</span> return true;<a name="line.299"></a>
<span class="sourceLineNo">300</span> }<a name="line.300"></a>
<span class="sourceLineNo">301</span> if (isEmptyStopRow(scan.getStopRow())) {<a name="line.301"></a>
<span class="sourceLineNo">302</span> return false;<a name="line.302"></a>
<span class="sourceLineNo">303</span> }<a name="line.303"></a>
<span class="sourceLineNo">304</span> // no need to test the inclusive of the stop row as the start key of a region is included in<a name="line.304"></a>
<span class="sourceLineNo">305</span> // the region.<a name="line.305"></a>
<span class="sourceLineNo">306</span> return Bytes.compareTo(info.getStartKey(), scan.getStopRow()) &lt;= 0;<a name="line.306"></a>
<span class="sourceLineNo">307</span> }<a name="line.307"></a>
<span class="sourceLineNo">308</span><a name="line.308"></a>
<span class="sourceLineNo">309</span> public static ScanResultCache createScanResultCache(Scan scan) {<a name="line.309"></a>
<span class="sourceLineNo">310</span> if (scan.getAllowPartialResults()) {<a name="line.310"></a>
<span class="sourceLineNo">311</span> return new AllowPartialScanResultCache();<a name="line.311"></a>
<span class="sourceLineNo">312</span> } else if (scan.getBatch() &gt; 0) {<a name="line.312"></a>
<span class="sourceLineNo">313</span> return new BatchScanResultCache(scan.getBatch());<a name="line.313"></a>
<span class="sourceLineNo">314</span> } else {<a name="line.314"></a>
<span class="sourceLineNo">315</span> return new CompleteScanResultCache();<a name="line.315"></a>
<span class="sourceLineNo">316</span> }<a name="line.316"></a>
<span class="sourceLineNo">317</span> }<a name="line.317"></a>
<span class="sourceLineNo">318</span><a name="line.318"></a>
<span class="sourceLineNo">319</span> private static final String MY_ADDRESS = getMyAddress();<a name="line.319"></a>
<span class="sourceLineNo">320</span><a name="line.320"></a>
<span class="sourceLineNo">321</span> private static String getMyAddress() {<a name="line.321"></a>
<span class="sourceLineNo">322</span> try {<a name="line.322"></a>
<span class="sourceLineNo">323</span> return DNS.getDefaultHost("default", "default");<a name="line.323"></a>
<span class="sourceLineNo">324</span> } catch (UnknownHostException uhe) {<a name="line.324"></a>
<span class="sourceLineNo">325</span> LOG.error("cannot determine my address", uhe);<a name="line.325"></a>
<span class="sourceLineNo">326</span> return null;<a name="line.326"></a>
<span class="sourceLineNo">327</span> }<a name="line.327"></a>
<span class="sourceLineNo">328</span> }<a name="line.328"></a>
<span class="sourceLineNo">329</span><a name="line.329"></a>
<span class="sourceLineNo">330</span> static boolean isRemote(String host) {<a name="line.330"></a>
<span class="sourceLineNo">331</span> return !host.equalsIgnoreCase(MY_ADDRESS);<a name="line.331"></a>
<span class="sourceLineNo">332</span> }<a name="line.332"></a>
<span class="sourceLineNo">333</span><a name="line.333"></a>
<span class="sourceLineNo">334</span> static void incRPCCallsMetrics(ScanMetrics scanMetrics, boolean isRegionServerRemote) {<a name="line.334"></a>
<span class="sourceLineNo">335</span> if (scanMetrics == null) {<a name="line.335"></a>
<span class="sourceLineNo">336</span> return;<a name="line.336"></a>
<span class="sourceLineNo">337</span> }<a name="line.337"></a>
<span class="sourceLineNo">338</span> scanMetrics.countOfRPCcalls.incrementAndGet();<a name="line.338"></a>
<span class="sourceLineNo">339</span> if (isRegionServerRemote) {<a name="line.339"></a>
<span class="sourceLineNo">340</span> scanMetrics.countOfRemoteRPCcalls.incrementAndGet();<a name="line.340"></a>
<span class="sourceLineNo">341</span> }<a name="line.341"></a>
<span class="sourceLineNo">342</span> }<a name="line.342"></a>
<span class="sourceLineNo">343</span><a name="line.343"></a>
<span class="sourceLineNo">344</span> static void incRPCRetriesMetrics(ScanMetrics scanMetrics, boolean isRegionServerRemote) {<a name="line.344"></a>
<span class="sourceLineNo">345</span> if (scanMetrics == null) {<a name="line.345"></a>
<span class="sourceLineNo">346</span> return;<a name="line.346"></a>
<span class="sourceLineNo">347</span> }<a name="line.347"></a>
<span class="sourceLineNo">348</span> scanMetrics.countOfRPCRetries.incrementAndGet();<a name="line.348"></a>
<span class="sourceLineNo">349</span> if (isRegionServerRemote) {<a name="line.349"></a>
<span class="sourceLineNo">350</span> scanMetrics.countOfRemoteRPCRetries.incrementAndGet();<a name="line.350"></a>
<span class="sourceLineNo">351</span> }<a name="line.351"></a>
<span class="sourceLineNo">352</span> }<a name="line.352"></a>
<span class="sourceLineNo">353</span><a name="line.353"></a>
<span class="sourceLineNo">354</span> static void updateResultsMetrics(ScanMetrics scanMetrics, Result[] rrs,<a name="line.354"></a>
<span class="sourceLineNo">355</span> boolean isRegionServerRemote) {<a name="line.355"></a>
<span class="sourceLineNo">356</span> if (scanMetrics == null || rrs == null || rrs.length == 0) {<a name="line.356"></a>
<span class="sourceLineNo">357</span> return;<a name="line.357"></a>
<span class="sourceLineNo">358</span> }<a name="line.358"></a>
<span class="sourceLineNo">359</span> long resultSize = 0;<a name="line.359"></a>
<span class="sourceLineNo">360</span> for (Result rr : rrs) {<a name="line.360"></a>
<span class="sourceLineNo">361</span> for (Cell cell : rr.rawCells()) {<a name="line.361"></a>
<span class="sourceLineNo">362</span> resultSize += PrivateCellUtil.estimatedSerializedSizeOf(cell);<a name="line.362"></a>
<span class="sourceLineNo">363</span> }<a name="line.363"></a>
<span class="sourceLineNo">364</span> }<a name="line.364"></a>
<span class="sourceLineNo">365</span> scanMetrics.countOfBytesInResults.addAndGet(resultSize);<a name="line.365"></a>
<span class="sourceLineNo">366</span> if (isRegionServerRemote) {<a name="line.366"></a>
<span class="sourceLineNo">367</span> scanMetrics.countOfBytesInRemoteResults.addAndGet(resultSize);<a name="line.367"></a>
<span class="sourceLineNo">368</span> }<a name="line.368"></a>
<span class="sourceLineNo">369</span> }<a name="line.369"></a>
<span class="sourceLineNo">370</span><a name="line.370"></a>
<span class="sourceLineNo">371</span> /**<a name="line.371"></a>
<span class="sourceLineNo">372</span> * Use the scan metrics returned by the server to add to the identically named counters in the<a name="line.372"></a>
<span class="sourceLineNo">373</span> * client side metrics. If a counter does not exist with the same name as the server side metric,<a name="line.373"></a>
<span class="sourceLineNo">374</span> * the attempt to increase the counter will fail.<a name="line.374"></a>
<span class="sourceLineNo">375</span> */<a name="line.375"></a>
<span class="sourceLineNo">376</span> static void updateServerSideMetrics(ScanMetrics scanMetrics, ScanResponse response) {<a name="line.376"></a>
<span class="sourceLineNo">377</span> if (scanMetrics == null || response == null || !response.hasScanMetrics()) {<a name="line.377"></a>
<span class="sourceLineNo">378</span> return;<a name="line.378"></a>
<span class="sourceLineNo">379</span> }<a name="line.379"></a>
<span class="sourceLineNo">380</span> ResponseConverter.getScanMetrics(response).forEach(scanMetrics::addToCounter);<a name="line.380"></a>
<span class="sourceLineNo">381</span> }<a name="line.381"></a>
<span class="sourceLineNo">382</span><a name="line.382"></a>
<span class="sourceLineNo">383</span> static void incRegionCountMetrics(ScanMetrics scanMetrics) {<a name="line.383"></a>
<span class="sourceLineNo">384</span> if (scanMetrics == null) {<a name="line.384"></a>
<span class="sourceLineNo">385</span> return;<a name="line.385"></a>
<span class="sourceLineNo">386</span> }<a name="line.386"></a>
<span class="sourceLineNo">387</span> scanMetrics.countOfRegions.incrementAndGet();<a name="line.387"></a>
<span class="sourceLineNo">388</span> }<a name="line.388"></a>
<span class="sourceLineNo">389</span><a name="line.389"></a>
<span class="sourceLineNo">390</span> /**<a name="line.390"></a>
<span class="sourceLineNo">391</span> * Connect the two futures, if the src future is done, then mark the dst future as done. And if<a name="line.391"></a>
<span class="sourceLineNo">392</span> * the dst future is done, then cancel the src future. This is used for timeline consistent read.<a name="line.392"></a>
<span class="sourceLineNo">393</span> * &lt;p/&gt;<a name="line.393"></a>
<span class="sourceLineNo">394</span> * Pass empty metrics if you want to link the primary future and the dst future so we will not<a name="line.394"></a>
<span class="sourceLineNo">395</span> * increase the hedge read related metrics.<a name="line.395"></a>
<span class="sourceLineNo">396</span> */<a name="line.396"></a>
<span class="sourceLineNo">397</span> private static &lt;T&gt; void connect(CompletableFuture&lt;T&gt; srcFuture, CompletableFuture&lt;T&gt; dstFuture,<a name="line.397"></a>
<span class="sourceLineNo">398</span> Optional&lt;MetricsConnection&gt; metrics) {<a name="line.398"></a>
<span class="sourceLineNo">399</span> addListener(srcFuture, (r, e) -&gt; {<a name="line.399"></a>
<span class="sourceLineNo">400</span> if (e != null) {<a name="line.400"></a>
<span class="sourceLineNo">401</span> dstFuture.completeExceptionally(e);<a name="line.401"></a>
<span class="sourceLineNo">402</span> } else {<a name="line.402"></a>
<span class="sourceLineNo">403</span> if (dstFuture.complete(r)) {<a name="line.403"></a>
<span class="sourceLineNo">404</span> metrics.ifPresent(MetricsConnection::incrHedgedReadWin);<a name="line.404"></a>
<span class="sourceLineNo">405</span> }<a name="line.405"></a>
<span class="sourceLineNo">406</span> }<a name="line.406"></a>
<span class="sourceLineNo">407</span> });<a name="line.407"></a>
<span class="sourceLineNo">408</span> // The cancellation may be a dummy one as the dstFuture may be completed by this srcFuture.<a name="line.408"></a>
<span class="sourceLineNo">409</span> // Notice that this is a bit tricky, as the execution chain maybe 'complete src -&gt; complete dst<a name="line.409"></a>
<span class="sourceLineNo">410</span> // -&gt; cancel src', for now it seems to be fine, as the will use CAS to set the result first in<a name="line.410"></a>
<span class="sourceLineNo">411</span> // CompletableFuture. If later this causes problems, we could use whenCompleteAsync to break the<a name="line.411"></a>
<span class="sourceLineNo">412</span> // tie.<a name="line.412"></a>
<span class="sourceLineNo">413</span> addListener(dstFuture, (r, e) -&gt; srcFuture.cancel(false));<a name="line.413"></a>
<span class="sourceLineNo">414</span> }<a name="line.414"></a>
<span class="sourceLineNo">415</span><a name="line.415"></a>
<span class="sourceLineNo">416</span> private static &lt;T&gt; void sendRequestsToSecondaryReplicas(<a name="line.416"></a>
<span class="sourceLineNo">417</span> Function&lt;Integer, CompletableFuture&lt;T&gt;&gt; requestReplica, RegionLocations locs,<a name="line.417"></a>
<span class="sourceLineNo">418</span> CompletableFuture&lt;T&gt; future, Optional&lt;MetricsConnection&gt; metrics) {<a name="line.418"></a>
<span class="sourceLineNo">419</span> if (future.isDone()) {<a name="line.419"></a>
<span class="sourceLineNo">420</span> // do not send requests to secondary replicas if the future is done, i.e, the primary request<a name="line.420"></a>
<span class="sourceLineNo">421</span> // has already been finished.<a name="line.421"></a>
<span class="sourceLineNo">422</span> return;<a name="line.422"></a>
<span class="sourceLineNo">423</span> }<a name="line.423"></a>
<span class="sourceLineNo">424</span> for (int replicaId = 1, n = locs.size(); replicaId &lt; n; replicaId++) {<a name="line.424"></a>
<span class="sourceLineNo">425</span> CompletableFuture&lt;T&gt; secondaryFuture = requestReplica.apply(replicaId);<a name="line.425"></a>
<span class="sourceLineNo">426</span> metrics.ifPresent(MetricsConnection::incrHedgedReadOps);<a name="line.426"></a>
<span class="sourceLineNo">427</span> connect(secondaryFuture, future, metrics);<a name="line.427"></a>
<span class="sourceLineNo">428</span> }<a name="line.428"></a>
<span class="sourceLineNo">429</span> }<a name="line.429"></a>
<span class="sourceLineNo">430</span><a name="line.430"></a>
<span class="sourceLineNo">431</span> static &lt;T&gt; CompletableFuture&lt;T&gt; timelineConsistentRead(AsyncRegionLocator locator,<a name="line.431"></a>
<span class="sourceLineNo">432</span> TableName tableName, Query query, byte[] row, RegionLocateType locateType,<a name="line.432"></a>
<span class="sourceLineNo">433</span> Function&lt;Integer, CompletableFuture&lt;T&gt;&gt; requestReplica, long rpcTimeoutNs,<a name="line.433"></a>
<span class="sourceLineNo">434</span> long primaryCallTimeoutNs, Timer retryTimer, Optional&lt;MetricsConnection&gt; metrics) {<a name="line.434"></a>
<span class="sourceLineNo">435</span> if (query.getConsistency() != Consistency.TIMELINE) {<a name="line.435"></a>
<span class="sourceLineNo">436</span> return requestReplica.apply(RegionReplicaUtil.DEFAULT_REPLICA_ID);<a name="line.436"></a>
<span class="sourceLineNo">437</span> }<a name="line.437"></a>
<span class="sourceLineNo">438</span> // user specifies a replica id explicitly, just send request to the specific replica<a name="line.438"></a>
<span class="sourceLineNo">439</span> if (query.getReplicaId() &gt;= 0) {<a name="line.439"></a>
<span class="sourceLineNo">440</span> return requestReplica.apply(query.getReplicaId());<a name="line.440"></a>
<span class="sourceLineNo">441</span> }<a name="line.441"></a>
<span class="sourceLineNo">442</span> // Timeline consistent read, where we may send requests to other region replicas<a name="line.442"></a>
<span class="sourceLineNo">443</span> CompletableFuture&lt;T&gt; primaryFuture = requestReplica.apply(RegionReplicaUtil.DEFAULT_REPLICA_ID);<a name="line.443"></a>
<span class="sourceLineNo">444</span> CompletableFuture&lt;T&gt; future = new CompletableFuture&lt;&gt;();<a name="line.444"></a>
<span class="sourceLineNo">445</span> connect(primaryFuture, future, Optional.empty());<a name="line.445"></a>
<span class="sourceLineNo">446</span> long startNs = System.nanoTime();<a name="line.446"></a>
<span class="sourceLineNo">447</span> // after the getRegionLocations, all the locations for the replicas of this region should have<a name="line.447"></a>
<span class="sourceLineNo">448</span> // been cached, so it is not big deal to locate them again when actually sending requests to<a name="line.448"></a>
<span class="sourceLineNo">449</span> // these replicas.<a name="line.449"></a>
<span class="sourceLineNo">450</span> addListener(locator.getRegionLocations(tableName, row, locateType, false, rpcTimeoutNs),<a name="line.450"></a>
<span class="sourceLineNo">451</span> (locs, error) -&gt; {<a name="line.451"></a>
<span class="sourceLineNo">452</span> if (error != null) {<a name="line.452"></a>
<span class="sourceLineNo">453</span> LOG.warn(<a name="line.453"></a>
<span class="sourceLineNo">454</span> "Failed to locate all the replicas for table={}, row='{}', locateType={}"<a name="line.454"></a>
<span class="sourceLineNo">455</span> + " give up timeline consistent read",<a name="line.455"></a>
<span class="sourceLineNo">456</span> tableName, Bytes.toStringBinary(row), locateType, error);<a name="line.456"></a>
<span class="sourceLineNo">457</span> return;<a name="line.457"></a>
<span class="sourceLineNo">458</span> }<a name="line.458"></a>
<span class="sourceLineNo">459</span> if (locs.size() &lt;= 1) {<a name="line.459"></a>
<span class="sourceLineNo">460</span> LOG.warn(<a name="line.460"></a>
<span class="sourceLineNo">461</span> "There are no secondary replicas for region {}, give up timeline consistent read",<a name="line.461"></a>
<span class="sourceLineNo">462</span> locs.getDefaultRegionLocation().getRegion());<a name="line.462"></a>
<span class="sourceLineNo">463</span> return;<a name="line.463"></a>
<span class="sourceLineNo">464</span> }<a name="line.464"></a>
<span class="sourceLineNo">465</span> long delayNs = primaryCallTimeoutNs - (System.nanoTime() - startNs);<a name="line.465"></a>
<span class="sourceLineNo">466</span> if (delayNs &lt;= 0) {<a name="line.466"></a>
<span class="sourceLineNo">467</span> sendRequestsToSecondaryReplicas(requestReplica, locs, future, metrics);<a name="line.467"></a>
<span class="sourceLineNo">468</span> } else {<a name="line.468"></a>
<span class="sourceLineNo">469</span> retryTimer.newTimeout(<a name="line.469"></a>
<span class="sourceLineNo">470</span> timeout -&gt; sendRequestsToSecondaryReplicas(requestReplica, locs, future, metrics),<a name="line.470"></a>
<span class="sourceLineNo">471</span> delayNs, TimeUnit.NANOSECONDS);<a name="line.471"></a>
<span class="sourceLineNo">472</span> }<a name="line.472"></a>
<span class="sourceLineNo">473</span> });<a name="line.473"></a>
<span class="sourceLineNo">474</span> return future;<a name="line.474"></a>
<span class="sourceLineNo">475</span> }<a name="line.475"></a>
<span class="sourceLineNo">476</span><a name="line.476"></a>
<span class="sourceLineNo">477</span> // validate for well-formedness<a name="line.477"></a>
<span class="sourceLineNo">478</span> static void validatePut(Put put, int maxKeyValueSize) {<a name="line.478"></a>
<span class="sourceLineNo">479</span> if (put.isEmpty()) {<a name="line.479"></a>
<span class="sourceLineNo">480</span> throw new IllegalArgumentException("No columns to insert");<a name="line.480"></a>
<span class="sourceLineNo">481</span> }<a name="line.481"></a>
<span class="sourceLineNo">482</span> if (maxKeyValueSize &gt; 0) {<a name="line.482"></a>
<span class="sourceLineNo">483</span> for (List&lt;Cell&gt; list : put.getFamilyCellMap().values()) {<a name="line.483"></a>
<span class="sourceLineNo">484</span> for (Cell cell : list) {<a name="line.484"></a>
<span class="sourceLineNo">485</span> if (cell.getSerializedSize() &gt; maxKeyValueSize) {<a name="line.485"></a>
<span class="sourceLineNo">486</span> throw new IllegalArgumentException("KeyValue size too large");<a name="line.486"></a>
<span class="sourceLineNo">487</span> }<a name="line.487"></a>
<span class="sourceLineNo">488</span> }<a name="line.488"></a>
<span class="sourceLineNo">489</span> }<a name="line.489"></a>
<span class="sourceLineNo">490</span> }<a name="line.490"></a>
<span class="sourceLineNo">491</span> }<a name="line.491"></a>
<span class="sourceLineNo">492</span><a name="line.492"></a>
<span class="sourceLineNo">493</span> static void validatePutsInRowMutations(RowMutations rowMutations, int maxKeyValueSize) {<a name="line.493"></a>
<span class="sourceLineNo">494</span> for (Mutation mutation : rowMutations.getMutations()) {<a name="line.494"></a>
<span class="sourceLineNo">495</span> if (mutation instanceof Put) {<a name="line.495"></a>
<span class="sourceLineNo">496</span> validatePut((Put) mutation, maxKeyValueSize);<a name="line.496"></a>
<span class="sourceLineNo">497</span> }<a name="line.497"></a>
<span class="sourceLineNo">498</span> }<a name="line.498"></a>
<span class="sourceLineNo">499</span> }<a name="line.499"></a>
<span class="sourceLineNo">500</span><a name="line.500"></a>
<span class="sourceLineNo">501</span> /**<a name="line.501"></a>
<span class="sourceLineNo">502</span> * Select the priority for the rpc call.<a name="line.502"></a>
<span class="sourceLineNo">503</span> * &lt;p/&gt;<a name="line.503"></a>
<span class="sourceLineNo">504</span> * The rules are:<a name="line.504"></a>
<span class="sourceLineNo">505</span> * &lt;ol&gt;<a name="line.505"></a>
<span class="sourceLineNo">506</span> * &lt;li&gt;If user set a priority explicitly, then just use it.&lt;/li&gt;<a name="line.506"></a>
<span class="sourceLineNo">507</span> * &lt;li&gt;For system table, use {@link HConstants#SYSTEMTABLE_QOS}.&lt;/li&gt;<a name="line.507"></a>
<span class="sourceLineNo">508</span> * &lt;li&gt;For other tables, use {@link HConstants#NORMAL_QOS}.&lt;/li&gt;<a name="line.508"></a>
<span class="sourceLineNo">509</span> * &lt;/ol&gt;<a name="line.509"></a>
<span class="sourceLineNo">510</span> * @param priority the priority set by user, can be {@link HConstants#PRIORITY_UNSET}.<a name="line.510"></a>
<span class="sourceLineNo">511</span> * @param tableName the table we operate on<a name="line.511"></a>
<span class="sourceLineNo">512</span> */<a name="line.512"></a>
<span class="sourceLineNo">513</span> static int calcPriority(int priority, TableName tableName) {<a name="line.513"></a>
<span class="sourceLineNo">514</span> if (priority != HConstants.PRIORITY_UNSET) {<a name="line.514"></a>
<span class="sourceLineNo">515</span> return priority;<a name="line.515"></a>
<span class="sourceLineNo">516</span> } else {<a name="line.516"></a>
<span class="sourceLineNo">517</span> return getPriority(tableName);<a name="line.517"></a>
<span class="sourceLineNo">518</span> }<a name="line.518"></a>
<span class="sourceLineNo">519</span> }<a name="line.519"></a>
<span class="sourceLineNo">520</span><a name="line.520"></a>
<span class="sourceLineNo">521</span> static int getPriority(TableName tableName) {<a name="line.521"></a>
<span class="sourceLineNo">522</span> if (tableName.isSystemTable()) {<a name="line.522"></a>
<span class="sourceLineNo">523</span> return HConstants.SYSTEMTABLE_QOS;<a name="line.523"></a>
<span class="sourceLineNo">524</span> } else {<a name="line.524"></a>
<span class="sourceLineNo">525</span> return HConstants.NORMAL_QOS;<a name="line.525"></a>
<span class="sourceLineNo">526</span> }<a name="line.526"></a>
<span class="sourceLineNo">527</span> }<a name="line.527"></a>
<span class="sourceLineNo">528</span><a name="line.528"></a>
<span class="sourceLineNo">529</span> static &lt;T&gt; CompletableFuture&lt;T&gt; getOrFetch(AtomicReference&lt;T&gt; cacheRef,<a name="line.529"></a>
<span class="sourceLineNo">530</span> AtomicReference&lt;CompletableFuture&lt;T&gt;&gt; futureRef, boolean reload,<a name="line.530"></a>
<span class="sourceLineNo">531</span> Supplier&lt;CompletableFuture&lt;T&gt;&gt; fetch, Predicate&lt;T&gt; validator, String type) {<a name="line.531"></a>
<span class="sourceLineNo">532</span> for (;;) {<a name="line.532"></a>
<span class="sourceLineNo">533</span> if (!reload) {<a name="line.533"></a>
<span class="sourceLineNo">534</span> T value = cacheRef.get();<a name="line.534"></a>
<span class="sourceLineNo">535</span> if (value != null &amp;&amp; validator.test(value)) {<a name="line.535"></a>
<span class="sourceLineNo">536</span> return CompletableFuture.completedFuture(value);<a name="line.536"></a>
<span class="sourceLineNo">537</span> }<a name="line.537"></a>
<span class="sourceLineNo">538</span> }<a name="line.538"></a>
<span class="sourceLineNo">539</span> LOG.trace("{} cache is null, try fetching from registry", type);<a name="line.539"></a>
<span class="sourceLineNo">540</span> if (futureRef.compareAndSet(null, new CompletableFuture&lt;&gt;())) {<a name="line.540"></a>
<span class="sourceLineNo">541</span> LOG.debug("Start fetching {} from registry", type);<a name="line.541"></a>
<span class="sourceLineNo">542</span> CompletableFuture&lt;T&gt; future = futureRef.get();<a name="line.542"></a>
<span class="sourceLineNo">543</span> addListener(fetch.get(), (value, error) -&gt; {<a name="line.543"></a>
<span class="sourceLineNo">544</span> if (error != null) {<a name="line.544"></a>
<span class="sourceLineNo">545</span> LOG.debug("Failed to fetch {} from registry", type, error);<a name="line.545"></a>
<span class="sourceLineNo">546</span> futureRef.getAndSet(null).completeExceptionally(error);<a name="line.546"></a>
<span class="sourceLineNo">547</span> return;<a name="line.547"></a>
<span class="sourceLineNo">548</span> }<a name="line.548"></a>
<span class="sourceLineNo">549</span> LOG.debug("The fetched {} is {}", type, value);<a name="line.549"></a>
<span class="sourceLineNo">550</span> // Here we update cache before reset future, so it is possible that someone can get a<a name="line.550"></a>
<span class="sourceLineNo">551</span> // stale value. Consider this:<a name="line.551"></a>
<span class="sourceLineNo">552</span> // 1. update cacheRef<a name="line.552"></a>
<span class="sourceLineNo">553</span> // 2. someone clears the cache and relocates again<a name="line.553"></a>
<span class="sourceLineNo">554</span> // 3. the futureRef is not null so the old future is used.<a name="line.554"></a>
<span class="sourceLineNo">555</span> // 4. we clear futureRef and complete the future in it with the value being<a name="line.555"></a>
<span class="sourceLineNo">556</span> // cleared in step 2.<a name="line.556"></a>
<span class="sourceLineNo">557</span> // But we do not think it is a big deal as it rarely happens, and even if it happens, the<a name="line.557"></a>
<span class="sourceLineNo">558</span> // caller will retry again later, no correctness problems.<a name="line.558"></a>
<span class="sourceLineNo">559</span> cacheRef.set(value);<a name="line.559"></a>
<span class="sourceLineNo">560</span> futureRef.set(null);<a name="line.560"></a>
<span class="sourceLineNo">561</span> future.complete(value);<a name="line.561"></a>
<span class="sourceLineNo">562</span> });<a name="line.562"></a>
<span class="sourceLineNo">563</span> return future;<a name="line.563"></a>
<span class="sourceLineNo">564</span> } else {<a name="line.564"></a>
<span class="sourceLineNo">565</span> CompletableFuture&lt;T&gt; future = futureRef.get();<a name="line.565"></a>
<span class="sourceLineNo">566</span> if (future != null) {<a name="line.566"></a>
<span class="sourceLineNo">567</span> return future;<a name="line.567"></a>
<span class="sourceLineNo">568</span> }<a name="line.568"></a>
<span class="sourceLineNo">569</span> }<a name="line.569"></a>
<span class="sourceLineNo">570</span> }<a name="line.570"></a>
<span class="sourceLineNo">571</span> }<a name="line.571"></a>
<span class="sourceLineNo">572</span><a name="line.572"></a>
<span class="sourceLineNo">573</span> static void updateStats(Optional&lt;ServerStatisticTracker&gt; optStats,<a name="line.573"></a>
<span class="sourceLineNo">574</span> Optional&lt;MetricsConnection&gt; optMetrics, ServerName serverName, MultiResponse resp) {<a name="line.574"></a>
<span class="sourceLineNo">575</span> if (!optStats.isPresent() &amp;&amp; !optMetrics.isPresent()) {<a name="line.575"></a>
<span class="sourceLineNo">576</span> // ServerStatisticTracker and MetricsConnection are both not present, just return<a name="line.576"></a>
<span class="sourceLineNo">577</span> return;<a name="line.577"></a>
<span class="sourceLineNo">578</span> }<a name="line.578"></a>
<span class="sourceLineNo">579</span> resp.getResults().forEach((regionName, regionResult) -&gt; {<a name="line.579"></a>
<span class="sourceLineNo">580</span> ClientProtos.RegionLoadStats stat = regionResult.getStat();<a name="line.580"></a>
<span class="sourceLineNo">581</span> if (stat == null) {<a name="line.581"></a>
<span class="sourceLineNo">582</span> if (LOG.isDebugEnabled()) {<a name="line.582"></a>
<span class="sourceLineNo">583</span> LOG.debug("No ClientProtos.RegionLoadStats found for server={}, region={}", serverName,<a name="line.583"></a>
<span class="sourceLineNo">584</span> Bytes.toStringBinary(regionName));<a name="line.584"></a>
<span class="sourceLineNo">585</span> }<a name="line.585"></a>
<span class="sourceLineNo">586</span> return;<a name="line.586"></a>
<span class="sourceLineNo">587</span> }<a name="line.587"></a>
<span class="sourceLineNo">588</span> RegionLoadStats regionLoadStats = ProtobufUtil.createRegionLoadStats(stat);<a name="line.588"></a>
<span class="sourceLineNo">589</span> optStats.ifPresent(<a name="line.589"></a>
<span class="sourceLineNo">590</span> stats -&gt; ResultStatsUtil.updateStats(stats, serverName, regionName, regionLoadStats));<a name="line.590"></a>
<span class="sourceLineNo">591</span> optMetrics.ifPresent(<a name="line.591"></a>
<span class="sourceLineNo">592</span> metrics -&gt; ResultStatsUtil.updateStats(metrics, serverName, regionName, regionLoadStats));<a name="line.592"></a>
<span class="sourceLineNo">593</span> });<a name="line.593"></a>
<span class="sourceLineNo">594</span> }<a name="line.594"></a>
<span class="sourceLineNo">595</span><a name="line.595"></a>
<span class="sourceLineNo">596</span> @FunctionalInterface<a name="line.596"></a>
<span class="sourceLineNo">597</span> interface Converter&lt;D, I, S&gt; {<a name="line.597"></a>
<span class="sourceLineNo">598</span> D convert(I info, S src) throws IOException;<a name="line.598"></a>
<span class="sourceLineNo">599</span> }<a name="line.599"></a>
<span class="sourceLineNo">600</span><a name="line.600"></a>
<span class="sourceLineNo">601</span> @FunctionalInterface<a name="line.601"></a>
<span class="sourceLineNo">602</span> interface RpcCall&lt;RESP, REQ&gt; {<a name="line.602"></a>
<span class="sourceLineNo">603</span> void call(ClientService.Interface stub, HBaseRpcController controller, REQ req,<a name="line.603"></a>
<span class="sourceLineNo">604</span> RpcCallback&lt;RESP&gt; done);<a name="line.604"></a>
<span class="sourceLineNo">605</span> }<a name="line.605"></a>
<span class="sourceLineNo">606</span><a name="line.606"></a>
<span class="sourceLineNo">607</span> static &lt;REQ, PREQ, PRESP, RESP&gt; CompletableFuture&lt;RESP&gt; call(HBaseRpcController controller,<a name="line.607"></a>
<span class="sourceLineNo">608</span> HRegionLocation loc, ClientService.Interface stub, REQ req,<a name="line.608"></a>
<span class="sourceLineNo">609</span> Converter&lt;PREQ, byte[], REQ&gt; reqConvert, RpcCall&lt;PRESP, PREQ&gt; rpcCall,<a name="line.609"></a>
<span class="sourceLineNo">610</span> Converter&lt;RESP, HBaseRpcController, PRESP&gt; respConverter) {<a name="line.610"></a>
<span class="sourceLineNo">611</span> CompletableFuture&lt;RESP&gt; future = new CompletableFuture&lt;&gt;();<a name="line.611"></a>
<span class="sourceLineNo">612</span> try {<a name="line.612"></a>
<span class="sourceLineNo">613</span> rpcCall.call(stub, controller, reqConvert.convert(loc.getRegion().getRegionName(), req),<a name="line.613"></a>
<span class="sourceLineNo">614</span> new RpcCallback&lt;PRESP&gt;() {<a name="line.614"></a>
<span class="sourceLineNo">615</span><a name="line.615"></a>
<span class="sourceLineNo">616</span> @Override<a name="line.616"></a>
<span class="sourceLineNo">617</span> public void run(PRESP resp) {<a name="line.617"></a>
<span class="sourceLineNo">618</span> if (controller.failed()) {<a name="line.618"></a>
<span class="sourceLineNo">619</span> future.completeExceptionally(controller.getFailed());<a name="line.619"></a>
<span class="sourceLineNo">620</span> } else {<a name="line.620"></a>
<span class="sourceLineNo">621</span> try {<a name="line.621"></a>
<span class="sourceLineNo">622</span> future.complete(respConverter.convert(controller, resp));<a name="line.622"></a>
<span class="sourceLineNo">623</span> } catch (IOException e) {<a name="line.623"></a>
<span class="sourceLineNo">624</span> future.completeExceptionally(e);<a name="line.624"></a>
<span class="sourceLineNo">625</span> }<a name="line.625"></a>
<span class="sourceLineNo">626</span> }<a name="line.626"></a>
<span class="sourceLineNo">627</span> }<a name="line.627"></a>
<span class="sourceLineNo">628</span> });<a name="line.628"></a>
<span class="sourceLineNo">629</span> } catch (IOException e) {<a name="line.629"></a>
<span class="sourceLineNo">630</span> future.completeExceptionally(e);<a name="line.630"></a>
<span class="sourceLineNo">631</span> }<a name="line.631"></a>
<span class="sourceLineNo">632</span> return future;<a name="line.632"></a>
<span class="sourceLineNo">633</span> }<a name="line.633"></a>
<span class="sourceLineNo">634</span><a name="line.634"></a>
<span class="sourceLineNo">635</span> static void shutdownPool(ExecutorService pool) {<a name="line.635"></a>
<span class="sourceLineNo">636</span> pool.shutdown();<a name="line.636"></a>
<span class="sourceLineNo">637</span> try {<a name="line.637"></a>
<span class="sourceLineNo">638</span> if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {<a name="line.638"></a>
<span class="sourceLineNo">639</span> pool.shutdownNow();<a name="line.639"></a>
<span class="sourceLineNo">640</span> }<a name="line.640"></a>
<span class="sourceLineNo">641</span> } catch (InterruptedException e) {<a name="line.641"></a>
<span class="sourceLineNo">642</span> pool.shutdownNow();<a name="line.642"></a>
<span class="sourceLineNo">643</span> }<a name="line.643"></a>
<span class="sourceLineNo">644</span> }<a name="line.644"></a>
<span class="sourceLineNo">645</span><a name="line.645"></a>
<span class="sourceLineNo">646</span> static void setCoprocessorError(RpcController controller, Throwable error) {<a name="line.646"></a>
<span class="sourceLineNo">647</span> if (controller == null) {<a name="line.647"></a>
<span class="sourceLineNo">648</span> return;<a name="line.648"></a>
<span class="sourceLineNo">649</span> }<a name="line.649"></a>
<span class="sourceLineNo">650</span> if (controller instanceof ServerRpcController) {<a name="line.650"></a>
<span class="sourceLineNo">651</span> if (error instanceof IOException) {<a name="line.651"></a>
<span class="sourceLineNo">652</span> ((ServerRpcController) controller).setFailedOn((IOException) error);<a name="line.652"></a>
<span class="sourceLineNo">653</span> } else {<a name="line.653"></a>
<span class="sourceLineNo">654</span> ((ServerRpcController) controller).setFailedOn(new IOException(error));<a name="line.654"></a>
<span class="sourceLineNo">655</span> }<a name="line.655"></a>
<span class="sourceLineNo">656</span> } else if (controller instanceof ClientCoprocessorRpcController) {<a name="line.656"></a>
<span class="sourceLineNo">657</span> ((ClientCoprocessorRpcController) controller).setFailed(error);<a name="line.657"></a>
<span class="sourceLineNo">658</span> } else {<a name="line.658"></a>
<span class="sourceLineNo">659</span> controller.setFailed(error.toString());<a name="line.659"></a>
<span class="sourceLineNo">660</span> }<a name="line.660"></a>
<span class="sourceLineNo">661</span> }<a name="line.661"></a>
<span class="sourceLineNo">662</span>}<a name="line.662"></a>
</pre>
</div>
</body>
</html>