blob: 2e4ac12a8248a7c77f13b8bd4f7fd833c808e370 [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: AsyncClientExample">
<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 static org.apache.hadoop.hbase.util.FutureUtils.addListener;</span>
<span class="source-line-no">021</span><span id="line-21"></span>
<span class="source-line-no">022</span><span id="line-22">import java.util.concurrent.CompletableFuture;</span>
<span class="source-line-no">023</span><span id="line-23">import java.util.concurrent.CountDownLatch;</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.atomic.AtomicReference;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.stream.IntStream;</span>
<span class="source-line-no">028</span><span id="line-28">import org.apache.commons.io.IOUtils;</span>
<span class="source-line-no">029</span><span id="line-29">import org.apache.hadoop.conf.Configured;</span>
<span class="source-line-no">030</span><span id="line-30">import org.apache.hadoop.hbase.TableName;</span>
<span class="source-line-no">031</span><span id="line-31">import org.apache.hadoop.hbase.client.AsyncConnection;</span>
<span class="source-line-no">032</span><span id="line-32">import org.apache.hadoop.hbase.client.AsyncTable;</span>
<span class="source-line-no">033</span><span id="line-33">import org.apache.hadoop.hbase.client.ConnectionFactory;</span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.hbase.client.Get;</span>
<span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.hbase.client.Put;</span>
<span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.hadoop.hbase.util.Threads;</span>
<span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.util.Tool;</span>
<span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.util.ToolRunner;</span>
<span class="source-line-no">040</span><span id="line-40">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">041</span><span id="line-41">import org.slf4j.Logger;</span>
<span class="source-line-no">042</span><span id="line-42">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">043</span><span id="line-43"></span>
<span class="source-line-no">044</span><span id="line-44">import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;</span>
<span class="source-line-no">045</span><span id="line-45"></span>
<span class="source-line-no">046</span><span id="line-46">/**</span>
<span class="source-line-no">047</span><span id="line-47"> * A simple example shows how to use asynchronous client.</span>
<span class="source-line-no">048</span><span id="line-48"> */</span>
<span class="source-line-no">049</span><span id="line-49">@InterfaceAudience.Private</span>
<span class="source-line-no">050</span><span id="line-50">public class AsyncClientExample extends Configured implements Tool {</span>
<span class="source-line-no">051</span><span id="line-51"></span>
<span class="source-line-no">052</span><span id="line-52"> private static final Logger LOG = LoggerFactory.getLogger(AsyncClientExample.class);</span>
<span class="source-line-no">053</span><span id="line-53"></span>
<span class="source-line-no">054</span><span id="line-54"> /**</span>
<span class="source-line-no">055</span><span id="line-55"> * The size for thread pool.</span>
<span class="source-line-no">056</span><span id="line-56"> */</span>
<span class="source-line-no">057</span><span id="line-57"> private static final int THREAD_POOL_SIZE = 16;</span>
<span class="source-line-no">058</span><span id="line-58"></span>
<span class="source-line-no">059</span><span id="line-59"> /**</span>
<span class="source-line-no">060</span><span id="line-60"> * The default number of operations.</span>
<span class="source-line-no">061</span><span id="line-61"> */</span>
<span class="source-line-no">062</span><span id="line-62"> private static final int DEFAULT_NUM_OPS = 100;</span>
<span class="source-line-no">063</span><span id="line-63"></span>
<span class="source-line-no">064</span><span id="line-64"> /**</span>
<span class="source-line-no">065</span><span id="line-65"> * The name of the column family. d for default.</span>
<span class="source-line-no">066</span><span id="line-66"> */</span>
<span class="source-line-no">067</span><span id="line-67"> private static final byte[] FAMILY = Bytes.toBytes("d");</span>
<span class="source-line-no">068</span><span id="line-68"></span>
<span class="source-line-no">069</span><span id="line-69"> /**</span>
<span class="source-line-no">070</span><span id="line-70"> * For the example we're just using one qualifier.</span>
<span class="source-line-no">071</span><span id="line-71"> */</span>
<span class="source-line-no">072</span><span id="line-72"> private static final byte[] QUAL = Bytes.toBytes("test");</span>
<span class="source-line-no">073</span><span id="line-73"></span>
<span class="source-line-no">074</span><span id="line-74"> private final AtomicReference&lt;CompletableFuture&lt;AsyncConnection&gt;&gt; future =</span>
<span class="source-line-no">075</span><span id="line-75"> new AtomicReference&lt;&gt;();</span>
<span class="source-line-no">076</span><span id="line-76"></span>
<span class="source-line-no">077</span><span id="line-77"> private CompletableFuture&lt;AsyncConnection&gt; getConn() {</span>
<span class="source-line-no">078</span><span id="line-78"> CompletableFuture&lt;AsyncConnection&gt; f = future.get();</span>
<span class="source-line-no">079</span><span id="line-79"> if (f != null) {</span>
<span class="source-line-no">080</span><span id="line-80"> return f;</span>
<span class="source-line-no">081</span><span id="line-81"> }</span>
<span class="source-line-no">082</span><span id="line-82"> for (;;) {</span>
<span class="source-line-no">083</span><span id="line-83"> if (future.compareAndSet(null, new CompletableFuture&lt;&gt;())) {</span>
<span class="source-line-no">084</span><span id="line-84"> CompletableFuture&lt;AsyncConnection&gt; toComplete = future.get();</span>
<span class="source-line-no">085</span><span id="line-85"> addListener(ConnectionFactory.createAsyncConnection(getConf()), (conn, error) -&gt; {</span>
<span class="source-line-no">086</span><span id="line-86"> if (error != null) {</span>
<span class="source-line-no">087</span><span id="line-87"> toComplete.completeExceptionally(error);</span>
<span class="source-line-no">088</span><span id="line-88"> // we need to reset the future holder so we will get a chance to recreate an async</span>
<span class="source-line-no">089</span><span id="line-89"> // connection at next try.</span>
<span class="source-line-no">090</span><span id="line-90"> future.set(null);</span>
<span class="source-line-no">091</span><span id="line-91"> return;</span>
<span class="source-line-no">092</span><span id="line-92"> }</span>
<span class="source-line-no">093</span><span id="line-93"> toComplete.complete(conn);</span>
<span class="source-line-no">094</span><span id="line-94"> });</span>
<span class="source-line-no">095</span><span id="line-95"> return toComplete;</span>
<span class="source-line-no">096</span><span id="line-96"> } else {</span>
<span class="source-line-no">097</span><span id="line-97"> f = future.get();</span>
<span class="source-line-no">098</span><span id="line-98"> if (f != null) {</span>
<span class="source-line-no">099</span><span id="line-99"> return f;</span>
<span class="source-line-no">100</span><span id="line-100"> }</span>
<span class="source-line-no">101</span><span id="line-101"> }</span>
<span class="source-line-no">102</span><span id="line-102"> }</span>
<span class="source-line-no">103</span><span id="line-103"> }</span>
<span class="source-line-no">104</span><span id="line-104"></span>
<span class="source-line-no">105</span><span id="line-105"> @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NONNULL_PARAM_VIOLATION",</span>
<span class="source-line-no">106</span><span id="line-106"> justification = "it is valid to pass NULL to CompletableFuture#completedFuture")</span>
<span class="source-line-no">107</span><span id="line-107"> private CompletableFuture&lt;Void&gt; closeConn() {</span>
<span class="source-line-no">108</span><span id="line-108"> CompletableFuture&lt;AsyncConnection&gt; f = future.get();</span>
<span class="source-line-no">109</span><span id="line-109"> if (f == null) {</span>
<span class="source-line-no">110</span><span id="line-110"> return CompletableFuture.completedFuture(null);</span>
<span class="source-line-no">111</span><span id="line-111"> }</span>
<span class="source-line-no">112</span><span id="line-112"> CompletableFuture&lt;Void&gt; closeFuture = new CompletableFuture&lt;&gt;();</span>
<span class="source-line-no">113</span><span id="line-113"> addListener(f, (conn, error) -&gt; {</span>
<span class="source-line-no">114</span><span id="line-114"> if (error == null) {</span>
<span class="source-line-no">115</span><span id="line-115"> IOUtils.closeQuietly(conn, e -&gt; LOG.warn("failed to close conn", e));</span>
<span class="source-line-no">116</span><span id="line-116"> }</span>
<span class="source-line-no">117</span><span id="line-117"> closeFuture.complete(null);</span>
<span class="source-line-no">118</span><span id="line-118"> });</span>
<span class="source-line-no">119</span><span id="line-119"> return closeFuture;</span>
<span class="source-line-no">120</span><span id="line-120"> }</span>
<span class="source-line-no">121</span><span id="line-121"></span>
<span class="source-line-no">122</span><span id="line-122"> private byte[] getKey(int i) {</span>
<span class="source-line-no">123</span><span id="line-123"> return Bytes.toBytes(String.format("%08x", i));</span>
<span class="source-line-no">124</span><span id="line-124"> }</span>
<span class="source-line-no">125</span><span id="line-125"></span>
<span class="source-line-no">126</span><span id="line-126"> @Override</span>
<span class="source-line-no">127</span><span id="line-127"> public int run(String[] args) throws Exception {</span>
<span class="source-line-no">128</span><span id="line-128"> if (args.length &lt; 1 || args.length &gt; 2) {</span>
<span class="source-line-no">129</span><span id="line-129"> System.out.println("Usage: " + this.getClass().getName() + " tableName [num_operations]");</span>
<span class="source-line-no">130</span><span id="line-130"> return -1;</span>
<span class="source-line-no">131</span><span id="line-131"> }</span>
<span class="source-line-no">132</span><span id="line-132"> TableName tableName = TableName.valueOf(args[0]);</span>
<span class="source-line-no">133</span><span id="line-133"> int numOps = args.length &gt; 1 ? Integer.parseInt(args[1]) : DEFAULT_NUM_OPS;</span>
<span class="source-line-no">134</span><span id="line-134"> ExecutorService threadPool = Executors.newFixedThreadPool(THREAD_POOL_SIZE,</span>
<span class="source-line-no">135</span><span id="line-135"> new ThreadFactoryBuilder().setNameFormat("AsyncClientExample-pool-%d").setDaemon(true)</span>
<span class="source-line-no">136</span><span id="line-136"> .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());</span>
<span class="source-line-no">137</span><span id="line-137"> // We use AsyncTable here so we need to provide a separated thread pool. RawAsyncTable does not</span>
<span class="source-line-no">138</span><span id="line-138"> // need a thread pool and may have a better performance if you use it correctly as it can save</span>
<span class="source-line-no">139</span><span id="line-139"> // some context switches. But if you use RawAsyncTable incorrectly, you may have a very bad</span>
<span class="source-line-no">140</span><span id="line-140"> // impact on performance so use it with caution.</span>
<span class="source-line-no">141</span><span id="line-141"> CountDownLatch latch = new CountDownLatch(numOps);</span>
<span class="source-line-no">142</span><span id="line-142"> IntStream.range(0, numOps).forEach(i -&gt; {</span>
<span class="source-line-no">143</span><span id="line-143"> CompletableFuture&lt;AsyncConnection&gt; future = getConn();</span>
<span class="source-line-no">144</span><span id="line-144"> addListener(future, (conn, error) -&gt; {</span>
<span class="source-line-no">145</span><span id="line-145"> if (error != null) {</span>
<span class="source-line-no">146</span><span id="line-146"> LOG.warn("failed to get async connection for " + i, error);</span>
<span class="source-line-no">147</span><span id="line-147"> latch.countDown();</span>
<span class="source-line-no">148</span><span id="line-148"> return;</span>
<span class="source-line-no">149</span><span id="line-149"> }</span>
<span class="source-line-no">150</span><span id="line-150"> AsyncTable&lt;?&gt; table = conn.getTable(tableName, threadPool);</span>
<span class="source-line-no">151</span><span id="line-151"> addListener(table.put(new Put(getKey(i)).addColumn(FAMILY, QUAL, Bytes.toBytes(i))),</span>
<span class="source-line-no">152</span><span id="line-152"> (putResp, putErr) -&gt; {</span>
<span class="source-line-no">153</span><span id="line-153"> if (putErr != null) {</span>
<span class="source-line-no">154</span><span id="line-154"> LOG.warn("put failed for " + i, putErr);</span>
<span class="source-line-no">155</span><span id="line-155"> latch.countDown();</span>
<span class="source-line-no">156</span><span id="line-156"> return;</span>
<span class="source-line-no">157</span><span id="line-157"> }</span>
<span class="source-line-no">158</span><span id="line-158"> LOG.info("put for " + i + " succeeded, try getting");</span>
<span class="source-line-no">159</span><span id="line-159"> addListener(table.get(new Get(getKey(i))), (result, getErr) -&gt; {</span>
<span class="source-line-no">160</span><span id="line-160"> if (getErr != null) {</span>
<span class="source-line-no">161</span><span id="line-161"> LOG.warn("get failed for " + i);</span>
<span class="source-line-no">162</span><span id="line-162"> latch.countDown();</span>
<span class="source-line-no">163</span><span id="line-163"> return;</span>
<span class="source-line-no">164</span><span id="line-164"> }</span>
<span class="source-line-no">165</span><span id="line-165"> if (result.isEmpty()) {</span>
<span class="source-line-no">166</span><span id="line-166"> LOG.warn("get failed for " + i + ", server returns empty result");</span>
<span class="source-line-no">167</span><span id="line-167"> } else if (!result.containsColumn(FAMILY, QUAL)) {</span>
<span class="source-line-no">168</span><span id="line-168"> LOG.warn("get failed for " + i + ", the result does not contain "</span>
<span class="source-line-no">169</span><span id="line-169"> + Bytes.toString(FAMILY) + ":" + Bytes.toString(QUAL));</span>
<span class="source-line-no">170</span><span id="line-170"> } else {</span>
<span class="source-line-no">171</span><span id="line-171"> int v = Bytes.toInt(result.getValue(FAMILY, QUAL));</span>
<span class="source-line-no">172</span><span id="line-172"> if (v != i) {</span>
<span class="source-line-no">173</span><span id="line-173"> LOG.warn("get failed for " + i + ", the value of " + Bytes.toString(FAMILY) + ":"</span>
<span class="source-line-no">174</span><span id="line-174"> + Bytes.toString(QUAL) + " is " + v + ", exected " + i);</span>
<span class="source-line-no">175</span><span id="line-175"> } else {</span>
<span class="source-line-no">176</span><span id="line-176"> LOG.info("get for " + i + " succeeded");</span>
<span class="source-line-no">177</span><span id="line-177"> }</span>
<span class="source-line-no">178</span><span id="line-178"> }</span>
<span class="source-line-no">179</span><span id="line-179"> latch.countDown();</span>
<span class="source-line-no">180</span><span id="line-180"> });</span>
<span class="source-line-no">181</span><span id="line-181"> });</span>
<span class="source-line-no">182</span><span id="line-182"> });</span>
<span class="source-line-no">183</span><span id="line-183"> });</span>
<span class="source-line-no">184</span><span id="line-184"> latch.await();</span>
<span class="source-line-no">185</span><span id="line-185"> closeConn().get();</span>
<span class="source-line-no">186</span><span id="line-186"> return 0;</span>
<span class="source-line-no">187</span><span id="line-187"> }</span>
<span class="source-line-no">188</span><span id="line-188"></span>
<span class="source-line-no">189</span><span id="line-189"> public static void main(String[] args) throws Exception {</span>
<span class="source-line-no">190</span><span id="line-190"> ToolRunner.run(new AsyncClientExample(), args);</span>
<span class="source-line-no">191</span><span id="line-191"> }</span>
<span class="source-line-no">192</span><span id="line-192">}</span>
</pre>
</div>
</main>
</body>
</html>