blob: 06dae022b3b219ae2606bf159260fcf6b2aa47ac [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.io.asyncfs, class: WrapperAsyncFSOutput">
<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.io.asyncfs;</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.nio.ByteBuffer;</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.ExecutorService;</span>
<span class="source-line-no">024</span><span id="line-24">import java.util.concurrent.Executors;</span>
<span class="source-line-no">025</span><span id="line-25">import org.apache.hadoop.fs.FSDataOutputStream;</span>
<span class="source-line-no">026</span><span id="line-26">import org.apache.hadoop.fs.Path;</span>
<span class="source-line-no">027</span><span id="line-27">import org.apache.hadoop.hbase.io.ByteArrayOutputStream;</span>
<span class="source-line-no">028</span><span id="line-28">import org.apache.hadoop.hbase.util.CancelableProgressable;</span>
<span class="source-line-no">029</span><span id="line-29">import org.apache.hadoop.hdfs.protocol.DatanodeInfo;</span>
<span class="source-line-no">030</span><span id="line-30">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">031</span><span id="line-31"></span>
<span class="source-line-no">032</span><span id="line-32">import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;</span>
<span class="source-line-no">033</span><span id="line-33">import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;</span>
<span class="source-line-no">034</span><span id="line-34"></span>
<span class="source-line-no">035</span><span id="line-35">/**</span>
<span class="source-line-no">036</span><span id="line-36"> * An {@link AsyncFSOutput} wraps a {@link FSDataOutputStream}.</span>
<span class="source-line-no">037</span><span id="line-37"> */</span>
<span class="source-line-no">038</span><span id="line-38">@InterfaceAudience.Private</span>
<span class="source-line-no">039</span><span id="line-39">public class WrapperAsyncFSOutput implements AsyncFSOutput {</span>
<span class="source-line-no">040</span><span id="line-40"></span>
<span class="source-line-no">041</span><span id="line-41"> private final FSDataOutputStream out;</span>
<span class="source-line-no">042</span><span id="line-42"></span>
<span class="source-line-no">043</span><span id="line-43"> private ByteArrayOutputStream buffer = new ByteArrayOutputStream();</span>
<span class="source-line-no">044</span><span id="line-44"></span>
<span class="source-line-no">045</span><span id="line-45"> private final ExecutorService executor;</span>
<span class="source-line-no">046</span><span id="line-46"></span>
<span class="source-line-no">047</span><span id="line-47"> private volatile long syncedLength = 0;</span>
<span class="source-line-no">048</span><span id="line-48"></span>
<span class="source-line-no">049</span><span id="line-49"> public WrapperAsyncFSOutput(Path file, FSDataOutputStream out) {</span>
<span class="source-line-no">050</span><span id="line-50"> this.out = out;</span>
<span class="source-line-no">051</span><span id="line-51"> this.executor = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder().setDaemon(true)</span>
<span class="source-line-no">052</span><span id="line-52"> .setNameFormat("AsyncFSOutputFlusher-" + file.toString().replace("%", "%%")).build());</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"> @Override</span>
<span class="source-line-no">056</span><span id="line-56"> public void write(byte[] b) {</span>
<span class="source-line-no">057</span><span id="line-57"> write(b, 0, b.length);</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"> @Override</span>
<span class="source-line-no">061</span><span id="line-61"> public void write(byte[] b, int off, int len) {</span>
<span class="source-line-no">062</span><span id="line-62"> buffer.write(b, off, len);</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"> @Override</span>
<span class="source-line-no">066</span><span id="line-66"> public void writeInt(int i) {</span>
<span class="source-line-no">067</span><span id="line-67"> buffer.writeInt(i);</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"> @Override</span>
<span class="source-line-no">071</span><span id="line-71"> public void write(ByteBuffer bb) {</span>
<span class="source-line-no">072</span><span id="line-72"> buffer.write(bb, bb.position(), bb.remaining());</span>
<span class="source-line-no">073</span><span id="line-73"> }</span>
<span class="source-line-no">074</span><span id="line-74"></span>
<span class="source-line-no">075</span><span id="line-75"> @Override</span>
<span class="source-line-no">076</span><span id="line-76"> public int buffered() {</span>
<span class="source-line-no">077</span><span id="line-77"> return buffer.size();</span>
<span class="source-line-no">078</span><span id="line-78"> }</span>
<span class="source-line-no">079</span><span id="line-79"></span>
<span class="source-line-no">080</span><span id="line-80"> @Override</span>
<span class="source-line-no">081</span><span id="line-81"> public DatanodeInfo[] getPipeline() {</span>
<span class="source-line-no">082</span><span id="line-82"> return new DatanodeInfo[0];</span>
<span class="source-line-no">083</span><span id="line-83"> }</span>
<span class="source-line-no">084</span><span id="line-84"></span>
<span class="source-line-no">085</span><span id="line-85"> private void flush0(CompletableFuture&lt;Long&gt; future, ByteArrayOutputStream buffer, boolean sync) {</span>
<span class="source-line-no">086</span><span id="line-86"> try {</span>
<span class="source-line-no">087</span><span id="line-87"> if (buffer.size() &gt; 0) {</span>
<span class="source-line-no">088</span><span id="line-88"> out.write(buffer.getBuffer(), 0, buffer.size());</span>
<span class="source-line-no">089</span><span id="line-89"> if (sync) {</span>
<span class="source-line-no">090</span><span id="line-90"> out.hsync();</span>
<span class="source-line-no">091</span><span id="line-91"> } else {</span>
<span class="source-line-no">092</span><span id="line-92"> out.hflush();</span>
<span class="source-line-no">093</span><span id="line-93"> }</span>
<span class="source-line-no">094</span><span id="line-94"> }</span>
<span class="source-line-no">095</span><span id="line-95"> long pos = out.getPos();</span>
<span class="source-line-no">096</span><span id="line-96"> /**</span>
<span class="source-line-no">097</span><span id="line-97"> * This flush0 method could only be called by single thread, so here we could safely overwrite</span>
<span class="source-line-no">098</span><span id="line-98"> * without any synchronization.</span>
<span class="source-line-no">099</span><span id="line-99"> */</span>
<span class="source-line-no">100</span><span id="line-100"> this.syncedLength = pos;</span>
<span class="source-line-no">101</span><span id="line-101"> future.complete(pos);</span>
<span class="source-line-no">102</span><span id="line-102"> } catch (IOException e) {</span>
<span class="source-line-no">103</span><span id="line-103"> future.completeExceptionally(e);</span>
<span class="source-line-no">104</span><span id="line-104"> return;</span>
<span class="source-line-no">105</span><span id="line-105"> }</span>
<span class="source-line-no">106</span><span id="line-106"> }</span>
<span class="source-line-no">107</span><span id="line-107"></span>
<span class="source-line-no">108</span><span id="line-108"> @Override</span>
<span class="source-line-no">109</span><span id="line-109"> public CompletableFuture&lt;Long&gt; flush(boolean sync) {</span>
<span class="source-line-no">110</span><span id="line-110"> CompletableFuture&lt;Long&gt; future = new CompletableFuture&lt;&gt;();</span>
<span class="source-line-no">111</span><span id="line-111"> ByteArrayOutputStream buffer = this.buffer;</span>
<span class="source-line-no">112</span><span id="line-112"> this.buffer = new ByteArrayOutputStream();</span>
<span class="source-line-no">113</span><span id="line-113"> executor.execute(() -&gt; flush0(future, buffer, sync));</span>
<span class="source-line-no">114</span><span id="line-114"> return future;</span>
<span class="source-line-no">115</span><span id="line-115"> }</span>
<span class="source-line-no">116</span><span id="line-116"></span>
<span class="source-line-no">117</span><span id="line-117"> @Override</span>
<span class="source-line-no">118</span><span id="line-118"> public void recoverAndClose(CancelableProgressable reporter) throws IOException {</span>
<span class="source-line-no">119</span><span id="line-119"> executor.shutdown();</span>
<span class="source-line-no">120</span><span id="line-120"> out.close();</span>
<span class="source-line-no">121</span><span id="line-121"> }</span>
<span class="source-line-no">122</span><span id="line-122"></span>
<span class="source-line-no">123</span><span id="line-123"> @Override</span>
<span class="source-line-no">124</span><span id="line-124"> public void close() throws IOException {</span>
<span class="source-line-no">125</span><span id="line-125"> Preconditions.checkState(buffer.size() == 0, "should call flush first before calling close");</span>
<span class="source-line-no">126</span><span id="line-126"> executor.shutdown();</span>
<span class="source-line-no">127</span><span id="line-127"> out.close();</span>
<span class="source-line-no">128</span><span id="line-128"> }</span>
<span class="source-line-no">129</span><span id="line-129"></span>
<span class="source-line-no">130</span><span id="line-130"> @Override</span>
<span class="source-line-no">131</span><span id="line-131"> public boolean isBroken() {</span>
<span class="source-line-no">132</span><span id="line-132"> return false;</span>
<span class="source-line-no">133</span><span id="line-133"> }</span>
<span class="source-line-no">134</span><span id="line-134"></span>
<span class="source-line-no">135</span><span id="line-135"> @Override</span>
<span class="source-line-no">136</span><span id="line-136"> public long getSyncedLength() {</span>
<span class="source-line-no">137</span><span id="line-137"> return this.syncedLength;</span>
<span class="source-line-no">138</span><span id="line-138"> }</span>
<span class="source-line-no">139</span><span id="line-139">}</span>
</pre>
</div>
</main>
</body>
</html>