blob: d11014f3291ab012e7e98ad32539f66e55b308d5 [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 java.io.Closeable;<a name="line.20"></a>
<span class="sourceLineNo">021</span>import java.io.IOException;<a name="line.21"></a>
<span class="sourceLineNo">022</span>import java.util.Collections;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import java.util.List;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import java.util.Map;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import java.util.TreeMap;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import java.util.concurrent.TimeUnit;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import org.apache.commons.lang3.NotImplementedException;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import org.apache.hadoop.conf.Configuration;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import org.apache.hadoop.hbase.Cell;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import org.apache.hadoop.hbase.CompareOperator;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import org.apache.hadoop.hbase.TableName;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import org.apache.hadoop.hbase.client.coprocessor.Batch;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import org.apache.hadoop.hbase.filter.Filter;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import org.apache.hadoop.hbase.io.TimeRange;<a name="line.34"></a>
<span class="sourceLineNo">035</span>import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;<a name="line.35"></a>
<span class="sourceLineNo">036</span>import org.apache.hadoop.hbase.util.Bytes;<a name="line.36"></a>
<span class="sourceLineNo">037</span>import org.apache.yetus.audience.InterfaceAudience;<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.Descriptors;<a name="line.39"></a>
<span class="sourceLineNo">040</span>import org.apache.hbase.thirdparty.com.google.protobuf.Message;<a name="line.40"></a>
<span class="sourceLineNo">041</span>import org.apache.hbase.thirdparty.com.google.protobuf.Service;<a name="line.41"></a>
<span class="sourceLineNo">042</span>import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;<a name="line.42"></a>
<span class="sourceLineNo">043</span><a name="line.43"></a>
<span class="sourceLineNo">044</span>/**<a name="line.44"></a>
<span class="sourceLineNo">045</span> * Used to communicate with a single HBase table. Obtain an instance from a {@link Connection} and<a name="line.45"></a>
<span class="sourceLineNo">046</span> * call {@link #close()} afterwards.<a name="line.46"></a>
<span class="sourceLineNo">047</span> * &lt;p&gt;<a name="line.47"></a>
<span class="sourceLineNo">048</span> * &lt;code&gt;Table&lt;/code&gt; can be used to get, put, delete or scan data from a table.<a name="line.48"></a>
<span class="sourceLineNo">049</span> * @see ConnectionFactory<a name="line.49"></a>
<span class="sourceLineNo">050</span> * @see Connection<a name="line.50"></a>
<span class="sourceLineNo">051</span> * @see Admin<a name="line.51"></a>
<span class="sourceLineNo">052</span> * @see RegionLocator<a name="line.52"></a>
<span class="sourceLineNo">053</span> * @since 0.99.0<a name="line.53"></a>
<span class="sourceLineNo">054</span> */<a name="line.54"></a>
<span class="sourceLineNo">055</span>@InterfaceAudience.Public<a name="line.55"></a>
<span class="sourceLineNo">056</span>public interface Table extends Closeable {<a name="line.56"></a>
<span class="sourceLineNo">057</span> /**<a name="line.57"></a>
<span class="sourceLineNo">058</span> * Gets the fully qualified table name instance of this table.<a name="line.58"></a>
<span class="sourceLineNo">059</span> */<a name="line.59"></a>
<span class="sourceLineNo">060</span> TableName getName();<a name="line.60"></a>
<span class="sourceLineNo">061</span><a name="line.61"></a>
<span class="sourceLineNo">062</span> /**<a name="line.62"></a>
<span class="sourceLineNo">063</span> * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.<a name="line.63"></a>
<span class="sourceLineNo">064</span> * &lt;p&gt;<a name="line.64"></a>
<span class="sourceLineNo">065</span> * The reference returned is not a copy, so any change made to it will affect this instance.<a name="line.65"></a>
<span class="sourceLineNo">066</span> */<a name="line.66"></a>
<span class="sourceLineNo">067</span> Configuration getConfiguration();<a name="line.67"></a>
<span class="sourceLineNo">068</span><a name="line.68"></a>
<span class="sourceLineNo">069</span> /**<a name="line.69"></a>
<span class="sourceLineNo">070</span> * Gets the {@link org.apache.hadoop.hbase.client.TableDescriptor table descriptor} for this<a name="line.70"></a>
<span class="sourceLineNo">071</span> * table.<a name="line.71"></a>
<span class="sourceLineNo">072</span> * @throws java.io.IOException if a remote or network exception occurs.<a name="line.72"></a>
<span class="sourceLineNo">073</span> */<a name="line.73"></a>
<span class="sourceLineNo">074</span> TableDescriptor getDescriptor() throws IOException;<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> * Gets the {@link RegionLocator} for this table.<a name="line.77"></a>
<span class="sourceLineNo">078</span> */<a name="line.78"></a>
<span class="sourceLineNo">079</span> RegionLocator getRegionLocator() throws IOException;<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> * Test for the existence of columns in the table, as specified by the Get.<a name="line.82"></a>
<span class="sourceLineNo">083</span> * &lt;p&gt;<a name="line.83"></a>
<span class="sourceLineNo">084</span> * This will return true if the Get matches one or more keys, false if not.<a name="line.84"></a>
<span class="sourceLineNo">085</span> * &lt;p&gt;<a name="line.85"></a>
<span class="sourceLineNo">086</span> * This is a server-side call so it prevents any data from being transfered to the client.<a name="line.86"></a>
<span class="sourceLineNo">087</span> * @param get the Get<a name="line.87"></a>
<span class="sourceLineNo">088</span> * @return true if the specified Get matches one or more keys, false if not<a name="line.88"></a>
<span class="sourceLineNo">089</span> * @throws IOException e<a name="line.89"></a>
<span class="sourceLineNo">090</span> */<a name="line.90"></a>
<span class="sourceLineNo">091</span> default boolean exists(Get get) throws IOException {<a name="line.91"></a>
<span class="sourceLineNo">092</span> return exists(Collections.singletonList(get))[0];<a name="line.92"></a>
<span class="sourceLineNo">093</span> }<a name="line.93"></a>
<span class="sourceLineNo">094</span><a name="line.94"></a>
<span class="sourceLineNo">095</span> /**<a name="line.95"></a>
<span class="sourceLineNo">096</span> * Test for the existence of columns in the table, as specified by the Gets.<a name="line.96"></a>
<span class="sourceLineNo">097</span> * &lt;p&gt;<a name="line.97"></a>
<span class="sourceLineNo">098</span> * This will return an array of booleans. Each value will be true if the related Get matches one<a name="line.98"></a>
<span class="sourceLineNo">099</span> * or more keys, false if not.<a name="line.99"></a>
<span class="sourceLineNo">100</span> * &lt;p&gt;<a name="line.100"></a>
<span class="sourceLineNo">101</span> * This is a server-side call so it prevents any data from being transferred to the client.<a name="line.101"></a>
<span class="sourceLineNo">102</span> * @param gets the Gets<a name="line.102"></a>
<span class="sourceLineNo">103</span> * @return Array of boolean. True if the specified Get matches one or more keys, false if not.<a name="line.103"></a>
<span class="sourceLineNo">104</span> * @throws IOException e<a name="line.104"></a>
<span class="sourceLineNo">105</span> */<a name="line.105"></a>
<span class="sourceLineNo">106</span> default boolean[] exists(List&lt;Get&gt; gets) throws IOException {<a name="line.106"></a>
<span class="sourceLineNo">107</span> throw new NotImplementedException("Add an implementation!");<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> /**<a name="line.110"></a>
<span class="sourceLineNo">111</span> * Method that does a batch call on Deletes, Gets, Puts, Increments, Appends, RowMutations. The<a name="line.111"></a>
<span class="sourceLineNo">112</span> * ordering of execution of the actions is not defined. Meaning if you do a Put and a Get in the<a name="line.112"></a>
<span class="sourceLineNo">113</span> * same {@link #batch} call, you will not necessarily be guaranteed that the Get returns what the<a name="line.113"></a>
<span class="sourceLineNo">114</span> * Put had put.<a name="line.114"></a>
<span class="sourceLineNo">115</span> * @param actions list of Get, Put, Delete, Increment, Append, RowMutations.<a name="line.115"></a>
<span class="sourceLineNo">116</span> * @param results Empty Object[], same size as actions. Provides access to partial results, in<a name="line.116"></a>
<span class="sourceLineNo">117</span> * case an exception is thrown. A null in the result array means that the call for<a name="line.117"></a>
<span class="sourceLineNo">118</span> * that action failed, even after retries. The order of the objects in the results<a name="line.118"></a>
<span class="sourceLineNo">119</span> * array corresponds to the order of actions in the request list.<a name="line.119"></a>
<span class="sourceLineNo">120</span> * @since 0.90.0<a name="line.120"></a>
<span class="sourceLineNo">121</span> */<a name="line.121"></a>
<span class="sourceLineNo">122</span> default void batch(final List&lt;? extends Row&gt; actions, final Object[] results)<a name="line.122"></a>
<span class="sourceLineNo">123</span> throws IOException, InterruptedException {<a name="line.123"></a>
<span class="sourceLineNo">124</span> throw new NotImplementedException("Add an implementation!");<a name="line.124"></a>
<span class="sourceLineNo">125</span> }<a name="line.125"></a>
<span class="sourceLineNo">126</span><a name="line.126"></a>
<span class="sourceLineNo">127</span> /**<a name="line.127"></a>
<span class="sourceLineNo">128</span> * Same as {@link #batch(List, Object[])}, but with a callback.<a name="line.128"></a>
<span class="sourceLineNo">129</span> * @since 0.96.0<a name="line.129"></a>
<span class="sourceLineNo">130</span> * @deprecated since 3.0.0, will removed in 4.0.0. Please use the batch related methods in<a name="line.130"></a>
<span class="sourceLineNo">131</span> * {@link AsyncTable} directly if you want to use callback. We reuse the callback for<a name="line.131"></a>
<span class="sourceLineNo">132</span> * coprocessor here, and the problem is that for batch operation, the<a name="line.132"></a>
<span class="sourceLineNo">133</span> * {@link AsyncTable} does not tell us the region, so in this method we need an extra<a name="line.133"></a>
<span class="sourceLineNo">134</span> * locating after we get the result, which is not good.<a name="line.134"></a>
<span class="sourceLineNo">135</span> */<a name="line.135"></a>
<span class="sourceLineNo">136</span> @Deprecated<a name="line.136"></a>
<span class="sourceLineNo">137</span> default &lt;R&gt; void batchCallback(final List&lt;? extends Row&gt; actions, final Object[] results,<a name="line.137"></a>
<span class="sourceLineNo">138</span> final Batch.Callback&lt;R&gt; callback) throws IOException, InterruptedException {<a name="line.138"></a>
<span class="sourceLineNo">139</span> throw new NotImplementedException("Add an implementation!");<a name="line.139"></a>
<span class="sourceLineNo">140</span> }<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> * Extracts certain cells from a given row.<a name="line.143"></a>
<span class="sourceLineNo">144</span> * @param get The object that specifies what data to fetch and from which row.<a name="line.144"></a>
<span class="sourceLineNo">145</span> * @return The data coming from the specified row, if it exists. If the row specified doesn't<a name="line.145"></a>
<span class="sourceLineNo">146</span> * exist, the {@link Result} instance returned won't contain any<a name="line.146"></a>
<span class="sourceLineNo">147</span> * {@link org.apache.hadoop.hbase.KeyValue}, as indicated by {@link Result#isEmpty()}.<a name="line.147"></a>
<span class="sourceLineNo">148</span> * @throws IOException if a remote or network exception occurs.<a name="line.148"></a>
<span class="sourceLineNo">149</span> * @since 0.20.0<a name="line.149"></a>
<span class="sourceLineNo">150</span> */<a name="line.150"></a>
<span class="sourceLineNo">151</span> default Result get(Get get) throws IOException {<a name="line.151"></a>
<span class="sourceLineNo">152</span> return get(Collections.singletonList(get))[0];<a name="line.152"></a>
<span class="sourceLineNo">153</span> }<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> * Extracts specified cells from the given rows, as a batch.<a name="line.156"></a>
<span class="sourceLineNo">157</span> * @param gets The objects that specify what data to fetch and from which rows.<a name="line.157"></a>
<span class="sourceLineNo">158</span> * @return The data coming from the specified rows, if it exists. If the row specified doesn't<a name="line.158"></a>
<span class="sourceLineNo">159</span> * exist, the {@link Result} instance returned won't contain any<a name="line.159"></a>
<span class="sourceLineNo">160</span> * {@link org.apache.hadoop.hbase.Cell}s, as indicated by {@link Result#isEmpty()}. If<a name="line.160"></a>
<span class="sourceLineNo">161</span> * there are any failures even after retries, there will be a &lt;code&gt;null&lt;/code&gt; in the<a name="line.161"></a>
<span class="sourceLineNo">162</span> * results' array for those Gets, AND an exception will be thrown. The ordering of the<a name="line.162"></a>
<span class="sourceLineNo">163</span> * Result array corresponds to the order of the list of passed in Gets.<a name="line.163"></a>
<span class="sourceLineNo">164</span> * @throws IOException if a remote or network exception occurs.<a name="line.164"></a>
<span class="sourceLineNo">165</span> * @since 0.90.0<a name="line.165"></a>
<span class="sourceLineNo">166</span> * @apiNote {@link #put(List)} runs pre-flight validations on the input list on client. Currently<a name="line.166"></a>
<span class="sourceLineNo">167</span> * {@link #get(List)} doesn't run any validations on the client-side, currently there is<a name="line.167"></a>
<span class="sourceLineNo">168</span> * no need, but this may change in the future. An {@link IllegalArgumentException} will<a name="line.168"></a>
<span class="sourceLineNo">169</span> * be thrown in this case.<a name="line.169"></a>
<span class="sourceLineNo">170</span> */<a name="line.170"></a>
<span class="sourceLineNo">171</span> default Result[] get(List&lt;Get&gt; gets) throws IOException {<a name="line.171"></a>
<span class="sourceLineNo">172</span> throw new NotImplementedException("Add an implementation!");<a name="line.172"></a>
<span class="sourceLineNo">173</span> }<a name="line.173"></a>
<span class="sourceLineNo">174</span><a name="line.174"></a>
<span class="sourceLineNo">175</span> /**<a name="line.175"></a>
<span class="sourceLineNo">176</span> * Returns a scanner on the current table as specified by the {@link Scan} object. Note that the<a name="line.176"></a>
<span class="sourceLineNo">177</span> * passed {@link Scan}'s start row and caching properties maybe changed.<a name="line.177"></a>
<span class="sourceLineNo">178</span> * @param scan A configured {@link Scan} object.<a name="line.178"></a>
<span class="sourceLineNo">179</span> * @return A scanner.<a name="line.179"></a>
<span class="sourceLineNo">180</span> * @throws IOException if a remote or network exception occurs.<a name="line.180"></a>
<span class="sourceLineNo">181</span> * @since 0.20.0<a name="line.181"></a>
<span class="sourceLineNo">182</span> */<a name="line.182"></a>
<span class="sourceLineNo">183</span> default ResultScanner getScanner(Scan scan) throws IOException {<a name="line.183"></a>
<span class="sourceLineNo">184</span> throw new NotImplementedException("Add an implementation!");<a name="line.184"></a>
<span class="sourceLineNo">185</span> }<a name="line.185"></a>
<span class="sourceLineNo">186</span><a name="line.186"></a>
<span class="sourceLineNo">187</span> /**<a name="line.187"></a>
<span class="sourceLineNo">188</span> * Gets a scanner on the current table for the given family.<a name="line.188"></a>
<span class="sourceLineNo">189</span> * @param family The column family to scan.<a name="line.189"></a>
<span class="sourceLineNo">190</span> * @return A scanner.<a name="line.190"></a>
<span class="sourceLineNo">191</span> * @throws IOException if a remote or network exception occurs.<a name="line.191"></a>
<span class="sourceLineNo">192</span> * @since 0.20.0<a name="line.192"></a>
<span class="sourceLineNo">193</span> */<a name="line.193"></a>
<span class="sourceLineNo">194</span> default ResultScanner getScanner(byte[] family) throws IOException {<a name="line.194"></a>
<span class="sourceLineNo">195</span> throw new NotImplementedException("Add an implementation!");<a name="line.195"></a>
<span class="sourceLineNo">196</span> }<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> * Gets a scanner on the current table for the given family and qualifier.<a name="line.199"></a>
<span class="sourceLineNo">200</span> * @param family The column family to scan.<a name="line.200"></a>
<span class="sourceLineNo">201</span> * @param qualifier The column qualifier to scan.<a name="line.201"></a>
<span class="sourceLineNo">202</span> * @return A scanner.<a name="line.202"></a>
<span class="sourceLineNo">203</span> * @throws IOException if a remote or network exception occurs.<a name="line.203"></a>
<span class="sourceLineNo">204</span> * @since 0.20.0<a name="line.204"></a>
<span class="sourceLineNo">205</span> */<a name="line.205"></a>
<span class="sourceLineNo">206</span> default ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException {<a name="line.206"></a>
<span class="sourceLineNo">207</span> throw new NotImplementedException("Add an implementation!");<a name="line.207"></a>
<span class="sourceLineNo">208</span> }<a name="line.208"></a>
<span class="sourceLineNo">209</span><a name="line.209"></a>
<span class="sourceLineNo">210</span> /**<a name="line.210"></a>
<span class="sourceLineNo">211</span> * Puts some data in the table.<a name="line.211"></a>
<span class="sourceLineNo">212</span> * @param put The data to put.<a name="line.212"></a>
<span class="sourceLineNo">213</span> * @throws IOException if a remote or network exception occurs.<a name="line.213"></a>
<span class="sourceLineNo">214</span> * @since 0.20.0<a name="line.214"></a>
<span class="sourceLineNo">215</span> */<a name="line.215"></a>
<span class="sourceLineNo">216</span> default void put(Put put) throws IOException {<a name="line.216"></a>
<span class="sourceLineNo">217</span> put(Collections.singletonList(put));<a name="line.217"></a>
<span class="sourceLineNo">218</span> }<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> * Batch puts the specified data into the table.<a name="line.221"></a>
<span class="sourceLineNo">222</span> * &lt;p&gt;<a name="line.222"></a>
<span class="sourceLineNo">223</span> * This can be used for group commit, or for submitting user defined batches. Before sending a<a name="line.223"></a>
<span class="sourceLineNo">224</span> * batch of mutations to the server, the client runs a few validations on the input list. If an<a name="line.224"></a>
<span class="sourceLineNo">225</span> * error is found, for example, a mutation was supplied but was missing it's column an<a name="line.225"></a>
<span class="sourceLineNo">226</span> * {@link IllegalArgumentException} will be thrown and no mutations will be applied. If there are<a name="line.226"></a>
<span class="sourceLineNo">227</span> * any failures even after retries, a {@link RetriesExhaustedWithDetailsException} will be thrown.<a name="line.227"></a>
<span class="sourceLineNo">228</span> * RetriesExhaustedWithDetailsException contains lists of failed mutations and corresponding<a name="line.228"></a>
<span class="sourceLineNo">229</span> * remote exceptions. The ordering of mutations and exceptions in the encapsulating exception<a name="line.229"></a>
<span class="sourceLineNo">230</span> * corresponds to the order of the input list of Put requests.<a name="line.230"></a>
<span class="sourceLineNo">231</span> * @param puts The list of mutations to apply.<a name="line.231"></a>
<span class="sourceLineNo">232</span> * @throws IOException if a remote or network exception occurs.<a name="line.232"></a>
<span class="sourceLineNo">233</span> * @since 0.20.0<a name="line.233"></a>
<span class="sourceLineNo">234</span> */<a name="line.234"></a>
<span class="sourceLineNo">235</span> default void put(List&lt;Put&gt; puts) throws IOException {<a name="line.235"></a>
<span class="sourceLineNo">236</span> throw new NotImplementedException("Add an implementation!");<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>
<span class="sourceLineNo">240</span> * Deletes the specified cells/row.<a name="line.240"></a>
<span class="sourceLineNo">241</span> * @param delete The object that specifies what to delete.<a name="line.241"></a>
<span class="sourceLineNo">242</span> * @throws IOException if a remote or network exception occurs.<a name="line.242"></a>
<span class="sourceLineNo">243</span> * @since 0.20.0<a name="line.243"></a>
<span class="sourceLineNo">244</span> */<a name="line.244"></a>
<span class="sourceLineNo">245</span> default void delete(Delete delete) throws IOException {<a name="line.245"></a>
<span class="sourceLineNo">246</span> throw new NotImplementedException("Add an implementation!");<a name="line.246"></a>
<span class="sourceLineNo">247</span> }<a name="line.247"></a>
<span class="sourceLineNo">248</span><a name="line.248"></a>
<span class="sourceLineNo">249</span> /**<a name="line.249"></a>
<span class="sourceLineNo">250</span> * Batch Deletes the specified cells/rows from the table.<a name="line.250"></a>
<span class="sourceLineNo">251</span> * &lt;p&gt;<a name="line.251"></a>
<span class="sourceLineNo">252</span> * If a specified row does not exist, {@link Delete} will report as though sucessful delete; no<a name="line.252"></a>
<span class="sourceLineNo">253</span> * exception will be thrown. If there are any failures even after retries, a<a name="line.253"></a>
<span class="sourceLineNo">254</span> * {@link RetriesExhaustedWithDetailsException} will be thrown.<a name="line.254"></a>
<span class="sourceLineNo">255</span> * RetriesExhaustedWithDetailsException contains lists of failed {@link Delete}s and corresponding<a name="line.255"></a>
<span class="sourceLineNo">256</span> * remote exceptions.<a name="line.256"></a>
<span class="sourceLineNo">257</span> * @param deletes List of things to delete. The input list gets modified by this method. All<a name="line.257"></a>
<span class="sourceLineNo">258</span> * successfully applied {@link Delete}s in the list are removed (in particular it<a name="line.258"></a>
<span class="sourceLineNo">259</span> * gets re-ordered, so the order in which the elements are inserted in the list<a name="line.259"></a>
<span class="sourceLineNo">260</span> * gives no guarantee as to the order in which the {@link Delete}s are executed).<a name="line.260"></a>
<span class="sourceLineNo">261</span> * @throws IOException if a remote or network exception occurs. In that case the {@code deletes}<a name="line.261"></a>
<span class="sourceLineNo">262</span> * argument will contain the {@link Delete} instances that have not be<a name="line.262"></a>
<span class="sourceLineNo">263</span> * successfully applied.<a name="line.263"></a>
<span class="sourceLineNo">264</span> * @since 0.20.1<a name="line.264"></a>
<span class="sourceLineNo">265</span> * @apiNote In 3.0.0 version, the input list {@code deletes} will no longer be modified. Also,<a name="line.265"></a>
<span class="sourceLineNo">266</span> * {@link #put(List)} runs pre-flight validations on the input list on client. Currently<a name="line.266"></a>
<span class="sourceLineNo">267</span> * {@link #delete(List)} doesn't run validations on the client, there is no need<a name="line.267"></a>
<span class="sourceLineNo">268</span> * currently, but this may change in the future. An {@link IllegalArgumentException} will<a name="line.268"></a>
<span class="sourceLineNo">269</span> * be thrown in this case.<a name="line.269"></a>
<span class="sourceLineNo">270</span> */<a name="line.270"></a>
<span class="sourceLineNo">271</span> default void delete(List&lt;Delete&gt; deletes) throws IOException {<a name="line.271"></a>
<span class="sourceLineNo">272</span> throw new NotImplementedException("Add an implementation!");<a name="line.272"></a>
<span class="sourceLineNo">273</span> }<a name="line.273"></a>
<span class="sourceLineNo">274</span><a name="line.274"></a>
<span class="sourceLineNo">275</span> /**<a name="line.275"></a>
<span class="sourceLineNo">276</span> * Atomically checks if a row/family/qualifier value matches the expected value. If it does, it<a name="line.276"></a>
<span class="sourceLineNo">277</span> * adds the Put/Delete/RowMutations.<a name="line.277"></a>
<span class="sourceLineNo">278</span> * &lt;p&gt;<a name="line.278"></a>
<span class="sourceLineNo">279</span> * Use the returned {@link CheckAndMutateBuilder} to construct your request and then execute it.<a name="line.279"></a>
<span class="sourceLineNo">280</span> * This is a fluent style API, the code is like:<a name="line.280"></a>
<span class="sourceLineNo">281</span> *<a name="line.281"></a>
<span class="sourceLineNo">282</span> * &lt;pre&gt;<a name="line.282"></a>
<span class="sourceLineNo">283</span> * &lt;code&gt;<a name="line.283"></a>
<span class="sourceLineNo">284</span> * table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put);<a name="line.284"></a>
<span class="sourceLineNo">285</span> * &lt;/code&gt;<a name="line.285"></a>
<span class="sourceLineNo">286</span> * &lt;/pre&gt;<a name="line.286"></a>
<span class="sourceLineNo">287</span> *<a name="line.287"></a>
<span class="sourceLineNo">288</span> * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it<a name="line.288"></a>
<span class="sourceLineNo">289</span> * any more.<a name="line.289"></a>
<span class="sourceLineNo">290</span> */<a name="line.290"></a>
<span class="sourceLineNo">291</span> @Deprecated<a name="line.291"></a>
<span class="sourceLineNo">292</span> default CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {<a name="line.292"></a>
<span class="sourceLineNo">293</span> throw new NotImplementedException("Add an implementation!");<a name="line.293"></a>
<span class="sourceLineNo">294</span> }<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> * A helper class for sending checkAndMutate request.<a name="line.297"></a>
<span class="sourceLineNo">298</span> * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it<a name="line.298"></a>
<span class="sourceLineNo">299</span> * any more.<a name="line.299"></a>
<span class="sourceLineNo">300</span> */<a name="line.300"></a>
<span class="sourceLineNo">301</span> @Deprecated<a name="line.301"></a>
<span class="sourceLineNo">302</span> interface CheckAndMutateBuilder {<a name="line.302"></a>
<span class="sourceLineNo">303</span><a name="line.303"></a>
<span class="sourceLineNo">304</span> /**<a name="line.304"></a>
<span class="sourceLineNo">305</span> * Specify a column qualifer<a name="line.305"></a>
<span class="sourceLineNo">306</span> * @param qualifier column qualifier to check.<a name="line.306"></a>
<span class="sourceLineNo">307</span> */<a name="line.307"></a>
<span class="sourceLineNo">308</span> CheckAndMutateBuilder qualifier(byte[] qualifier);<a name="line.308"></a>
<span class="sourceLineNo">309</span><a name="line.309"></a>
<span class="sourceLineNo">310</span> /**<a name="line.310"></a>
<span class="sourceLineNo">311</span> * Specify a timerange<a name="line.311"></a>
<span class="sourceLineNo">312</span> * @param timeRange timeRange to check<a name="line.312"></a>
<span class="sourceLineNo">313</span> */<a name="line.313"></a>
<span class="sourceLineNo">314</span> CheckAndMutateBuilder timeRange(TimeRange timeRange);<a name="line.314"></a>
<span class="sourceLineNo">315</span><a name="line.315"></a>
<span class="sourceLineNo">316</span> /**<a name="line.316"></a>
<span class="sourceLineNo">317</span> * Check for lack of column.<a name="line.317"></a>
<span class="sourceLineNo">318</span> */<a name="line.318"></a>
<span class="sourceLineNo">319</span> CheckAndMutateBuilder ifNotExists();<a name="line.319"></a>
<span class="sourceLineNo">320</span><a name="line.320"></a>
<span class="sourceLineNo">321</span> /**<a name="line.321"></a>
<span class="sourceLineNo">322</span> * Check for equality.<a name="line.322"></a>
<span class="sourceLineNo">323</span> * @param value the expected value<a name="line.323"></a>
<span class="sourceLineNo">324</span> */<a name="line.324"></a>
<span class="sourceLineNo">325</span> default CheckAndMutateBuilder ifEquals(byte[] value) {<a name="line.325"></a>
<span class="sourceLineNo">326</span> return ifMatches(CompareOperator.EQUAL, value);<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> * Check for match.<a name="line.330"></a>
<span class="sourceLineNo">331</span> * @param compareOp comparison operator to use<a name="line.331"></a>
<span class="sourceLineNo">332</span> * @param value the expected value<a name="line.332"></a>
<span class="sourceLineNo">333</span> */<a name="line.333"></a>
<span class="sourceLineNo">334</span> CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value);<a name="line.334"></a>
<span class="sourceLineNo">335</span><a name="line.335"></a>
<span class="sourceLineNo">336</span> /**<a name="line.336"></a>
<span class="sourceLineNo">337</span> * Specify a Put to commit if the check succeeds.<a name="line.337"></a>
<span class="sourceLineNo">338</span> * @param put data to put if check succeeds<a name="line.338"></a>
<span class="sourceLineNo">339</span> * @return {@code true} if the new put was executed, {@code false} otherwise.<a name="line.339"></a>
<span class="sourceLineNo">340</span> */<a name="line.340"></a>
<span class="sourceLineNo">341</span> boolean thenPut(Put put) throws IOException;<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> * Specify a Delete to commit if the check succeeds.<a name="line.344"></a>
<span class="sourceLineNo">345</span> * @param delete data to delete if check succeeds<a name="line.345"></a>
<span class="sourceLineNo">346</span> * @return {@code true} if the new delete was executed, {@code false} otherwise.<a name="line.346"></a>
<span class="sourceLineNo">347</span> */<a name="line.347"></a>
<span class="sourceLineNo">348</span> boolean thenDelete(Delete delete) throws IOException;<a name="line.348"></a>
<span class="sourceLineNo">349</span><a name="line.349"></a>
<span class="sourceLineNo">350</span> /**<a name="line.350"></a>
<span class="sourceLineNo">351</span> * Specify a RowMutations to commit if the check succeeds.<a name="line.351"></a>
<span class="sourceLineNo">352</span> * @param mutation mutations to perform if check succeeds<a name="line.352"></a>
<span class="sourceLineNo">353</span> * @return true if the new mutation was executed, false otherwise.<a name="line.353"></a>
<span class="sourceLineNo">354</span> */<a name="line.354"></a>
<span class="sourceLineNo">355</span> boolean thenMutate(RowMutations mutation) throws IOException;<a name="line.355"></a>
<span class="sourceLineNo">356</span> }<a name="line.356"></a>
<span class="sourceLineNo">357</span><a name="line.357"></a>
<span class="sourceLineNo">358</span> /**<a name="line.358"></a>
<span class="sourceLineNo">359</span> * Atomically checks if a row matches the specified filter. If it does, it adds the<a name="line.359"></a>
<span class="sourceLineNo">360</span> * Put/Delete/RowMutations.<a name="line.360"></a>
<span class="sourceLineNo">361</span> * &lt;p&gt;<a name="line.361"></a>
<span class="sourceLineNo">362</span> * Use the returned {@link CheckAndMutateWithFilterBuilder} to construct your request and then<a name="line.362"></a>
<span class="sourceLineNo">363</span> * execute it. This is a fluent style API, the code is like:<a name="line.363"></a>
<span class="sourceLineNo">364</span> *<a name="line.364"></a>
<span class="sourceLineNo">365</span> * &lt;pre&gt;<a name="line.365"></a>
<span class="sourceLineNo">366</span> * &lt;code&gt;<a name="line.366"></a>
<span class="sourceLineNo">367</span> * table.checkAndMutate(row, filter).thenPut(put);<a name="line.367"></a>
<span class="sourceLineNo">368</span> * &lt;/code&gt;<a name="line.368"></a>
<span class="sourceLineNo">369</span> * &lt;/pre&gt;<a name="line.369"></a>
<span class="sourceLineNo">370</span> *<a name="line.370"></a>
<span class="sourceLineNo">371</span> * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it<a name="line.371"></a>
<span class="sourceLineNo">372</span> * any more.<a name="line.372"></a>
<span class="sourceLineNo">373</span> */<a name="line.373"></a>
<span class="sourceLineNo">374</span> @Deprecated<a name="line.374"></a>
<span class="sourceLineNo">375</span> default CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter) {<a name="line.375"></a>
<span class="sourceLineNo">376</span> throw new NotImplementedException("Add an implementation!");<a name="line.376"></a>
<span class="sourceLineNo">377</span> }<a name="line.377"></a>
<span class="sourceLineNo">378</span><a name="line.378"></a>
<span class="sourceLineNo">379</span> /**<a name="line.379"></a>
<span class="sourceLineNo">380</span> * A helper class for sending checkAndMutate request with a filter.<a name="line.380"></a>
<span class="sourceLineNo">381</span> * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it<a name="line.381"></a>
<span class="sourceLineNo">382</span> * any more.<a name="line.382"></a>
<span class="sourceLineNo">383</span> */<a name="line.383"></a>
<span class="sourceLineNo">384</span> @Deprecated<a name="line.384"></a>
<span class="sourceLineNo">385</span> interface CheckAndMutateWithFilterBuilder {<a name="line.385"></a>
<span class="sourceLineNo">386</span><a name="line.386"></a>
<span class="sourceLineNo">387</span> /**<a name="line.387"></a>
<span class="sourceLineNo">388</span> * Specify a timerange.<a name="line.388"></a>
<span class="sourceLineNo">389</span> * @param timeRange timeRange to check<a name="line.389"></a>
<span class="sourceLineNo">390</span> */<a name="line.390"></a>
<span class="sourceLineNo">391</span> CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange);<a name="line.391"></a>
<span class="sourceLineNo">392</span><a name="line.392"></a>
<span class="sourceLineNo">393</span> /**<a name="line.393"></a>
<span class="sourceLineNo">394</span> * Specify a Put to commit if the check succeeds.<a name="line.394"></a>
<span class="sourceLineNo">395</span> * @param put data to put if check succeeds<a name="line.395"></a>
<span class="sourceLineNo">396</span> * @return {@code true} if the new put was executed, {@code false} otherwise.<a name="line.396"></a>
<span class="sourceLineNo">397</span> */<a name="line.397"></a>
<span class="sourceLineNo">398</span> boolean thenPut(Put put) throws IOException;<a name="line.398"></a>
<span class="sourceLineNo">399</span><a name="line.399"></a>
<span class="sourceLineNo">400</span> /**<a name="line.400"></a>
<span class="sourceLineNo">401</span> * Specify a Delete to commit if the check succeeds.<a name="line.401"></a>
<span class="sourceLineNo">402</span> * @param delete data to delete if check succeeds<a name="line.402"></a>
<span class="sourceLineNo">403</span> * @return {@code true} if the new delete was executed, {@code false} otherwise.<a name="line.403"></a>
<span class="sourceLineNo">404</span> */<a name="line.404"></a>
<span class="sourceLineNo">405</span> boolean thenDelete(Delete delete) throws IOException;<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> * Specify a RowMutations to commit if the check succeeds.<a name="line.408"></a>
<span class="sourceLineNo">409</span> * @param mutation mutations to perform if check succeeds<a name="line.409"></a>
<span class="sourceLineNo">410</span> * @return true if the new mutation was executed, false otherwise.<a name="line.410"></a>
<span class="sourceLineNo">411</span> */<a name="line.411"></a>
<span class="sourceLineNo">412</span> boolean thenMutate(RowMutations mutation) throws IOException;<a name="line.412"></a>
<span class="sourceLineNo">413</span> }<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> * checkAndMutate that atomically checks if a row matches the specified condition. If it does, it<a name="line.416"></a>
<span class="sourceLineNo">417</span> * performs the specified action.<a name="line.417"></a>
<span class="sourceLineNo">418</span> * @param checkAndMutate The CheckAndMutate object.<a name="line.418"></a>
<span class="sourceLineNo">419</span> * @return A CheckAndMutateResult object that represents the result for the CheckAndMutate.<a name="line.419"></a>
<span class="sourceLineNo">420</span> * @throws IOException if a remote or network exception occurs.<a name="line.420"></a>
<span class="sourceLineNo">421</span> */<a name="line.421"></a>
<span class="sourceLineNo">422</span> default CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException {<a name="line.422"></a>
<span class="sourceLineNo">423</span> return checkAndMutate(Collections.singletonList(checkAndMutate)).get(0);<a name="line.423"></a>
<span class="sourceLineNo">424</span> }<a name="line.424"></a>
<span class="sourceLineNo">425</span><a name="line.425"></a>
<span class="sourceLineNo">426</span> /**<a name="line.426"></a>
<span class="sourceLineNo">427</span> * Batch version of checkAndMutate. The specified CheckAndMutates are batched only in the sense<a name="line.427"></a>
<span class="sourceLineNo">428</span> * that they are sent to a RS in one RPC, but each CheckAndMutate operation is still executed<a name="line.428"></a>
<span class="sourceLineNo">429</span> * atomically (and thus, each may fail independently of others).<a name="line.429"></a>
<span class="sourceLineNo">430</span> * @param checkAndMutates The list of CheckAndMutate.<a name="line.430"></a>
<span class="sourceLineNo">431</span> * @return A list of CheckAndMutateResult objects that represents the result for each<a name="line.431"></a>
<span class="sourceLineNo">432</span> * CheckAndMutate.<a name="line.432"></a>
<span class="sourceLineNo">433</span> * @throws IOException if a remote or network exception occurs.<a name="line.433"></a>
<span class="sourceLineNo">434</span> */<a name="line.434"></a>
<span class="sourceLineNo">435</span> default List&lt;CheckAndMutateResult&gt; checkAndMutate(List&lt;CheckAndMutate&gt; checkAndMutates)<a name="line.435"></a>
<span class="sourceLineNo">436</span> throws IOException {<a name="line.436"></a>
<span class="sourceLineNo">437</span> throw new NotImplementedException("Add an implementation!");<a name="line.437"></a>
<span class="sourceLineNo">438</span> }<a name="line.438"></a>
<span class="sourceLineNo">439</span><a name="line.439"></a>
<span class="sourceLineNo">440</span> /**<a name="line.440"></a>
<span class="sourceLineNo">441</span> * Performs multiple mutations atomically on a single row. Currently {@link Put} and<a name="line.441"></a>
<span class="sourceLineNo">442</span> * {@link Delete} are supported.<a name="line.442"></a>
<span class="sourceLineNo">443</span> * @param rm object that specifies the set of mutations to perform atomically<a name="line.443"></a>
<span class="sourceLineNo">444</span> * @return results of Increment/Append operations<a name="line.444"></a>
<span class="sourceLineNo">445</span> * @throws IOException if a remote or network exception occurs.<a name="line.445"></a>
<span class="sourceLineNo">446</span> */<a name="line.446"></a>
<span class="sourceLineNo">447</span> default Result mutateRow(final RowMutations rm) throws IOException {<a name="line.447"></a>
<span class="sourceLineNo">448</span> throw new NotImplementedException("Add an implementation!");<a name="line.448"></a>
<span class="sourceLineNo">449</span> }<a name="line.449"></a>
<span class="sourceLineNo">450</span><a name="line.450"></a>
<span class="sourceLineNo">451</span> /**<a name="line.451"></a>
<span class="sourceLineNo">452</span> * Appends values to one or more columns within a single row.<a name="line.452"></a>
<span class="sourceLineNo">453</span> * &lt;p&gt;<a name="line.453"></a>
<span class="sourceLineNo">454</span> * This operation guaranteed atomicity to readers. Appends are done under a single row lock, so<a name="line.454"></a>
<span class="sourceLineNo">455</span> * write operations to a row are synchronized, and readers are guaranteed to see this operation<a name="line.455"></a>
<span class="sourceLineNo">456</span> * fully completed.<a name="line.456"></a>
<span class="sourceLineNo">457</span> * @param append object that specifies the columns and values to be appended<a name="line.457"></a>
<span class="sourceLineNo">458</span> * @throws IOException e<a name="line.458"></a>
<span class="sourceLineNo">459</span> * @return values of columns after the append operation (maybe null)<a name="line.459"></a>
<span class="sourceLineNo">460</span> */<a name="line.460"></a>
<span class="sourceLineNo">461</span> default Result append(final Append append) throws IOException {<a name="line.461"></a>
<span class="sourceLineNo">462</span> throw new NotImplementedException("Add an implementation!");<a name="line.462"></a>
<span class="sourceLineNo">463</span> }<a name="line.463"></a>
<span class="sourceLineNo">464</span><a name="line.464"></a>
<span class="sourceLineNo">465</span> /**<a name="line.465"></a>
<span class="sourceLineNo">466</span> * Increments one or more columns within a single row.<a name="line.466"></a>
<span class="sourceLineNo">467</span> * &lt;p&gt;<a name="line.467"></a>
<span class="sourceLineNo">468</span> * This operation ensures atomicity to readers. Increments are done under a single row lock, so<a name="line.468"></a>
<span class="sourceLineNo">469</span> * write operations to a row are synchronized, and readers are guaranteed to see this operation<a name="line.469"></a>
<span class="sourceLineNo">470</span> * fully completed.<a name="line.470"></a>
<span class="sourceLineNo">471</span> * @param increment object that specifies the columns and amounts to be used for the increment<a name="line.471"></a>
<span class="sourceLineNo">472</span> * operations<a name="line.472"></a>
<span class="sourceLineNo">473</span> * @throws IOException e<a name="line.473"></a>
<span class="sourceLineNo">474</span> * @return values of columns after the increment<a name="line.474"></a>
<span class="sourceLineNo">475</span> */<a name="line.475"></a>
<span class="sourceLineNo">476</span> default Result increment(final Increment increment) throws IOException {<a name="line.476"></a>
<span class="sourceLineNo">477</span> throw new NotImplementedException("Add an implementation!");<a name="line.477"></a>
<span class="sourceLineNo">478</span> }<a name="line.478"></a>
<span class="sourceLineNo">479</span><a name="line.479"></a>
<span class="sourceLineNo">480</span> /**<a name="line.480"></a>
<span class="sourceLineNo">481</span> * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}<a name="line.481"></a>
<span class="sourceLineNo">482</span> * &lt;p&gt;<a name="line.482"></a>
<span class="sourceLineNo">483</span> * The {@link Durability} is defaulted to {@link Durability#SYNC_WAL}.<a name="line.483"></a>
<span class="sourceLineNo">484</span> * @param row The row that contains the cell to increment.<a name="line.484"></a>
<span class="sourceLineNo">485</span> * @param family The column family of the cell to increment.<a name="line.485"></a>
<span class="sourceLineNo">486</span> * @param qualifier The column qualifier of the cell to increment.<a name="line.486"></a>
<span class="sourceLineNo">487</span> * @param amount The amount to increment the cell with (or decrement, if the amount is<a name="line.487"></a>
<span class="sourceLineNo">488</span> * negative).<a name="line.488"></a>
<span class="sourceLineNo">489</span> * @return The new value, post increment.<a name="line.489"></a>
<span class="sourceLineNo">490</span> * @throws IOException if a remote or network exception occurs.<a name="line.490"></a>
<span class="sourceLineNo">491</span> */<a name="line.491"></a>
<span class="sourceLineNo">492</span> default long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount)<a name="line.492"></a>
<span class="sourceLineNo">493</span> throws IOException {<a name="line.493"></a>
<span class="sourceLineNo">494</span> Increment increment = new Increment(row).addColumn(family, qualifier, amount);<a name="line.494"></a>
<span class="sourceLineNo">495</span> Cell cell = increment(increment).getColumnLatestCell(family, qualifier);<a name="line.495"></a>
<span class="sourceLineNo">496</span> return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());<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> * Atomically increments a column value. If the column value already exists and is not a<a name="line.500"></a>
<span class="sourceLineNo">501</span> * big-endian long, this could throw an exception. If the column value does not yet exist it is<a name="line.501"></a>
<span class="sourceLineNo">502</span> * initialized to &lt;code&gt;amount&lt;/code&gt; and written to the specified column.<a name="line.502"></a>
<span class="sourceLineNo">503</span> * &lt;p&gt;<a name="line.503"></a>
<span class="sourceLineNo">504</span> * Setting durability to {@link Durability#SKIP_WAL} means that in a fail scenario you will lose<a name="line.504"></a>
<span class="sourceLineNo">505</span> * any increments that have not been flushed.<a name="line.505"></a>
<span class="sourceLineNo">506</span> * @param row The row that contains the cell to increment.<a name="line.506"></a>
<span class="sourceLineNo">507</span> * @param family The column family of the cell to increment.<a name="line.507"></a>
<span class="sourceLineNo">508</span> * @param qualifier The column qualifier of the cell to increment.<a name="line.508"></a>
<span class="sourceLineNo">509</span> * @param amount The amount to increment the cell with (or decrement, if the amount is<a name="line.509"></a>
<span class="sourceLineNo">510</span> * negative).<a name="line.510"></a>
<span class="sourceLineNo">511</span> * @param durability The persistence guarantee for this increment.<a name="line.511"></a>
<span class="sourceLineNo">512</span> * @return The new value, post increment.<a name="line.512"></a>
<span class="sourceLineNo">513</span> * @throws IOException if a remote or network exception occurs.<a name="line.513"></a>
<span class="sourceLineNo">514</span> */<a name="line.514"></a>
<span class="sourceLineNo">515</span> default long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount,<a name="line.515"></a>
<span class="sourceLineNo">516</span> Durability durability) throws IOException {<a name="line.516"></a>
<span class="sourceLineNo">517</span> Increment increment =<a name="line.517"></a>
<span class="sourceLineNo">518</span> new Increment(row).addColumn(family, qualifier, amount).setDurability(durability);<a name="line.518"></a>
<span class="sourceLineNo">519</span> Cell cell = increment(increment).getColumnLatestCell(family, qualifier);<a name="line.519"></a>
<span class="sourceLineNo">520</span> return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());<a name="line.520"></a>
<span class="sourceLineNo">521</span> }<a name="line.521"></a>
<span class="sourceLineNo">522</span><a name="line.522"></a>
<span class="sourceLineNo">523</span> /**<a name="line.523"></a>
<span class="sourceLineNo">524</span> * Releases any resources held or pending changes in internal buffers.<a name="line.524"></a>
<span class="sourceLineNo">525</span> * @throws IOException if a remote or network exception occurs.<a name="line.525"></a>
<span class="sourceLineNo">526</span> */<a name="line.526"></a>
<span class="sourceLineNo">527</span> @Override<a name="line.527"></a>
<span class="sourceLineNo">528</span> default void close() throws IOException {<a name="line.528"></a>
<span class="sourceLineNo">529</span> throw new NotImplementedException("Add an implementation!");<a name="line.529"></a>
<span class="sourceLineNo">530</span> }<a name="line.530"></a>
<span class="sourceLineNo">531</span><a name="line.531"></a>
<span class="sourceLineNo">532</span> /**<a name="line.532"></a>
<span class="sourceLineNo">533</span> * Creates and returns a {@link org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel}<a name="line.533"></a>
<span class="sourceLineNo">534</span> * instance connected to the table region containing the specified row. The row given does not<a name="line.534"></a>
<span class="sourceLineNo">535</span> * actually have to exist. Whichever region would contain the row based on start and end keys will<a name="line.535"></a>
<span class="sourceLineNo">536</span> * be used. Note that the {@code row} parameter is also not passed to the coprocessor handler<a name="line.536"></a>
<span class="sourceLineNo">537</span> * registered for this protocol, unless the {@code row} is separately passed as an argument in the<a name="line.537"></a>
<span class="sourceLineNo">538</span> * service request. The parameter here is only used to locate the region used to handle the call.<a name="line.538"></a>
<span class="sourceLineNo">539</span> * &lt;p/&gt;<a name="line.539"></a>
<span class="sourceLineNo">540</span> * The obtained {@link org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel} instance can be<a name="line.540"></a>
<span class="sourceLineNo">541</span> * used to access a published coprocessor {@link Service} using standard protobuf service<a name="line.541"></a>
<span class="sourceLineNo">542</span> * invocations:<a name="line.542"></a>
<span class="sourceLineNo">543</span> * &lt;p/&gt;<a name="line.543"></a>
<span class="sourceLineNo">544</span> * &lt;div style="background-color: #cccccc; padding: 2px"&gt; &lt;blockquote&gt;<a name="line.544"></a>
<span class="sourceLineNo">545</span> *<a name="line.545"></a>
<span class="sourceLineNo">546</span> * &lt;pre&gt;<a name="line.546"></a>
<span class="sourceLineNo">547</span> * CoprocessorRpcChannel channel = myTable.coprocessorService(rowkey);<a name="line.547"></a>
<span class="sourceLineNo">548</span> * MyService.BlockingInterface service = MyService.newBlockingStub(channel);<a name="line.548"></a>
<span class="sourceLineNo">549</span> * MyCallRequest request = MyCallRequest.newBuilder()<a name="line.549"></a>
<span class="sourceLineNo">550</span> * ...<a name="line.550"></a>
<span class="sourceLineNo">551</span> * .build();<a name="line.551"></a>
<span class="sourceLineNo">552</span> * MyCallResponse response = service.myCall(null, request);<a name="line.552"></a>
<span class="sourceLineNo">553</span> * &lt;/pre&gt;<a name="line.553"></a>
<span class="sourceLineNo">554</span> *<a name="line.554"></a>
<span class="sourceLineNo">555</span> * &lt;/blockquote&gt; &lt;/div&gt;<a name="line.555"></a>
<span class="sourceLineNo">556</span> * @param row The row key used to identify the remote region location<a name="line.556"></a>
<span class="sourceLineNo">557</span> * @return A CoprocessorRpcChannel instance<a name="line.557"></a>
<span class="sourceLineNo">558</span> * @deprecated since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any<a name="line.558"></a>
<span class="sourceLineNo">559</span> * more. Use the coprocessorService methods in {@link AsyncTable} instead.<a name="line.559"></a>
<span class="sourceLineNo">560</span> * @see Connection#toAsyncConnection()<a name="line.560"></a>
<span class="sourceLineNo">561</span> */<a name="line.561"></a>
<span class="sourceLineNo">562</span> @Deprecated<a name="line.562"></a>
<span class="sourceLineNo">563</span> default CoprocessorRpcChannel coprocessorService(byte[] row) {<a name="line.563"></a>
<span class="sourceLineNo">564</span> throw new NotImplementedException("Add an implementation!");<a name="line.564"></a>
<span class="sourceLineNo">565</span> }<a name="line.565"></a>
<span class="sourceLineNo">566</span><a name="line.566"></a>
<span class="sourceLineNo">567</span> /**<a name="line.567"></a>
<span class="sourceLineNo">568</span> * Creates an instance of the given {@link Service} subclass for each table region spanning the<a name="line.568"></a>
<span class="sourceLineNo">569</span> * range from the {@code startKey} row to {@code endKey} row (inclusive), and invokes the passed<a name="line.569"></a>
<span class="sourceLineNo">570</span> * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method with each<a name="line.570"></a>
<span class="sourceLineNo">571</span> * {@link Service} instance.<a name="line.571"></a>
<span class="sourceLineNo">572</span> * @param service the protocol buffer {@code Service} implementation to call<a name="line.572"></a>
<span class="sourceLineNo">573</span> * @param startKey start region selection with region containing this row. If {@code null}, the<a name="line.573"></a>
<span class="sourceLineNo">574</span> * selection will start with the first table region.<a name="line.574"></a>
<span class="sourceLineNo">575</span> * @param endKey select regions up to and including the region containing this row. If<a name="line.575"></a>
<span class="sourceLineNo">576</span> * {@code null}, selection will continue through the last table region.<a name="line.576"></a>
<span class="sourceLineNo">577</span> * @param callable this instance's<a name="line.577"></a>
<span class="sourceLineNo">578</span> * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method will<a name="line.578"></a>
<span class="sourceLineNo">579</span> * be invoked once per table region, using the {@link Service} instance connected<a name="line.579"></a>
<span class="sourceLineNo">580</span> * to that region.<a name="line.580"></a>
<span class="sourceLineNo">581</span> * @param &lt;T&gt; the {@link Service} subclass to connect to<a name="line.581"></a>
<span class="sourceLineNo">582</span> * @param &lt;R&gt; Return type for the {@code callable} parameter's<a name="line.582"></a>
<span class="sourceLineNo">583</span> * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method<a name="line.583"></a>
<span class="sourceLineNo">584</span> * @return a map of result values keyed by region name<a name="line.584"></a>
<span class="sourceLineNo">585</span> * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking<a name="line.585"></a>
<span class="sourceLineNo">586</span> * interface for of a protobuf stub, so it is not possible to do it in an asynchronous<a name="line.586"></a>
<span class="sourceLineNo">587</span> * way, even if now we are building the {@link Table} implementation based on the<a name="line.587"></a>
<span class="sourceLineNo">588</span> * {@link AsyncTable}, which is not good. Use the coprocessorService methods in<a name="line.588"></a>
<span class="sourceLineNo">589</span> * {@link AsyncTable} directly instead.<a name="line.589"></a>
<span class="sourceLineNo">590</span> * @see Connection#toAsyncConnection()<a name="line.590"></a>
<span class="sourceLineNo">591</span> */<a name="line.591"></a>
<span class="sourceLineNo">592</span> @Deprecated<a name="line.592"></a>
<span class="sourceLineNo">593</span> default &lt;T extends Service, R&gt; Map&lt;byte[], R&gt; coprocessorService(final Class&lt;T&gt; service,<a name="line.593"></a>
<span class="sourceLineNo">594</span> byte[] startKey, byte[] endKey, final Batch.Call&lt;T, R&gt; callable)<a name="line.594"></a>
<span class="sourceLineNo">595</span> throws ServiceException, Throwable {<a name="line.595"></a>
<span class="sourceLineNo">596</span> Map&lt;byte[], R&gt; results =<a name="line.596"></a>
<span class="sourceLineNo">597</span> Collections.synchronizedMap(new TreeMap&lt;byte[], R&gt;(Bytes.BYTES_COMPARATOR));<a name="line.597"></a>
<span class="sourceLineNo">598</span> coprocessorService(service, startKey, endKey, callable, new Batch.Callback&lt;R&gt;() {<a name="line.598"></a>
<span class="sourceLineNo">599</span> @Override<a name="line.599"></a>
<span class="sourceLineNo">600</span> public void update(byte[] region, byte[] row, R value) {<a name="line.600"></a>
<span class="sourceLineNo">601</span> if (region != null) {<a name="line.601"></a>
<span class="sourceLineNo">602</span> results.put(region, value);<a name="line.602"></a>
<span class="sourceLineNo">603</span> }<a name="line.603"></a>
<span class="sourceLineNo">604</span> }<a name="line.604"></a>
<span class="sourceLineNo">605</span> });<a name="line.605"></a>
<span class="sourceLineNo">606</span> return results;<a name="line.606"></a>
<span class="sourceLineNo">607</span> }<a name="line.607"></a>
<span class="sourceLineNo">608</span><a name="line.608"></a>
<span class="sourceLineNo">609</span> /**<a name="line.609"></a>
<span class="sourceLineNo">610</span> * Creates an instance of the given {@link Service} subclass for each table region spanning the<a name="line.610"></a>
<span class="sourceLineNo">611</span> * range from the {@code startKey} row to {@code endKey} row (inclusive), and invokes the passed<a name="line.611"></a>
<span class="sourceLineNo">612</span> * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method with each<a name="line.612"></a>
<span class="sourceLineNo">613</span> * {@link Service} instance.<a name="line.613"></a>
<span class="sourceLineNo">614</span> * &lt;p/&gt;<a name="line.614"></a>
<span class="sourceLineNo">615</span> * The given<a name="line.615"></a>
<span class="sourceLineNo">616</span> * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)}<a name="line.616"></a>
<span class="sourceLineNo">617</span> * method will be called with the return value from each region's<a name="line.617"></a>
<span class="sourceLineNo">618</span> * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} invocation.<a name="line.618"></a>
<span class="sourceLineNo">619</span> * @param service the protocol buffer {@code Service} implementation to call<a name="line.619"></a>
<span class="sourceLineNo">620</span> * @param startKey start region selection with region containing this row. If {@code null}, the<a name="line.620"></a>
<span class="sourceLineNo">621</span> * selection will start with the first table region.<a name="line.621"></a>
<span class="sourceLineNo">622</span> * @param endKey select regions up to and including the region containing this row. If<a name="line.622"></a>
<span class="sourceLineNo">623</span> * {@code null}, selection will continue through the last table region.<a name="line.623"></a>
<span class="sourceLineNo">624</span> * @param callable this instance's<a name="line.624"></a>
<span class="sourceLineNo">625</span> * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method will<a name="line.625"></a>
<span class="sourceLineNo">626</span> * be invoked once per table region, using the {@link Service} instance connected<a name="line.626"></a>
<span class="sourceLineNo">627</span> * to that region.<a name="line.627"></a>
<span class="sourceLineNo">628</span> * @param &lt;T&gt; the {@link Service} subclass to connect to<a name="line.628"></a>
<span class="sourceLineNo">629</span> * @param &lt;R&gt; Return type for the {@code callable} parameter's<a name="line.629"></a>
<span class="sourceLineNo">630</span> * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method<a name="line.630"></a>
<span class="sourceLineNo">631</span> * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking<a name="line.631"></a>
<span class="sourceLineNo">632</span> * interface for of a protobuf stub, so it is not possible to do it in an asynchronous<a name="line.632"></a>
<span class="sourceLineNo">633</span> * way, even if now we are building the {@link Table} implementation based on the<a name="line.633"></a>
<span class="sourceLineNo">634</span> * {@link AsyncTable}, which is not good. Use the coprocessorService methods in<a name="line.634"></a>
<span class="sourceLineNo">635</span> * {@link AsyncTable} directly instead.<a name="line.635"></a>
<span class="sourceLineNo">636</span> * @see Connection#toAsyncConnection()<a name="line.636"></a>
<span class="sourceLineNo">637</span> */<a name="line.637"></a>
<span class="sourceLineNo">638</span> @Deprecated<a name="line.638"></a>
<span class="sourceLineNo">639</span> default &lt;T extends Service, R&gt; void coprocessorService(final Class&lt;T&gt; service, byte[] startKey,<a name="line.639"></a>
<span class="sourceLineNo">640</span> byte[] endKey, final Batch.Call&lt;T, R&gt; callable, final Batch.Callback&lt;R&gt; callback)<a name="line.640"></a>
<span class="sourceLineNo">641</span> throws ServiceException, Throwable {<a name="line.641"></a>
<span class="sourceLineNo">642</span> throw new NotImplementedException("Add an implementation!");<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> * Creates an instance of the given {@link Service} subclass for each table region spanning the<a name="line.646"></a>
<span class="sourceLineNo">647</span> * range from the {@code startKey} row to {@code endKey} row (inclusive), all the invocations to<a name="line.647"></a>
<span class="sourceLineNo">648</span> * the same region server will be batched into one call. The coprocessor service is invoked<a name="line.648"></a>
<span class="sourceLineNo">649</span> * according to the service instance, method name and parameters.<a name="line.649"></a>
<span class="sourceLineNo">650</span> * @param methodDescriptor the descriptor for the protobuf service method to call.<a name="line.650"></a>
<span class="sourceLineNo">651</span> * @param request the method call parameters<a name="line.651"></a>
<span class="sourceLineNo">652</span> * @param startKey start region selection with region containing this row. If<a name="line.652"></a>
<span class="sourceLineNo">653</span> * {@code null}, the selection will start with the first table region.<a name="line.653"></a>
<span class="sourceLineNo">654</span> * @param endKey select regions up to and including the region containing this row. If<a name="line.654"></a>
<span class="sourceLineNo">655</span> * {@code null}, selection will continue through the last table region.<a name="line.655"></a>
<span class="sourceLineNo">656</span> * @param responsePrototype the proto type of the response of the method in Service.<a name="line.656"></a>
<span class="sourceLineNo">657</span> * @param &lt;R&gt; the response type for the coprocessor Service method<a name="line.657"></a>
<span class="sourceLineNo">658</span> * @return a map of result values keyed by region name<a name="line.658"></a>
<span class="sourceLineNo">659</span> * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking<a name="line.659"></a>
<span class="sourceLineNo">660</span> * interface for of a protobuf stub, so it is not possible to do it in an asynchronous<a name="line.660"></a>
<span class="sourceLineNo">661</span> * way, even if now we are building the {@link Table} implementation based on the<a name="line.661"></a>
<span class="sourceLineNo">662</span> * {@link AsyncTable}, which is not good. Use the coprocessorService methods in<a name="line.662"></a>
<span class="sourceLineNo">663</span> * {@link AsyncTable} directly instead.<a name="line.663"></a>
<span class="sourceLineNo">664</span> * @see Connection#toAsyncConnection()<a name="line.664"></a>
<span class="sourceLineNo">665</span> */<a name="line.665"></a>
<span class="sourceLineNo">666</span> @Deprecated<a name="line.666"></a>
<span class="sourceLineNo">667</span> default &lt;R extends Message&gt; Map&lt;byte[], R&gt; batchCoprocessorService(<a name="line.667"></a>
<span class="sourceLineNo">668</span> Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey, byte[] endKey,<a name="line.668"></a>
<span class="sourceLineNo">669</span> R responsePrototype) throws ServiceException, Throwable {<a name="line.669"></a>
<span class="sourceLineNo">670</span> final Map&lt;byte[], R&gt; results =<a name="line.670"></a>
<span class="sourceLineNo">671</span> Collections.synchronizedMap(new TreeMap&lt;byte[], R&gt;(Bytes.BYTES_COMPARATOR));<a name="line.671"></a>
<span class="sourceLineNo">672</span> batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype,<a name="line.672"></a>
<span class="sourceLineNo">673</span> new Batch.Callback&lt;R&gt;() {<a name="line.673"></a>
<span class="sourceLineNo">674</span> @Override<a name="line.674"></a>
<span class="sourceLineNo">675</span> public void update(byte[] region, byte[] row, R result) {<a name="line.675"></a>
<span class="sourceLineNo">676</span> if (region != null) {<a name="line.676"></a>
<span class="sourceLineNo">677</span> results.put(region, result);<a name="line.677"></a>
<span class="sourceLineNo">678</span> }<a name="line.678"></a>
<span class="sourceLineNo">679</span> }<a name="line.679"></a>
<span class="sourceLineNo">680</span> });<a name="line.680"></a>
<span class="sourceLineNo">681</span> return results;<a name="line.681"></a>
<span class="sourceLineNo">682</span> }<a name="line.682"></a>
<span class="sourceLineNo">683</span><a name="line.683"></a>
<span class="sourceLineNo">684</span> /**<a name="line.684"></a>
<span class="sourceLineNo">685</span> * Creates an instance of the given {@link Service} subclass for each table region spanning the<a name="line.685"></a>
<span class="sourceLineNo">686</span> * range from the {@code startKey} row to {@code endKey} row (inclusive), all the invocations to<a name="line.686"></a>
<span class="sourceLineNo">687</span> * the same region server will be batched into one call. The coprocessor service is invoked<a name="line.687"></a>
<span class="sourceLineNo">688</span> * according to the service instance, method name and parameters.<a name="line.688"></a>
<span class="sourceLineNo">689</span> * &lt;p/&gt;<a name="line.689"></a>
<span class="sourceLineNo">690</span> * The given<a name="line.690"></a>
<span class="sourceLineNo">691</span> * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)}<a name="line.691"></a>
<span class="sourceLineNo">692</span> * method will be called with the return value from each region's invocation.<a name="line.692"></a>
<span class="sourceLineNo">693</span> * @param methodDescriptor the descriptor for the protobuf service method to call.<a name="line.693"></a>
<span class="sourceLineNo">694</span> * @param request the method call parameters<a name="line.694"></a>
<span class="sourceLineNo">695</span> * @param startKey start region selection with region containing this row. If<a name="line.695"></a>
<span class="sourceLineNo">696</span> * {@code null}, the selection will start with the first table region.<a name="line.696"></a>
<span class="sourceLineNo">697</span> * @param endKey select regions up to and including the region containing this row. If<a name="line.697"></a>
<span class="sourceLineNo">698</span> * {@code null}, selection will continue through the last table region.<a name="line.698"></a>
<span class="sourceLineNo">699</span> * @param responsePrototype the proto type of the response of the method in Service.<a name="line.699"></a>
<span class="sourceLineNo">700</span> * @param callback callback to invoke with the response for each region<a name="line.700"></a>
<span class="sourceLineNo">701</span> * @param &lt;R&gt; the response type for the coprocessor Service method<a name="line.701"></a>
<span class="sourceLineNo">702</span> * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking<a name="line.702"></a>
<span class="sourceLineNo">703</span> * interface for of a protobuf stub, so it is not possible to do it in an asynchronous<a name="line.703"></a>
<span class="sourceLineNo">704</span> * way, even if now we are building the {@link Table} implementation based on the<a name="line.704"></a>
<span class="sourceLineNo">705</span> * {@link AsyncTable}, which is not good. Use the coprocessorService methods in<a name="line.705"></a>
<span class="sourceLineNo">706</span> * {@link AsyncTable} directly instead.<a name="line.706"></a>
<span class="sourceLineNo">707</span> * @see Connection#toAsyncConnection()<a name="line.707"></a>
<span class="sourceLineNo">708</span> */<a name="line.708"></a>
<span class="sourceLineNo">709</span> @Deprecated<a name="line.709"></a>
<span class="sourceLineNo">710</span> default &lt;R extends Message&gt; void batchCoprocessorService(<a name="line.710"></a>
<span class="sourceLineNo">711</span> Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey, byte[] endKey,<a name="line.711"></a>
<span class="sourceLineNo">712</span> R responsePrototype, Batch.Callback&lt;R&gt; callback) throws ServiceException, Throwable {<a name="line.712"></a>
<span class="sourceLineNo">713</span> throw new NotImplementedException("Add an implementation!");<a name="line.713"></a>
<span class="sourceLineNo">714</span> }<a name="line.714"></a>
<span class="sourceLineNo">715</span><a name="line.715"></a>
<span class="sourceLineNo">716</span> /**<a name="line.716"></a>
<span class="sourceLineNo">717</span> * Get timeout of each rpc request in this Table instance. It will be overridden by a more<a name="line.717"></a>
<span class="sourceLineNo">718</span> * specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.<a name="line.718"></a>
<span class="sourceLineNo">719</span> * @see #getReadRpcTimeout(TimeUnit)<a name="line.719"></a>
<span class="sourceLineNo">720</span> * @see #getWriteRpcTimeout(TimeUnit)<a name="line.720"></a>
<span class="sourceLineNo">721</span> * @param unit the unit of time the timeout to be represented in<a name="line.721"></a>
<span class="sourceLineNo">722</span> * @return rpc timeout in the specified time unit<a name="line.722"></a>
<span class="sourceLineNo">723</span> */<a name="line.723"></a>
<span class="sourceLineNo">724</span> default long getRpcTimeout(TimeUnit unit) {<a name="line.724"></a>
<span class="sourceLineNo">725</span> throw new NotImplementedException("Add an implementation!");<a name="line.725"></a>
<span class="sourceLineNo">726</span> }<a name="line.726"></a>
<span class="sourceLineNo">727</span><a name="line.727"></a>
<span class="sourceLineNo">728</span> /**<a name="line.728"></a>
<span class="sourceLineNo">729</span> * Get timeout of each rpc read request in this Table instance.<a name="line.729"></a>
<span class="sourceLineNo">730</span> * @param unit the unit of time the timeout to be represented in<a name="line.730"></a>
<span class="sourceLineNo">731</span> * @return read rpc timeout in the specified time unit<a name="line.731"></a>
<span class="sourceLineNo">732</span> */<a name="line.732"></a>
<span class="sourceLineNo">733</span> default long getReadRpcTimeout(TimeUnit unit) {<a name="line.733"></a>
<span class="sourceLineNo">734</span> throw new NotImplementedException("Add an implementation!");<a name="line.734"></a>
<span class="sourceLineNo">735</span> }<a name="line.735"></a>
<span class="sourceLineNo">736</span><a name="line.736"></a>
<span class="sourceLineNo">737</span> /**<a name="line.737"></a>
<span class="sourceLineNo">738</span> * Get timeout of each rpc write request in this Table instance.<a name="line.738"></a>
<span class="sourceLineNo">739</span> * @param unit the unit of time the timeout to be represented in<a name="line.739"></a>
<span class="sourceLineNo">740</span> * @return write rpc timeout in the specified time unit<a name="line.740"></a>
<span class="sourceLineNo">741</span> */<a name="line.741"></a>
<span class="sourceLineNo">742</span> default long getWriteRpcTimeout(TimeUnit unit) {<a name="line.742"></a>
<span class="sourceLineNo">743</span> throw new NotImplementedException("Add an implementation!");<a name="line.743"></a>
<span class="sourceLineNo">744</span> }<a name="line.744"></a>
<span class="sourceLineNo">745</span><a name="line.745"></a>
<span class="sourceLineNo">746</span> /**<a name="line.746"></a>
<span class="sourceLineNo">747</span> * Get timeout of each operation in Table instance.<a name="line.747"></a>
<span class="sourceLineNo">748</span> * @param unit the unit of time the timeout to be represented in<a name="line.748"></a>
<span class="sourceLineNo">749</span> * @return operation rpc timeout in the specified time unit<a name="line.749"></a>
<span class="sourceLineNo">750</span> */<a name="line.750"></a>
<span class="sourceLineNo">751</span> default long getOperationTimeout(TimeUnit unit) {<a name="line.751"></a>
<span class="sourceLineNo">752</span> throw new NotImplementedException("Add an implementation!");<a name="line.752"></a>
<span class="sourceLineNo">753</span> }<a name="line.753"></a>
<span class="sourceLineNo">754</span><a name="line.754"></a>
<span class="sourceLineNo">755</span> /**<a name="line.755"></a>
<span class="sourceLineNo">756</span> * Get the attributes to be submitted with requests<a name="line.756"></a>
<span class="sourceLineNo">757</span> * @return map of request attributes<a name="line.757"></a>
<span class="sourceLineNo">758</span> */<a name="line.758"></a>
<span class="sourceLineNo">759</span> default Map&lt;String, byte[]&gt; getRequestAttributes() {<a name="line.759"></a>
<span class="sourceLineNo">760</span> throw new NotImplementedException("Add an implementation!");<a name="line.760"></a>
<span class="sourceLineNo">761</span> }<a name="line.761"></a>
<span class="sourceLineNo">762</span>}<a name="line.762"></a>
</pre>
</div>
</body>
</html>