blob: f687570043e1e3603650ed60e98f58bbde7e77de [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.client.example, class: MultiThreadedClientExample, class: ReadExampleCallable">
<meta name="generator" content="javadoc/SourceToHTMLConverter">
<link rel="stylesheet" type="text/css" href="../../../../../../../stylesheet.css" title="Style">
</head>
<body class="source-page">
<main role="main">
<div class="source-container">
<pre><span class="source-line-no">001</span><span id="line-1">/*</span>
<span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span>
<span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span>
<span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span>
<span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span>
<span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span>
<span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span>
<span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span>
<span class="source-line-no">009</span><span id="line-9"> *</span>
<span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="source-line-no">011</span><span id="line-11"> *</span>
<span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span>
<span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span>
<span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span>
<span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span>
<span class="source-line-no">017</span><span id="line-17"> */</span>
<span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.client.example;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import java.io.IOException;</span>
<span class="source-line-no">021</span><span id="line-21">import java.util.ArrayList;</span>
<span class="source-line-no">022</span><span id="line-22">import java.util.List;</span>
<span class="source-line-no">023</span><span id="line-23">import java.util.concurrent.Callable;</span>
<span class="source-line-no">024</span><span id="line-24">import java.util.concurrent.ExecutorService;</span>
<span class="source-line-no">025</span><span id="line-25">import java.util.concurrent.Executors;</span>
<span class="source-line-no">026</span><span id="line-26">import java.util.concurrent.Future;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.concurrent.LinkedBlockingQueue;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.concurrent.ThreadFactory;</span>
<span class="source-line-no">029</span><span id="line-29">import java.util.concurrent.ThreadLocalRandom;</span>
<span class="source-line-no">030</span><span id="line-30">import java.util.concurrent.ThreadPoolExecutor;</span>
<span class="source-line-no">031</span><span id="line-31">import java.util.concurrent.TimeUnit;</span>
<span class="source-line-no">032</span><span id="line-32">import org.apache.hadoop.conf.Configured;</span>
<span class="source-line-no">033</span><span id="line-33">import org.apache.hadoop.hbase.Cell;</span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.hbase.CellBuilderFactory;</span>
<span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.hbase.CellBuilderType;</span>
<span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.hbase.TableName;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.hadoop.hbase.client.Connection;</span>
<span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.hbase.client.ConnectionFactory;</span>
<span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.hbase.client.Put;</span>
<span class="source-line-no">040</span><span id="line-40">import org.apache.hadoop.hbase.client.RegionLocator;</span>
<span class="source-line-no">041</span><span id="line-41">import org.apache.hadoop.hbase.client.Result;</span>
<span class="source-line-no">042</span><span id="line-42">import org.apache.hadoop.hbase.client.ResultScanner;</span>
<span class="source-line-no">043</span><span id="line-43">import org.apache.hadoop.hbase.client.Scan;</span>
<span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.hbase.client.Table;</span>
<span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.hbase.filter.KeyOnlyFilter;</span>
<span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.util.Tool;</span>
<span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.util.ToolRunner;</span>
<span class="source-line-no">049</span><span id="line-49">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">050</span><span id="line-50">import org.slf4j.Logger;</span>
<span class="source-line-no">051</span><span id="line-51">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">052</span><span id="line-52"></span>
<span class="source-line-no">053</span><span id="line-53">import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;</span>
<span class="source-line-no">054</span><span id="line-54"></span>
<span class="source-line-no">055</span><span id="line-55">/**</span>
<span class="source-line-no">056</span><span id="line-56"> * Example on how to use HBase's {@link Connection} and {@link Table} in a multi-threaded</span>
<span class="source-line-no">057</span><span id="line-57"> * environment. Each table is a light weight object that is created and thrown away. Connections are</span>
<span class="source-line-no">058</span><span id="line-58"> * heavy weight objects that hold on to zookeeper connections, async processes, and other state.</span>
<span class="source-line-no">059</span><span id="line-59"> *</span>
<span class="source-line-no">060</span><span id="line-60"> * &lt;pre&gt;</span>
<span class="source-line-no">061</span><span id="line-61"> * Usage:</span>
<span class="source-line-no">062</span><span id="line-62"> * bin/hbase org.apache.hadoop.hbase.client.example.MultiThreadedClientExample testTableName 500000</span>
<span class="source-line-no">063</span><span id="line-63"> * &lt;/pre&gt;</span>
<span class="source-line-no">064</span><span id="line-64"> * &lt;p&gt;</span>
<span class="source-line-no">065</span><span id="line-65"> * The table should already be created before running the command. This example expects one column</span>
<span class="source-line-no">066</span><span id="line-66"> * family named d.</span>
<span class="source-line-no">067</span><span id="line-67"> * &lt;/p&gt;</span>
<span class="source-line-no">068</span><span id="line-68"> * &lt;p&gt;</span>
<span class="source-line-no">069</span><span id="line-69"> * This is meant to show different operations that are likely to be done in a real world</span>
<span class="source-line-no">070</span><span id="line-70"> * application. These operations are:</span>
<span class="source-line-no">071</span><span id="line-71"> * &lt;/p&gt;</span>
<span class="source-line-no">072</span><span id="line-72"> * &lt;ul&gt;</span>
<span class="source-line-no">073</span><span id="line-73"> * &lt;li&gt;30% of all operations performed are batch writes. 30 puts are created and sent out at a time.</span>
<span class="source-line-no">074</span><span id="line-74"> * The response for all puts is waited on.&lt;/li&gt;</span>
<span class="source-line-no">075</span><span id="line-75"> * &lt;li&gt;20% of all operations are single writes. A single put is sent out and the response is waited</span>
<span class="source-line-no">076</span><span id="line-76"> * for.&lt;/li&gt;</span>
<span class="source-line-no">077</span><span id="line-77"> * &lt;li&gt;50% of all operations are scans. These scans start at a random place and scan up to 100 rows.</span>
<span class="source-line-no">078</span><span id="line-78"> * &lt;/li&gt;</span>
<span class="source-line-no">079</span><span id="line-79"> * &lt;/ul&gt;</span>
<span class="source-line-no">080</span><span id="line-80"> */</span>
<span class="source-line-no">081</span><span id="line-81">@InterfaceAudience.Private</span>
<span class="source-line-no">082</span><span id="line-82">public class MultiThreadedClientExample extends Configured implements Tool {</span>
<span class="source-line-no">083</span><span id="line-83"> private static final Logger LOG = LoggerFactory.getLogger(MultiThreadedClientExample.class);</span>
<span class="source-line-no">084</span><span id="line-84"> private static final int DEFAULT_NUM_OPERATIONS = 500000;</span>
<span class="source-line-no">085</span><span id="line-85"></span>
<span class="source-line-no">086</span><span id="line-86"> /**</span>
<span class="source-line-no">087</span><span id="line-87"> * The name of the column family. d for default.</span>
<span class="source-line-no">088</span><span id="line-88"> */</span>
<span class="source-line-no">089</span><span id="line-89"> private static final byte[] FAMILY = Bytes.toBytes("d");</span>
<span class="source-line-no">090</span><span id="line-90"></span>
<span class="source-line-no">091</span><span id="line-91"> /**</span>
<span class="source-line-no">092</span><span id="line-92"> * For the example we're just using one qualifier.</span>
<span class="source-line-no">093</span><span id="line-93"> */</span>
<span class="source-line-no">094</span><span id="line-94"> private static final byte[] QUAL = Bytes.toBytes("test");</span>
<span class="source-line-no">095</span><span id="line-95"></span>
<span class="source-line-no">096</span><span id="line-96"> private final ExecutorService internalPool;</span>
<span class="source-line-no">097</span><span id="line-97"></span>
<span class="source-line-no">098</span><span id="line-98"> private final int threads;</span>
<span class="source-line-no">099</span><span id="line-99"></span>
<span class="source-line-no">100</span><span id="line-100"> public MultiThreadedClientExample() throws IOException {</span>
<span class="source-line-no">101</span><span id="line-101"> // Base number of threads.</span>
<span class="source-line-no">102</span><span id="line-102"> // This represents the number of threads you application has</span>
<span class="source-line-no">103</span><span id="line-103"> // that can be interacting with an hbase client.</span>
<span class="source-line-no">104</span><span id="line-104"> this.threads = Runtime.getRuntime().availableProcessors() * 4;</span>
<span class="source-line-no">105</span><span id="line-105"></span>
<span class="source-line-no">106</span><span id="line-106"> // Daemon threads are great for things that get shut down.</span>
<span class="source-line-no">107</span><span id="line-107"> ThreadFactory threadFactory =</span>
<span class="source-line-no">108</span><span id="line-108"> new ThreadFactoryBuilder().setDaemon(true).setNameFormat("internal-pol-%d").build();</span>
<span class="source-line-no">109</span><span id="line-109"></span>
<span class="source-line-no">110</span><span id="line-110"> this.internalPool = Executors.newFixedThreadPool(threads, threadFactory);</span>
<span class="source-line-no">111</span><span id="line-111"> }</span>
<span class="source-line-no">112</span><span id="line-112"></span>
<span class="source-line-no">113</span><span id="line-113"> @Override</span>
<span class="source-line-no">114</span><span id="line-114"> public int run(String[] args) throws Exception {</span>
<span class="source-line-no">115</span><span id="line-115"></span>
<span class="source-line-no">116</span><span id="line-116"> if (args.length &lt; 1 || args.length &gt; 2) {</span>
<span class="source-line-no">117</span><span id="line-117"> System.out.println("Usage: " + this.getClass().getName() + " tableName [num_operations]");</span>
<span class="source-line-no">118</span><span id="line-118"> return -1;</span>
<span class="source-line-no">119</span><span id="line-119"> }</span>
<span class="source-line-no">120</span><span id="line-120"></span>
<span class="source-line-no">121</span><span id="line-121"> final TableName tableName = TableName.valueOf(args[0]);</span>
<span class="source-line-no">122</span><span id="line-122"> int numOperations = DEFAULT_NUM_OPERATIONS;</span>
<span class="source-line-no">123</span><span id="line-123"></span>
<span class="source-line-no">124</span><span id="line-124"> // the second arg is the number of operations to send.</span>
<span class="source-line-no">125</span><span id="line-125"> if (args.length == 2) {</span>
<span class="source-line-no">126</span><span id="line-126"> numOperations = Integer.parseInt(args[1]);</span>
<span class="source-line-no">127</span><span id="line-127"> }</span>
<span class="source-line-no">128</span><span id="line-128"></span>
<span class="source-line-no">129</span><span id="line-129"> // Threads for the client only.</span>
<span class="source-line-no">130</span><span id="line-130"> //</span>
<span class="source-line-no">131</span><span id="line-131"> // We don't want to mix hbase and business logic.</span>
<span class="source-line-no">132</span><span id="line-132"> //</span>
<span class="source-line-no">133</span><span id="line-133"> ThreadPoolExecutor service = new ThreadPoolExecutor(threads * 2, threads * 2, 60L,</span>
<span class="source-line-no">134</span><span id="line-134"> TimeUnit.SECONDS, new LinkedBlockingQueue&lt;&gt;());</span>
<span class="source-line-no">135</span><span id="line-135"></span>
<span class="source-line-no">136</span><span id="line-136"> // Create two different connections showing how it's possible to</span>
<span class="source-line-no">137</span><span id="line-137"> // separate different types of requests onto different connections</span>
<span class="source-line-no">138</span><span id="line-138"> final Connection writeConnection = ConnectionFactory.createConnection(getConf(), service);</span>
<span class="source-line-no">139</span><span id="line-139"> final Connection readConnection = ConnectionFactory.createConnection(getConf(), service);</span>
<span class="source-line-no">140</span><span id="line-140"></span>
<span class="source-line-no">141</span><span id="line-141"> // At this point the entire cache for the region locations is full.</span>
<span class="source-line-no">142</span><span id="line-142"> // Only do this if the number of regions in a table is easy to fit into memory.</span>
<span class="source-line-no">143</span><span id="line-143"> //</span>
<span class="source-line-no">144</span><span id="line-144"> // If you are interacting with more than 25k regions on a client then it's probably not good</span>
<span class="source-line-no">145</span><span id="line-145"> // to do this at all.</span>
<span class="source-line-no">146</span><span id="line-146"> warmUpConnectionCache(readConnection, tableName);</span>
<span class="source-line-no">147</span><span id="line-147"> warmUpConnectionCache(writeConnection, tableName);</span>
<span class="source-line-no">148</span><span id="line-148"></span>
<span class="source-line-no">149</span><span id="line-149"> List&lt;Future&lt;Boolean&gt;&gt; futures = new ArrayList&lt;&gt;(numOperations);</span>
<span class="source-line-no">150</span><span id="line-150"> for (int i = 0; i &lt; numOperations; i++) {</span>
<span class="source-line-no">151</span><span id="line-151"> double r = ThreadLocalRandom.current().nextDouble();</span>
<span class="source-line-no">152</span><span id="line-152"> Future&lt;Boolean&gt; f;</span>
<span class="source-line-no">153</span><span id="line-153"></span>
<span class="source-line-no">154</span><span id="line-154"> // For the sake of generating some synthetic load this queues</span>
<span class="source-line-no">155</span><span id="line-155"> // some different callables.</span>
<span class="source-line-no">156</span><span id="line-156"> // These callables are meant to represent real work done by your application.</span>
<span class="source-line-no">157</span><span id="line-157"> if (r &lt; .30) {</span>
<span class="source-line-no">158</span><span id="line-158"> f = internalPool.submit(new WriteExampleCallable(writeConnection, tableName));</span>
<span class="source-line-no">159</span><span id="line-159"> } else if (r &lt; .50) {</span>
<span class="source-line-no">160</span><span id="line-160"> f = internalPool.submit(new SingleWriteExampleCallable(writeConnection, tableName));</span>
<span class="source-line-no">161</span><span id="line-161"> } else {</span>
<span class="source-line-no">162</span><span id="line-162"> f = internalPool.submit(new ReadExampleCallable(writeConnection, tableName));</span>
<span class="source-line-no">163</span><span id="line-163"> }</span>
<span class="source-line-no">164</span><span id="line-164"> futures.add(f);</span>
<span class="source-line-no">165</span><span id="line-165"> }</span>
<span class="source-line-no">166</span><span id="line-166"></span>
<span class="source-line-no">167</span><span id="line-167"> // Wait a long time for all the reads/writes to complete</span>
<span class="source-line-no">168</span><span id="line-168"> for (Future&lt;Boolean&gt; f : futures) {</span>
<span class="source-line-no">169</span><span id="line-169"> f.get(10, TimeUnit.MINUTES);</span>
<span class="source-line-no">170</span><span id="line-170"> }</span>
<span class="source-line-no">171</span><span id="line-171"></span>
<span class="source-line-no">172</span><span id="line-172"> // Clean up after our selves for cleanliness</span>
<span class="source-line-no">173</span><span id="line-173"> internalPool.shutdownNow();</span>
<span class="source-line-no">174</span><span id="line-174"> service.shutdownNow();</span>
<span class="source-line-no">175</span><span id="line-175"> return 0;</span>
<span class="source-line-no">176</span><span id="line-176"> }</span>
<span class="source-line-no">177</span><span id="line-177"></span>
<span class="source-line-no">178</span><span id="line-178"> private void warmUpConnectionCache(Connection connection, TableName tn) throws IOException {</span>
<span class="source-line-no">179</span><span id="line-179"> try (RegionLocator locator = connection.getRegionLocator(tn)) {</span>
<span class="source-line-no">180</span><span id="line-180"> LOG.info("Warmed up region location cache for " + tn + " got "</span>
<span class="source-line-no">181</span><span id="line-181"> + locator.getAllRegionLocations().size());</span>
<span class="source-line-no">182</span><span id="line-182"> }</span>
<span class="source-line-no">183</span><span id="line-183"> }</span>
<span class="source-line-no">184</span><span id="line-184"></span>
<span class="source-line-no">185</span><span id="line-185"> /**</span>
<span class="source-line-no">186</span><span id="line-186"> * Class that will show how to send batches of puts at the same time.</span>
<span class="source-line-no">187</span><span id="line-187"> */</span>
<span class="source-line-no">188</span><span id="line-188"> public static class WriteExampleCallable implements Callable&lt;Boolean&gt; {</span>
<span class="source-line-no">189</span><span id="line-189"> private final Connection connection;</span>
<span class="source-line-no">190</span><span id="line-190"> private final TableName tableName;</span>
<span class="source-line-no">191</span><span id="line-191"></span>
<span class="source-line-no">192</span><span id="line-192"> public WriteExampleCallable(Connection connection, TableName tableName) {</span>
<span class="source-line-no">193</span><span id="line-193"> this.connection = connection;</span>
<span class="source-line-no">194</span><span id="line-194"> this.tableName = tableName;</span>
<span class="source-line-no">195</span><span id="line-195"> }</span>
<span class="source-line-no">196</span><span id="line-196"></span>
<span class="source-line-no">197</span><span id="line-197"> @Override</span>
<span class="source-line-no">198</span><span id="line-198"> public Boolean call() throws Exception {</span>
<span class="source-line-no">199</span><span id="line-199"></span>
<span class="source-line-no">200</span><span id="line-200"> // Table implements Closable so we use the try with resource structure here.</span>
<span class="source-line-no">201</span><span id="line-201"> // https://docs.oracle.com/javase/tutorial/essential/exceptions/tryResourceClose.html</span>
<span class="source-line-no">202</span><span id="line-202"> try (Table t = connection.getTable(tableName)) {</span>
<span class="source-line-no">203</span><span id="line-203"> byte[] value = Bytes.toBytes(Double.toString(ThreadLocalRandom.current().nextDouble()));</span>
<span class="source-line-no">204</span><span id="line-204"> int rows = 30;</span>
<span class="source-line-no">205</span><span id="line-205"></span>
<span class="source-line-no">206</span><span id="line-206"> // Array to put the batch</span>
<span class="source-line-no">207</span><span id="line-207"> ArrayList&lt;Put&gt; puts = new ArrayList&lt;&gt;(rows);</span>
<span class="source-line-no">208</span><span id="line-208"> for (int i = 0; i &lt; 30; i++) {</span>
<span class="source-line-no">209</span><span id="line-209"> byte[] rk = Bytes.toBytes(ThreadLocalRandom.current().nextLong());</span>
<span class="source-line-no">210</span><span id="line-210"> Put p = new Put(rk);</span>
<span class="source-line-no">211</span><span id="line-211"> p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(rk).setFamily(FAMILY)</span>
<span class="source-line-no">212</span><span id="line-212"> .setQualifier(QUAL).setTimestamp(p.getTimestamp()).setType(Cell.Type.Put)</span>
<span class="source-line-no">213</span><span id="line-213"> .setValue(value).build());</span>
<span class="source-line-no">214</span><span id="line-214"> puts.add(p);</span>
<span class="source-line-no">215</span><span id="line-215"> }</span>
<span class="source-line-no">216</span><span id="line-216"></span>
<span class="source-line-no">217</span><span id="line-217"> // now that we've assembled the batch it's time to push it to hbase.</span>
<span class="source-line-no">218</span><span id="line-218"> t.put(puts);</span>
<span class="source-line-no">219</span><span id="line-219"> }</span>
<span class="source-line-no">220</span><span id="line-220"> return true;</span>
<span class="source-line-no">221</span><span id="line-221"> }</span>
<span class="source-line-no">222</span><span id="line-222"> }</span>
<span class="source-line-no">223</span><span id="line-223"></span>
<span class="source-line-no">224</span><span id="line-224"> /**</span>
<span class="source-line-no">225</span><span id="line-225"> * Class to show how to send a single put.</span>
<span class="source-line-no">226</span><span id="line-226"> */</span>
<span class="source-line-no">227</span><span id="line-227"> public static class SingleWriteExampleCallable implements Callable&lt;Boolean&gt; {</span>
<span class="source-line-no">228</span><span id="line-228"> private final Connection connection;</span>
<span class="source-line-no">229</span><span id="line-229"> private final TableName tableName;</span>
<span class="source-line-no">230</span><span id="line-230"></span>
<span class="source-line-no">231</span><span id="line-231"> public SingleWriteExampleCallable(Connection connection, TableName tableName) {</span>
<span class="source-line-no">232</span><span id="line-232"> this.connection = connection;</span>
<span class="source-line-no">233</span><span id="line-233"> this.tableName = tableName;</span>
<span class="source-line-no">234</span><span id="line-234"> }</span>
<span class="source-line-no">235</span><span id="line-235"></span>
<span class="source-line-no">236</span><span id="line-236"> @Override</span>
<span class="source-line-no">237</span><span id="line-237"> public Boolean call() throws Exception {</span>
<span class="source-line-no">238</span><span id="line-238"> try (Table t = connection.getTable(tableName)) {</span>
<span class="source-line-no">239</span><span id="line-239"></span>
<span class="source-line-no">240</span><span id="line-240"> byte[] value = Bytes.toBytes(Double.toString(ThreadLocalRandom.current().nextDouble()));</span>
<span class="source-line-no">241</span><span id="line-241"> byte[] rk = Bytes.toBytes(ThreadLocalRandom.current().nextLong());</span>
<span class="source-line-no">242</span><span id="line-242"> Put p = new Put(rk);</span>
<span class="source-line-no">243</span><span id="line-243"> p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(rk).setFamily(FAMILY)</span>
<span class="source-line-no">244</span><span id="line-244"> .setQualifier(QUAL).setTimestamp(p.getTimestamp()).setType(Cell.Type.Put).setValue(value)</span>
<span class="source-line-no">245</span><span id="line-245"> .build());</span>
<span class="source-line-no">246</span><span id="line-246"> t.put(p);</span>
<span class="source-line-no">247</span><span id="line-247"> }</span>
<span class="source-line-no">248</span><span id="line-248"> return true;</span>
<span class="source-line-no">249</span><span id="line-249"> }</span>
<span class="source-line-no">250</span><span id="line-250"> }</span>
<span class="source-line-no">251</span><span id="line-251"></span>
<span class="source-line-no">252</span><span id="line-252"> /**</span>
<span class="source-line-no">253</span><span id="line-253"> * Class to show how to scan some rows starting at a random location.</span>
<span class="source-line-no">254</span><span id="line-254"> */</span>
<span class="source-line-no">255</span><span id="line-255"> public static class ReadExampleCallable implements Callable&lt;Boolean&gt; {</span>
<span class="source-line-no">256</span><span id="line-256"> private final Connection connection;</span>
<span class="source-line-no">257</span><span id="line-257"> private final TableName tableName;</span>
<span class="source-line-no">258</span><span id="line-258"></span>
<span class="source-line-no">259</span><span id="line-259"> public ReadExampleCallable(Connection connection, TableName tableName) {</span>
<span class="source-line-no">260</span><span id="line-260"> this.connection = connection;</span>
<span class="source-line-no">261</span><span id="line-261"> this.tableName = tableName;</span>
<span class="source-line-no">262</span><span id="line-262"> }</span>
<span class="source-line-no">263</span><span id="line-263"></span>
<span class="source-line-no">264</span><span id="line-264"> @Override</span>
<span class="source-line-no">265</span><span id="line-265"> public Boolean call() throws Exception {</span>
<span class="source-line-no">266</span><span id="line-266"></span>
<span class="source-line-no">267</span><span id="line-267"> // total length in bytes of all read rows.</span>
<span class="source-line-no">268</span><span id="line-268"> int result = 0;</span>
<span class="source-line-no">269</span><span id="line-269"></span>
<span class="source-line-no">270</span><span id="line-270"> // Number of rows the scan will read before being considered done.</span>
<span class="source-line-no">271</span><span id="line-271"> int toRead = 100;</span>
<span class="source-line-no">272</span><span id="line-272"> try (Table t = connection.getTable(tableName)) {</span>
<span class="source-line-no">273</span><span id="line-273"> byte[] rk = Bytes.toBytes(ThreadLocalRandom.current().nextLong());</span>
<span class="source-line-no">274</span><span id="line-274"> Scan s = new Scan().withStartRow(rk);</span>
<span class="source-line-no">275</span><span id="line-275"></span>
<span class="source-line-no">276</span><span id="line-276"> // This filter will keep the values from being sent accross the wire.</span>
<span class="source-line-no">277</span><span id="line-277"> // This is good for counting or other scans that are checking for</span>
<span class="source-line-no">278</span><span id="line-278"> // existence and don't rely on the value.</span>
<span class="source-line-no">279</span><span id="line-279"> s.setFilter(new KeyOnlyFilter());</span>
<span class="source-line-no">280</span><span id="line-280"></span>
<span class="source-line-no">281</span><span id="line-281"> // Don't go back to the server for every single row.</span>
<span class="source-line-no">282</span><span id="line-282"> // We know these rows are small. So ask for 20 at a time.</span>
<span class="source-line-no">283</span><span id="line-283"> // This would be application specific.</span>
<span class="source-line-no">284</span><span id="line-284"> //</span>
<span class="source-line-no">285</span><span id="line-285"> // The goal is to reduce round trips but asking for too</span>
<span class="source-line-no">286</span><span id="line-286"> // many rows can lead to GC problems on client and server sides.</span>
<span class="source-line-no">287</span><span id="line-287"> s.setCaching(20);</span>
<span class="source-line-no">288</span><span id="line-288"></span>
<span class="source-line-no">289</span><span id="line-289"> // Don't use the cache. While this is a silly test program it's still good to be</span>
<span class="source-line-no">290</span><span id="line-290"> // explicit that scans normally don't use the block cache.</span>
<span class="source-line-no">291</span><span id="line-291"> s.setCacheBlocks(false);</span>
<span class="source-line-no">292</span><span id="line-292"></span>
<span class="source-line-no">293</span><span id="line-293"> // Open up the scanner and close it automatically when done.</span>
<span class="source-line-no">294</span><span id="line-294"> try (ResultScanner rs = t.getScanner(s)) {</span>
<span class="source-line-no">295</span><span id="line-295"></span>
<span class="source-line-no">296</span><span id="line-296"> // Now go through rows.</span>
<span class="source-line-no">297</span><span id="line-297"> for (Result r : rs) {</span>
<span class="source-line-no">298</span><span id="line-298"> // Keep track of things size to simulate doing some real work.</span>
<span class="source-line-no">299</span><span id="line-299"> result += r.getRow().length;</span>
<span class="source-line-no">300</span><span id="line-300"> toRead -= 1;</span>
<span class="source-line-no">301</span><span id="line-301"></span>
<span class="source-line-no">302</span><span id="line-302"> // Most online applications won't be</span>
<span class="source-line-no">303</span><span id="line-303"> // reading the entire table so this break</span>
<span class="source-line-no">304</span><span id="line-304"> // simulates small to medium size scans,</span>
<span class="source-line-no">305</span><span id="line-305"> // without needing to know an end row.</span>
<span class="source-line-no">306</span><span id="line-306"> if (toRead &lt;= 0) {</span>
<span class="source-line-no">307</span><span id="line-307"> break;</span>
<span class="source-line-no">308</span><span id="line-308"> }</span>
<span class="source-line-no">309</span><span id="line-309"> }</span>
<span class="source-line-no">310</span><span id="line-310"> }</span>
<span class="source-line-no">311</span><span id="line-311"> }</span>
<span class="source-line-no">312</span><span id="line-312"> return result &gt; 0;</span>
<span class="source-line-no">313</span><span id="line-313"> }</span>
<span class="source-line-no">314</span><span id="line-314"> }</span>
<span class="source-line-no">315</span><span id="line-315"></span>
<span class="source-line-no">316</span><span id="line-316"> public static void main(String[] args) throws Exception {</span>
<span class="source-line-no">317</span><span id="line-317"> ToolRunner.run(new MultiThreadedClientExample(), args);</span>
<span class="source-line-no">318</span><span id="line-318"> }</span>
<span class="source-line-no">319</span><span id="line-319">}</span>
</pre>
</div>
</main>
</body>
</html>