blob: 6a8092972faad1ab989b7f1f58efab1d3d58d6c5 [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: FanOutOneBlockAsyncDFSOutput, class: Callback">
<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 static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.HEART_BEAT_SEQNO;</span>
<span class="source-line-no">021</span><span id="line-21">import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.READ_TIMEOUT;</span>
<span class="source-line-no">022</span><span id="line-22">import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.completeFile;</span>
<span class="source-line-no">023</span><span id="line-23">import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.endFileLease;</span>
<span class="source-line-no">024</span><span id="line-24">import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.getStatus;</span>
<span class="source-line-no">025</span><span id="line-25">import static org.apache.hadoop.hbase.util.LocatedBlockHelper.getLocatedBlockLocations;</span>
<span class="source-line-no">026</span><span id="line-26">import static org.apache.hadoop.hbase.util.NettyFutureUtils.consume;</span>
<span class="source-line-no">027</span><span id="line-27">import static org.apache.hadoop.hbase.util.NettyFutureUtils.safeWrite;</span>
<span class="source-line-no">028</span><span id="line-28">import static org.apache.hadoop.hbase.util.NettyFutureUtils.safeWriteAndFlush;</span>
<span class="source-line-no">029</span><span id="line-29">import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;</span>
<span class="source-line-no">030</span><span id="line-30">import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.READER_IDLE;</span>
<span class="source-line-no">031</span><span id="line-31">import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.WRITER_IDLE;</span>
<span class="source-line-no">032</span><span id="line-32"></span>
<span class="source-line-no">033</span><span id="line-33">import com.google.errorprone.annotations.RestrictedApi;</span>
<span class="source-line-no">034</span><span id="line-34">import java.io.IOException;</span>
<span class="source-line-no">035</span><span id="line-35">import java.nio.ByteBuffer;</span>
<span class="source-line-no">036</span><span id="line-36">import java.util.ArrayList;</span>
<span class="source-line-no">037</span><span id="line-37">import java.util.Collection;</span>
<span class="source-line-no">038</span><span id="line-38">import java.util.Collections;</span>
<span class="source-line-no">039</span><span id="line-39">import java.util.Iterator;</span>
<span class="source-line-no">040</span><span id="line-40">import java.util.List;</span>
<span class="source-line-no">041</span><span id="line-41">import java.util.Map;</span>
<span class="source-line-no">042</span><span id="line-42">import java.util.Set;</span>
<span class="source-line-no">043</span><span id="line-43">import java.util.concurrent.CompletableFuture;</span>
<span class="source-line-no">044</span><span id="line-44">import java.util.concurrent.ConcurrentHashMap;</span>
<span class="source-line-no">045</span><span id="line-45">import java.util.concurrent.ConcurrentLinkedDeque;</span>
<span class="source-line-no">046</span><span id="line-46">import java.util.concurrent.TimeUnit;</span>
<span class="source-line-no">047</span><span id="line-47">import java.util.function.Supplier;</span>
<span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.crypto.Encryptor;</span>
<span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.fs.Path;</span>
<span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.CancelOnClose;</span>
<span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor;</span>
<span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.util.CancelableProgressable;</span>
<span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span>
<span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.util.FutureUtils;</span>
<span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.hbase.util.NettyFutureUtils;</span>
<span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.hbase.util.RecoverLeaseFSUtils;</span>
<span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.hdfs.DFSClient;</span>
<span class="source-line-no">059</span><span id="line-59">import org.apache.hadoop.hdfs.DistributedFileSystem;</span>
<span class="source-line-no">060</span><span id="line-60">import org.apache.hadoop.hdfs.protocol.ClientProtocol;</span>
<span class="source-line-no">061</span><span id="line-61">import org.apache.hadoop.hdfs.protocol.DatanodeInfo;</span>
<span class="source-line-no">062</span><span id="line-62">import org.apache.hadoop.hdfs.protocol.ExtendedBlock;</span>
<span class="source-line-no">063</span><span id="line-63">import org.apache.hadoop.hdfs.protocol.LocatedBlock;</span>
<span class="source-line-no">064</span><span id="line-64">import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;</span>
<span class="source-line-no">065</span><span id="line-65">import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;</span>
<span class="source-line-no">066</span><span id="line-66">import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;</span>
<span class="source-line-no">067</span><span id="line-67">import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;</span>
<span class="source-line-no">068</span><span id="line-68">import org.apache.hadoop.util.DataChecksum;</span>
<span class="source-line-no">069</span><span id="line-69">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">070</span><span id="line-70"></span>
<span class="source-line-no">071</span><span id="line-71">import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;</span>
<span class="source-line-no">072</span><span id="line-72">import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;</span>
<span class="source-line-no">073</span><span id="line-73">import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufAllocator;</span>
<span class="source-line-no">074</span><span id="line-74">import org.apache.hbase.thirdparty.io.netty.channel.Channel;</span>
<span class="source-line-no">075</span><span id="line-75">import org.apache.hbase.thirdparty.io.netty.channel.ChannelFuture;</span>
<span class="source-line-no">076</span><span id="line-76">import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandler.Sharable;</span>
<span class="source-line-no">077</span><span id="line-77">import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;</span>
<span class="source-line-no">078</span><span id="line-78">import org.apache.hbase.thirdparty.io.netty.channel.ChannelId;</span>
<span class="source-line-no">079</span><span id="line-79">import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler;</span>
<span class="source-line-no">080</span><span id="line-80">import org.apache.hbase.thirdparty.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;</span>
<span class="source-line-no">081</span><span id="line-81">import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateEvent;</span>
<span class="source-line-no">082</span><span id="line-82">import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler;</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"> * An asynchronous HDFS output stream implementation which fans out data to datanode and only</span>
<span class="source-line-no">086</span><span id="line-86"> * supports writing file with only one block.</span>
<span class="source-line-no">087</span><span id="line-87"> * &lt;p&gt;</span>
<span class="source-line-no">088</span><span id="line-88"> * Use the createOutput method in {@link FanOutOneBlockAsyncDFSOutputHelper} to create. The main</span>
<span class="source-line-no">089</span><span id="line-89"> * usage of this class is implementing WAL, so we only expose a little HDFS configurations in the</span>
<span class="source-line-no">090</span><span id="line-90"> * method. And we place it here under io package because we want to make it independent of WAL</span>
<span class="source-line-no">091</span><span id="line-91"> * implementation thus easier to move it to HDFS project finally.</span>
<span class="source-line-no">092</span><span id="line-92"> * &lt;p&gt;</span>
<span class="source-line-no">093</span><span id="line-93"> * Note that, although we support pipelined flush, i.e, write new data and then flush before the</span>
<span class="source-line-no">094</span><span id="line-94"> * previous flush succeeds, the implementation is not thread safe, so you should not call its</span>
<span class="source-line-no">095</span><span id="line-95"> * methods concurrently.</span>
<span class="source-line-no">096</span><span id="line-96"> * &lt;p&gt;</span>
<span class="source-line-no">097</span><span id="line-97"> * Advantages compare to DFSOutputStream:</span>
<span class="source-line-no">098</span><span id="line-98"> * &lt;ol&gt;</span>
<span class="source-line-no">099</span><span id="line-99"> * &lt;li&gt;The fan out mechanism. This will reduce the latency.&lt;/li&gt;</span>
<span class="source-line-no">100</span><span id="line-100"> * &lt;li&gt;Fail-fast when connection to datanode error. The WAL implementation could open new writer</span>
<span class="source-line-no">101</span><span id="line-101"> * ASAP.&lt;/li&gt;</span>
<span class="source-line-no">102</span><span id="line-102"> * &lt;li&gt;We could benefit from netty's ByteBuf management mechanism.&lt;/li&gt;</span>
<span class="source-line-no">103</span><span id="line-103"> * &lt;/ol&gt;</span>
<span class="source-line-no">104</span><span id="line-104"> */</span>
<span class="source-line-no">105</span><span id="line-105">@InterfaceAudience.Private</span>
<span class="source-line-no">106</span><span id="line-106">public class FanOutOneBlockAsyncDFSOutput implements AsyncFSOutput {</span>
<span class="source-line-no">107</span><span id="line-107"></span>
<span class="source-line-no">108</span><span id="line-108"> // The MAX_PACKET_SIZE is 16MB, but it includes the header size and checksum size. So here we set</span>
<span class="source-line-no">109</span><span id="line-109"> // a smaller limit for data size.</span>
<span class="source-line-no">110</span><span id="line-110"> private static final int MAX_DATA_LEN = 12 * 1024 * 1024;</span>
<span class="source-line-no">111</span><span id="line-111"></span>
<span class="source-line-no">112</span><span id="line-112"> private final Configuration conf;</span>
<span class="source-line-no">113</span><span id="line-113"></span>
<span class="source-line-no">114</span><span id="line-114"> private final DistributedFileSystem dfs;</span>
<span class="source-line-no">115</span><span id="line-115"></span>
<span class="source-line-no">116</span><span id="line-116"> private final DFSClient client;</span>
<span class="source-line-no">117</span><span id="line-117"></span>
<span class="source-line-no">118</span><span id="line-118"> private final ClientProtocol namenode;</span>
<span class="source-line-no">119</span><span id="line-119"></span>
<span class="source-line-no">120</span><span id="line-120"> private final String clientName;</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 final String src;</span>
<span class="source-line-no">123</span><span id="line-123"></span>
<span class="source-line-no">124</span><span id="line-124"> private final long fileId;</span>
<span class="source-line-no">125</span><span id="line-125"></span>
<span class="source-line-no">126</span><span id="line-126"> private final ExtendedBlock block;</span>
<span class="source-line-no">127</span><span id="line-127"></span>
<span class="source-line-no">128</span><span id="line-128"> private final DatanodeInfo[] locations;</span>
<span class="source-line-no">129</span><span id="line-129"></span>
<span class="source-line-no">130</span><span id="line-130"> private final Encryptor encryptor;</span>
<span class="source-line-no">131</span><span id="line-131"></span>
<span class="source-line-no">132</span><span id="line-132"> private final Map&lt;Channel, DatanodeInfo&gt; datanodeInfoMap;</span>
<span class="source-line-no">133</span><span id="line-133"></span>
<span class="source-line-no">134</span><span id="line-134"> private final DataChecksum summer;</span>
<span class="source-line-no">135</span><span id="line-135"></span>
<span class="source-line-no">136</span><span id="line-136"> private final int maxDataLen;</span>
<span class="source-line-no">137</span><span id="line-137"></span>
<span class="source-line-no">138</span><span id="line-138"> private final ByteBufAllocator alloc;</span>
<span class="source-line-no">139</span><span id="line-139"></span>
<span class="source-line-no">140</span><span id="line-140"> private static final class Callback {</span>
<span class="source-line-no">141</span><span id="line-141"></span>
<span class="source-line-no">142</span><span id="line-142"> private final CompletableFuture&lt;Long&gt; future;</span>
<span class="source-line-no">143</span><span id="line-143"></span>
<span class="source-line-no">144</span><span id="line-144"> private final long ackedLength;</span>
<span class="source-line-no">145</span><span id="line-145"></span>
<span class="source-line-no">146</span><span id="line-146"> // should be backed by a thread safe collection</span>
<span class="source-line-no">147</span><span id="line-147"> private final Set&lt;ChannelId&gt; unfinishedReplicas;</span>
<span class="source-line-no">148</span><span id="line-148"> private final long packetDataLen;</span>
<span class="source-line-no">149</span><span id="line-149"> private final long flushTimestamp;</span>
<span class="source-line-no">150</span><span id="line-150"> private long lastAckTimestamp = -1;</span>
<span class="source-line-no">151</span><span id="line-151"></span>
<span class="source-line-no">152</span><span id="line-152"> public Callback(CompletableFuture&lt;Long&gt; future, long ackedLength,</span>
<span class="source-line-no">153</span><span id="line-153"> final Collection&lt;Channel&gt; replicas, long packetDataLen) {</span>
<span class="source-line-no">154</span><span id="line-154"> this.future = future;</span>
<span class="source-line-no">155</span><span id="line-155"> this.ackedLength = ackedLength;</span>
<span class="source-line-no">156</span><span id="line-156"> this.packetDataLen = packetDataLen;</span>
<span class="source-line-no">157</span><span id="line-157"> this.flushTimestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">158</span><span id="line-158"> if (replicas.isEmpty()) {</span>
<span class="source-line-no">159</span><span id="line-159"> this.unfinishedReplicas = Collections.emptySet();</span>
<span class="source-line-no">160</span><span id="line-160"> } else {</span>
<span class="source-line-no">161</span><span id="line-161"> this.unfinishedReplicas =</span>
<span class="source-line-no">162</span><span id="line-162"> Collections.newSetFromMap(new ConcurrentHashMap&lt;ChannelId, Boolean&gt;(replicas.size()));</span>
<span class="source-line-no">163</span><span id="line-163"> replicas.stream().map(Channel::id).forEachOrdered(unfinishedReplicas::add);</span>
<span class="source-line-no">164</span><span id="line-164"> }</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"></span>
<span class="source-line-no">168</span><span id="line-168"> private final ConcurrentLinkedDeque&lt;Callback&gt; waitingAckQueue = new ConcurrentLinkedDeque&lt;&gt;();</span>
<span class="source-line-no">169</span><span id="line-169"></span>
<span class="source-line-no">170</span><span id="line-170"> private volatile long ackedBlockLength = 0L;</span>
<span class="source-line-no">171</span><span id="line-171"></span>
<span class="source-line-no">172</span><span id="line-172"> // this could be different from acked block length because a packet can not start at the middle of</span>
<span class="source-line-no">173</span><span id="line-173"> // a chunk.</span>
<span class="source-line-no">174</span><span id="line-174"> private long nextPacketOffsetInBlock = 0L;</span>
<span class="source-line-no">175</span><span id="line-175"></span>
<span class="source-line-no">176</span><span id="line-176"> // the length of the trailing partial chunk, this is because the packet start offset must be</span>
<span class="source-line-no">177</span><span id="line-177"> // aligned with the length of checksum chunk, so we need to resend the same data.</span>
<span class="source-line-no">178</span><span id="line-178"> private int trailingPartialChunkLength = 0;</span>
<span class="source-line-no">179</span><span id="line-179"></span>
<span class="source-line-no">180</span><span id="line-180"> private long nextPacketSeqno = 0L;</span>
<span class="source-line-no">181</span><span id="line-181"></span>
<span class="source-line-no">182</span><span id="line-182"> private ByteBuf buf;</span>
<span class="source-line-no">183</span><span id="line-183"></span>
<span class="source-line-no">184</span><span id="line-184"> private final SendBufSizePredictor sendBufSizePRedictor = new SendBufSizePredictor();</span>
<span class="source-line-no">185</span><span id="line-185"></span>
<span class="source-line-no">186</span><span id="line-186"> // State for connections to DN</span>
<span class="source-line-no">187</span><span id="line-187"> private enum State {</span>
<span class="source-line-no">188</span><span id="line-188"> STREAMING,</span>
<span class="source-line-no">189</span><span id="line-189"> CLOSING,</span>
<span class="source-line-no">190</span><span id="line-190"> BROKEN,</span>
<span class="source-line-no">191</span><span id="line-191"> CLOSED</span>
<span class="source-line-no">192</span><span id="line-192"> }</span>
<span class="source-line-no">193</span><span id="line-193"></span>
<span class="source-line-no">194</span><span id="line-194"> private volatile State state;</span>
<span class="source-line-no">195</span><span id="line-195"></span>
<span class="source-line-no">196</span><span id="line-196"> private final StreamSlowMonitor streamSlowMonitor;</span>
<span class="source-line-no">197</span><span id="line-197"></span>
<span class="source-line-no">198</span><span id="line-198"> // all lock-free to make it run faster</span>
<span class="source-line-no">199</span><span id="line-199"> private void completed(Channel channel) {</span>
<span class="source-line-no">200</span><span id="line-200"> for (Iterator&lt;Callback&gt; iter = waitingAckQueue.iterator(); iter.hasNext();) {</span>
<span class="source-line-no">201</span><span id="line-201"> Callback c = iter.next();</span>
<span class="source-line-no">202</span><span id="line-202"> // if the current unfinished replicas does not contain us then it means that we have already</span>
<span class="source-line-no">203</span><span id="line-203"> // acked this one, let's iterate to find the one we have not acked yet.</span>
<span class="source-line-no">204</span><span id="line-204"> if (c.unfinishedReplicas.remove(channel.id())) {</span>
<span class="source-line-no">205</span><span id="line-205"> long current = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">206</span><span id="line-206"> streamSlowMonitor.checkProcessTimeAndSpeed(datanodeInfoMap.get(channel), c.packetDataLen,</span>
<span class="source-line-no">207</span><span id="line-207"> current - c.flushTimestamp, c.lastAckTimestamp, c.unfinishedReplicas.size());</span>
<span class="source-line-no">208</span><span id="line-208"> c.lastAckTimestamp = current;</span>
<span class="source-line-no">209</span><span id="line-209"> if (c.unfinishedReplicas.isEmpty()) {</span>
<span class="source-line-no">210</span><span id="line-210"> // we need to remove first before complete the future. It is possible that after we</span>
<span class="source-line-no">211</span><span id="line-211"> // complete the future the upper layer will call close immediately before we remove the</span>
<span class="source-line-no">212</span><span id="line-212"> // entry from waitingAckQueue and lead to an IllegalStateException. And also set the</span>
<span class="source-line-no">213</span><span id="line-213"> // ackedBlockLength first otherwise we may use a wrong length to commit the block. This</span>
<span class="source-line-no">214</span><span id="line-214"> // may lead to multiple remove and assign but is OK. The semantic of iter.remove is</span>
<span class="source-line-no">215</span><span id="line-215"> // removing the entry returned by calling previous next, so if the entry has already been</span>
<span class="source-line-no">216</span><span id="line-216"> // removed then it is a no-op, and for the assign, the values are the same so no problem.</span>
<span class="source-line-no">217</span><span id="line-217"> iter.remove();</span>
<span class="source-line-no">218</span><span id="line-218"> ackedBlockLength = c.ackedLength;</span>
<span class="source-line-no">219</span><span id="line-219"> // the future.complete check is to confirm that we are the only one who grabbed the work,</span>
<span class="source-line-no">220</span><span id="line-220"> // otherwise just give up and return.</span>
<span class="source-line-no">221</span><span id="line-221"> if (c.future.complete(c.ackedLength)) {</span>
<span class="source-line-no">222</span><span id="line-222"> // also wake up flush requests which have the same length.</span>
<span class="source-line-no">223</span><span id="line-223"> while (iter.hasNext()) {</span>
<span class="source-line-no">224</span><span id="line-224"> Callback maybeDummyCb = iter.next();</span>
<span class="source-line-no">225</span><span id="line-225"> if (maybeDummyCb.ackedLength == c.ackedLength) {</span>
<span class="source-line-no">226</span><span id="line-226"> iter.remove();</span>
<span class="source-line-no">227</span><span id="line-227"> maybeDummyCb.future.complete(c.ackedLength);</span>
<span class="source-line-no">228</span><span id="line-228"> } else {</span>
<span class="source-line-no">229</span><span id="line-229"> break;</span>
<span class="source-line-no">230</span><span id="line-230"> }</span>
<span class="source-line-no">231</span><span id="line-231"> }</span>
<span class="source-line-no">232</span><span id="line-232"> }</span>
<span class="source-line-no">233</span><span id="line-233"> }</span>
<span class="source-line-no">234</span><span id="line-234"> return;</span>
<span class="source-line-no">235</span><span id="line-235"> }</span>
<span class="source-line-no">236</span><span id="line-236"> }</span>
<span class="source-line-no">237</span><span id="line-237"> }</span>
<span class="source-line-no">238</span><span id="line-238"></span>
<span class="source-line-no">239</span><span id="line-239"> // this usually does not happen which means it is not on the critical path so make it synchronized</span>
<span class="source-line-no">240</span><span id="line-240"> // so that the implementation will not burn up our brain as there are multiple state changes and</span>
<span class="source-line-no">241</span><span id="line-241"> // checks.</span>
<span class="source-line-no">242</span><span id="line-242"> private synchronized void failed(Channel channel, Supplier&lt;Throwable&gt; errorSupplier) {</span>
<span class="source-line-no">243</span><span id="line-243"> if (state == State.CLOSED) {</span>
<span class="source-line-no">244</span><span id="line-244"> return;</span>
<span class="source-line-no">245</span><span id="line-245"> }</span>
<span class="source-line-no">246</span><span id="line-246"> if (state == State.BROKEN) {</span>
<span class="source-line-no">247</span><span id="line-247"> failWaitingAckQueue(channel, errorSupplier);</span>
<span class="source-line-no">248</span><span id="line-248"> return;</span>
<span class="source-line-no">249</span><span id="line-249"> }</span>
<span class="source-line-no">250</span><span id="line-250"> if (state == State.CLOSING) {</span>
<span class="source-line-no">251</span><span id="line-251"> Callback c = waitingAckQueue.peekFirst();</span>
<span class="source-line-no">252</span><span id="line-252"> if (c == null || !c.unfinishedReplicas.contains(channel.id())) {</span>
<span class="source-line-no">253</span><span id="line-253"> // nothing, the endBlock request has already finished.</span>
<span class="source-line-no">254</span><span id="line-254"> return;</span>
<span class="source-line-no">255</span><span id="line-255"> }</span>
<span class="source-line-no">256</span><span id="line-256"> }</span>
<span class="source-line-no">257</span><span id="line-257"> // disable further write, and fail all pending ack.</span>
<span class="source-line-no">258</span><span id="line-258"> state = State.BROKEN;</span>
<span class="source-line-no">259</span><span id="line-259"> failWaitingAckQueue(channel, errorSupplier);</span>
<span class="source-line-no">260</span><span id="line-260"> datanodeInfoMap.keySet().forEach(NettyFutureUtils::safeClose);</span>
<span class="source-line-no">261</span><span id="line-261"> }</span>
<span class="source-line-no">262</span><span id="line-262"></span>
<span class="source-line-no">263</span><span id="line-263"> private void failWaitingAckQueue(Channel channel, Supplier&lt;Throwable&gt; errorSupplier) {</span>
<span class="source-line-no">264</span><span id="line-264"> Throwable error = errorSupplier.get();</span>
<span class="source-line-no">265</span><span id="line-265"> for (Iterator&lt;Callback&gt; iter = waitingAckQueue.iterator(); iter.hasNext();) {</span>
<span class="source-line-no">266</span><span id="line-266"> Callback c = iter.next();</span>
<span class="source-line-no">267</span><span id="line-267"> // find the first sync request which we have not acked yet and fail all the request after it.</span>
<span class="source-line-no">268</span><span id="line-268"> if (!c.unfinishedReplicas.contains(channel.id())) {</span>
<span class="source-line-no">269</span><span id="line-269"> continue;</span>
<span class="source-line-no">270</span><span id="line-270"> }</span>
<span class="source-line-no">271</span><span id="line-271"> for (;;) {</span>
<span class="source-line-no">272</span><span id="line-272"> c.future.completeExceptionally(error);</span>
<span class="source-line-no">273</span><span id="line-273"> if (!iter.hasNext()) {</span>
<span class="source-line-no">274</span><span id="line-274"> break;</span>
<span class="source-line-no">275</span><span id="line-275"> }</span>
<span class="source-line-no">276</span><span id="line-276"> c = iter.next();</span>
<span class="source-line-no">277</span><span id="line-277"> }</span>
<span class="source-line-no">278</span><span id="line-278"> break;</span>
<span class="source-line-no">279</span><span id="line-279"> }</span>
<span class="source-line-no">280</span><span id="line-280"> }</span>
<span class="source-line-no">281</span><span id="line-281"></span>
<span class="source-line-no">282</span><span id="line-282"> @Sharable</span>
<span class="source-line-no">283</span><span id="line-283"> private final class AckHandler extends SimpleChannelInboundHandler&lt;PipelineAckProto&gt; {</span>
<span class="source-line-no">284</span><span id="line-284"></span>
<span class="source-line-no">285</span><span id="line-285"> private final int timeoutMs;</span>
<span class="source-line-no">286</span><span id="line-286"></span>
<span class="source-line-no">287</span><span id="line-287"> public AckHandler(int timeoutMs) {</span>
<span class="source-line-no">288</span><span id="line-288"> this.timeoutMs = timeoutMs;</span>
<span class="source-line-no">289</span><span id="line-289"> }</span>
<span class="source-line-no">290</span><span id="line-290"></span>
<span class="source-line-no">291</span><span id="line-291"> @Override</span>
<span class="source-line-no">292</span><span id="line-292"> protected void channelRead0(ChannelHandlerContext ctx, PipelineAckProto ack) throws Exception {</span>
<span class="source-line-no">293</span><span id="line-293"> Status reply = getStatus(ack);</span>
<span class="source-line-no">294</span><span id="line-294"> if (reply != Status.SUCCESS) {</span>
<span class="source-line-no">295</span><span id="line-295"> failed(ctx.channel(), () -&gt; new IOException("Bad response " + reply + " for block " + block</span>
<span class="source-line-no">296</span><span id="line-296"> + " from datanode " + ctx.channel().remoteAddress()));</span>
<span class="source-line-no">297</span><span id="line-297"> return;</span>
<span class="source-line-no">298</span><span id="line-298"> }</span>
<span class="source-line-no">299</span><span id="line-299"> if (PipelineAck.isRestartOOBStatus(reply)) {</span>
<span class="source-line-no">300</span><span id="line-300"> failed(ctx.channel(), () -&gt; new IOException("Restart response " + reply + " for block "</span>
<span class="source-line-no">301</span><span id="line-301"> + block + " from datanode " + ctx.channel().remoteAddress()));</span>
<span class="source-line-no">302</span><span id="line-302"> return;</span>
<span class="source-line-no">303</span><span id="line-303"> }</span>
<span class="source-line-no">304</span><span id="line-304"> if (ack.getSeqno() == HEART_BEAT_SEQNO) {</span>
<span class="source-line-no">305</span><span id="line-305"> return;</span>
<span class="source-line-no">306</span><span id="line-306"> }</span>
<span class="source-line-no">307</span><span id="line-307"> completed(ctx.channel());</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"> @Override</span>
<span class="source-line-no">311</span><span id="line-311"> public void channelInactive(ChannelHandlerContext ctx) throws Exception {</span>
<span class="source-line-no">312</span><span id="line-312"> if (state == State.CLOSED) {</span>
<span class="source-line-no">313</span><span id="line-313"> return;</span>
<span class="source-line-no">314</span><span id="line-314"> }</span>
<span class="source-line-no">315</span><span id="line-315"> failed(ctx.channel(),</span>
<span class="source-line-no">316</span><span id="line-316"> () -&gt; new IOException("Connection to " + ctx.channel().remoteAddress() + " closed"));</span>
<span class="source-line-no">317</span><span id="line-317"> }</span>
<span class="source-line-no">318</span><span id="line-318"></span>
<span class="source-line-no">319</span><span id="line-319"> @Override</span>
<span class="source-line-no">320</span><span id="line-320"> public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {</span>
<span class="source-line-no">321</span><span id="line-321"> failed(ctx.channel(), () -&gt; cause);</span>
<span class="source-line-no">322</span><span id="line-322"> }</span>
<span class="source-line-no">323</span><span id="line-323"></span>
<span class="source-line-no">324</span><span id="line-324"> @Override</span>
<span class="source-line-no">325</span><span id="line-325"> public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {</span>
<span class="source-line-no">326</span><span id="line-326"> if (evt instanceof IdleStateEvent) {</span>
<span class="source-line-no">327</span><span id="line-327"> IdleStateEvent e = (IdleStateEvent) evt;</span>
<span class="source-line-no">328</span><span id="line-328"> if (e.state() == READER_IDLE) {</span>
<span class="source-line-no">329</span><span id="line-329"> failed(ctx.channel(),</span>
<span class="source-line-no">330</span><span id="line-330"> () -&gt; new IOException("Timeout(" + timeoutMs + "ms) waiting for response"));</span>
<span class="source-line-no">331</span><span id="line-331"> } else if (e.state() == WRITER_IDLE) {</span>
<span class="source-line-no">332</span><span id="line-332"> PacketHeader heartbeat = new PacketHeader(4, 0, HEART_BEAT_SEQNO, false, 0, false);</span>
<span class="source-line-no">333</span><span id="line-333"> int len = heartbeat.getSerializedSize();</span>
<span class="source-line-no">334</span><span id="line-334"> ByteBuf buf = alloc.buffer(len);</span>
<span class="source-line-no">335</span><span id="line-335"> heartbeat.putInBuffer(buf.nioBuffer(0, len));</span>
<span class="source-line-no">336</span><span id="line-336"> buf.writerIndex(len);</span>
<span class="source-line-no">337</span><span id="line-337"> safeWriteAndFlush(ctx.channel(), buf);</span>
<span class="source-line-no">338</span><span id="line-338"> }</span>
<span class="source-line-no">339</span><span id="line-339"> return;</span>
<span class="source-line-no">340</span><span id="line-340"> }</span>
<span class="source-line-no">341</span><span id="line-341"> super.userEventTriggered(ctx, evt);</span>
<span class="source-line-no">342</span><span id="line-342"> }</span>
<span class="source-line-no">343</span><span id="line-343"> }</span>
<span class="source-line-no">344</span><span id="line-344"></span>
<span class="source-line-no">345</span><span id="line-345"> private void setupReceiver(int timeoutMs) {</span>
<span class="source-line-no">346</span><span id="line-346"> AckHandler ackHandler = new AckHandler(timeoutMs);</span>
<span class="source-line-no">347</span><span id="line-347"> for (Channel ch : datanodeInfoMap.keySet()) {</span>
<span class="source-line-no">348</span><span id="line-348"> ch.pipeline().addLast(</span>
<span class="source-line-no">349</span><span id="line-349"> new IdleStateHandler(timeoutMs, timeoutMs / 2, 0, TimeUnit.MILLISECONDS),</span>
<span class="source-line-no">350</span><span id="line-350"> new ProtobufVarint32FrameDecoder(),</span>
<span class="source-line-no">351</span><span id="line-351"> new ProtobufDecoder(PipelineAckProto.getDefaultInstance()), ackHandler);</span>
<span class="source-line-no">352</span><span id="line-352"> ch.config().setAutoRead(true);</span>
<span class="source-line-no">353</span><span id="line-353"> }</span>
<span class="source-line-no">354</span><span id="line-354"> }</span>
<span class="source-line-no">355</span><span id="line-355"></span>
<span class="source-line-no">356</span><span id="line-356"> FanOutOneBlockAsyncDFSOutput(Configuration conf, DistributedFileSystem dfs, DFSClient client,</span>
<span class="source-line-no">357</span><span id="line-357"> ClientProtocol namenode, String clientName, String src, long fileId, LocatedBlock locatedBlock,</span>
<span class="source-line-no">358</span><span id="line-358"> Encryptor encryptor, Map&lt;Channel, DatanodeInfo&gt; datanodeInfoMap, DataChecksum summer,</span>
<span class="source-line-no">359</span><span id="line-359"> ByteBufAllocator alloc, StreamSlowMonitor streamSlowMonitor) {</span>
<span class="source-line-no">360</span><span id="line-360"> this.conf = conf;</span>
<span class="source-line-no">361</span><span id="line-361"> this.dfs = dfs;</span>
<span class="source-line-no">362</span><span id="line-362"> this.client = client;</span>
<span class="source-line-no">363</span><span id="line-363"> this.namenode = namenode;</span>
<span class="source-line-no">364</span><span id="line-364"> this.fileId = fileId;</span>
<span class="source-line-no">365</span><span id="line-365"> this.clientName = clientName;</span>
<span class="source-line-no">366</span><span id="line-366"> this.src = src;</span>
<span class="source-line-no">367</span><span id="line-367"> this.block = locatedBlock.getBlock();</span>
<span class="source-line-no">368</span><span id="line-368"> this.locations = getLocatedBlockLocations(locatedBlock);</span>
<span class="source-line-no">369</span><span id="line-369"> this.encryptor = encryptor;</span>
<span class="source-line-no">370</span><span id="line-370"> this.datanodeInfoMap = datanodeInfoMap;</span>
<span class="source-line-no">371</span><span id="line-371"> this.summer = summer;</span>
<span class="source-line-no">372</span><span id="line-372"> this.maxDataLen = MAX_DATA_LEN - (MAX_DATA_LEN % summer.getBytesPerChecksum());</span>
<span class="source-line-no">373</span><span id="line-373"> this.alloc = alloc;</span>
<span class="source-line-no">374</span><span id="line-374"> this.buf = alloc.directBuffer(sendBufSizePRedictor.initialSize());</span>
<span class="source-line-no">375</span><span id="line-375"> this.state = State.STREAMING;</span>
<span class="source-line-no">376</span><span id="line-376"> setupReceiver(conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT));</span>
<span class="source-line-no">377</span><span id="line-377"> this.streamSlowMonitor = streamSlowMonitor;</span>
<span class="source-line-no">378</span><span id="line-378"> }</span>
<span class="source-line-no">379</span><span id="line-379"></span>
<span class="source-line-no">380</span><span id="line-380"> @Override</span>
<span class="source-line-no">381</span><span id="line-381"> public void writeInt(int i) {</span>
<span class="source-line-no">382</span><span id="line-382"> buf.ensureWritable(4);</span>
<span class="source-line-no">383</span><span id="line-383"> buf.writeInt(i);</span>
<span class="source-line-no">384</span><span id="line-384"> }</span>
<span class="source-line-no">385</span><span id="line-385"></span>
<span class="source-line-no">386</span><span id="line-386"> @Override</span>
<span class="source-line-no">387</span><span id="line-387"> public void write(ByteBuffer bb) {</span>
<span class="source-line-no">388</span><span id="line-388"> buf.ensureWritable(bb.remaining());</span>
<span class="source-line-no">389</span><span id="line-389"> buf.writeBytes(bb);</span>
<span class="source-line-no">390</span><span id="line-390"> }</span>
<span class="source-line-no">391</span><span id="line-391"></span>
<span class="source-line-no">392</span><span id="line-392"> @Override</span>
<span class="source-line-no">393</span><span id="line-393"> public void write(byte[] b) {</span>
<span class="source-line-no">394</span><span id="line-394"> write(b, 0, b.length);</span>
<span class="source-line-no">395</span><span id="line-395"> }</span>
<span class="source-line-no">396</span><span id="line-396"></span>
<span class="source-line-no">397</span><span id="line-397"> @Override</span>
<span class="source-line-no">398</span><span id="line-398"> public void write(byte[] b, int off, int len) {</span>
<span class="source-line-no">399</span><span id="line-399"> buf.ensureWritable(len);</span>
<span class="source-line-no">400</span><span id="line-400"> buf.writeBytes(b, off, len);</span>
<span class="source-line-no">401</span><span id="line-401"> }</span>
<span class="source-line-no">402</span><span id="line-402"></span>
<span class="source-line-no">403</span><span id="line-403"> @Override</span>
<span class="source-line-no">404</span><span id="line-404"> public int buffered() {</span>
<span class="source-line-no">405</span><span id="line-405"> return buf.readableBytes();</span>
<span class="source-line-no">406</span><span id="line-406"> }</span>
<span class="source-line-no">407</span><span id="line-407"></span>
<span class="source-line-no">408</span><span id="line-408"> @Override</span>
<span class="source-line-no">409</span><span id="line-409"> public DatanodeInfo[] getPipeline() {</span>
<span class="source-line-no">410</span><span id="line-410"> return locations;</span>
<span class="source-line-no">411</span><span id="line-411"> }</span>
<span class="source-line-no">412</span><span id="line-412"></span>
<span class="source-line-no">413</span><span id="line-413"> private void flushBuffer(CompletableFuture&lt;Long&gt; future, ByteBuf dataBuf,</span>
<span class="source-line-no">414</span><span id="line-414"> long nextPacketOffsetInBlock, boolean syncBlock) {</span>
<span class="source-line-no">415</span><span id="line-415"> int dataLen = dataBuf.readableBytes();</span>
<span class="source-line-no">416</span><span id="line-416"> int chunkLen = summer.getBytesPerChecksum();</span>
<span class="source-line-no">417</span><span id="line-417"> int trailingPartialChunkLen = dataLen % chunkLen;</span>
<span class="source-line-no">418</span><span id="line-418"> int numChecks = dataLen / chunkLen + (trailingPartialChunkLen != 0 ? 1 : 0);</span>
<span class="source-line-no">419</span><span id="line-419"> int checksumLen = numChecks * summer.getChecksumSize();</span>
<span class="source-line-no">420</span><span id="line-420"> ByteBuf checksumBuf = alloc.directBuffer(checksumLen);</span>
<span class="source-line-no">421</span><span id="line-421"> summer.calculateChunkedSums(dataBuf.nioBuffer(), checksumBuf.nioBuffer(0, checksumLen));</span>
<span class="source-line-no">422</span><span id="line-422"> checksumBuf.writerIndex(checksumLen);</span>
<span class="source-line-no">423</span><span id="line-423"> PacketHeader header = new PacketHeader(4 + checksumLen + dataLen, nextPacketOffsetInBlock,</span>
<span class="source-line-no">424</span><span id="line-424"> nextPacketSeqno, false, dataLen, syncBlock);</span>
<span class="source-line-no">425</span><span id="line-425"> int headerLen = header.getSerializedSize();</span>
<span class="source-line-no">426</span><span id="line-426"> ByteBuf headerBuf = alloc.buffer(headerLen);</span>
<span class="source-line-no">427</span><span id="line-427"> header.putInBuffer(headerBuf.nioBuffer(0, headerLen));</span>
<span class="source-line-no">428</span><span id="line-428"> headerBuf.writerIndex(headerLen);</span>
<span class="source-line-no">429</span><span id="line-429"> Callback c =</span>
<span class="source-line-no">430</span><span id="line-430"> new Callback(future, nextPacketOffsetInBlock + dataLen, datanodeInfoMap.keySet(), dataLen);</span>
<span class="source-line-no">431</span><span id="line-431"> waitingAckQueue.addLast(c);</span>
<span class="source-line-no">432</span><span id="line-432"> // recheck again after we pushed the callback to queue</span>
<span class="source-line-no">433</span><span id="line-433"> if (state != State.STREAMING &amp;&amp; waitingAckQueue.peekFirst() == c) {</span>
<span class="source-line-no">434</span><span id="line-434"> future.completeExceptionally(new IOException("stream already broken"));</span>
<span class="source-line-no">435</span><span id="line-435"> // it's the one we have just pushed or just a no-op</span>
<span class="source-line-no">436</span><span id="line-436"> waitingAckQueue.removeFirst();</span>
<span class="source-line-no">437</span><span id="line-437"></span>
<span class="source-line-no">438</span><span id="line-438"> checksumBuf.release();</span>
<span class="source-line-no">439</span><span id="line-439"> headerBuf.release();</span>
<span class="source-line-no">440</span><span id="line-440"></span>
<span class="source-line-no">441</span><span id="line-441"> // This method takes ownership of the dataBuf, so we need release it before returning.</span>
<span class="source-line-no">442</span><span id="line-442"> dataBuf.release();</span>
<span class="source-line-no">443</span><span id="line-443"> return;</span>
<span class="source-line-no">444</span><span id="line-444"> }</span>
<span class="source-line-no">445</span><span id="line-445"> // TODO: we should perhaps measure time taken per DN here;</span>
<span class="source-line-no">446</span><span id="line-446"> // we could collect statistics per DN, and/or exclude bad nodes in createOutput.</span>
<span class="source-line-no">447</span><span id="line-447"> datanodeInfoMap.keySet().forEach(ch -&gt; {</span>
<span class="source-line-no">448</span><span id="line-448"> safeWrite(ch, headerBuf.retainedDuplicate());</span>
<span class="source-line-no">449</span><span id="line-449"> safeWrite(ch, checksumBuf.retainedDuplicate());</span>
<span class="source-line-no">450</span><span id="line-450"> safeWriteAndFlush(ch, dataBuf.retainedDuplicate());</span>
<span class="source-line-no">451</span><span id="line-451"> });</span>
<span class="source-line-no">452</span><span id="line-452"> checksumBuf.release();</span>
<span class="source-line-no">453</span><span id="line-453"> headerBuf.release();</span>
<span class="source-line-no">454</span><span id="line-454"> dataBuf.release();</span>
<span class="source-line-no">455</span><span id="line-455"> nextPacketSeqno++;</span>
<span class="source-line-no">456</span><span id="line-456"> }</span>
<span class="source-line-no">457</span><span id="line-457"></span>
<span class="source-line-no">458</span><span id="line-458"> private void flush0(CompletableFuture&lt;Long&gt; future, boolean syncBlock) {</span>
<span class="source-line-no">459</span><span id="line-459"> if (state != State.STREAMING) {</span>
<span class="source-line-no">460</span><span id="line-460"> future.completeExceptionally(new IOException("stream already broken"));</span>
<span class="source-line-no">461</span><span id="line-461"> return;</span>
<span class="source-line-no">462</span><span id="line-462"> }</span>
<span class="source-line-no">463</span><span id="line-463"> int dataLen = buf.readableBytes();</span>
<span class="source-line-no">464</span><span id="line-464"> if (dataLen == trailingPartialChunkLength) {</span>
<span class="source-line-no">465</span><span id="line-465"> // no new data</span>
<span class="source-line-no">466</span><span id="line-466"> long lengthAfterFlush = nextPacketOffsetInBlock + dataLen;</span>
<span class="source-line-no">467</span><span id="line-467"> Callback lastFlush = waitingAckQueue.peekLast();</span>
<span class="source-line-no">468</span><span id="line-468"> if (lastFlush != null) {</span>
<span class="source-line-no">469</span><span id="line-469"> Callback c = new Callback(future, lengthAfterFlush, Collections.emptySet(), dataLen);</span>
<span class="source-line-no">470</span><span id="line-470"> waitingAckQueue.addLast(c);</span>
<span class="source-line-no">471</span><span id="line-471"> // recheck here if we have already removed the previous callback from the queue</span>
<span class="source-line-no">472</span><span id="line-472"> if (waitingAckQueue.peekFirst() == c) {</span>
<span class="source-line-no">473</span><span id="line-473"> // all previous callbacks have been removed</span>
<span class="source-line-no">474</span><span id="line-474"> // notice that this does mean we will always win here because the background thread may</span>
<span class="source-line-no">475</span><span id="line-475"> // have already started to mark the future here as completed in the completed or failed</span>
<span class="source-line-no">476</span><span id="line-476"> // methods but haven't removed it from the queue yet. That's also why the removeFirst</span>
<span class="source-line-no">477</span><span id="line-477"> // call below may be a no-op.</span>
<span class="source-line-no">478</span><span id="line-478"> if (state != State.STREAMING) {</span>
<span class="source-line-no">479</span><span id="line-479"> future.completeExceptionally(new IOException("stream already broken"));</span>
<span class="source-line-no">480</span><span id="line-480"> } else {</span>
<span class="source-line-no">481</span><span id="line-481"> future.complete(lengthAfterFlush);</span>
<span class="source-line-no">482</span><span id="line-482"> }</span>
<span class="source-line-no">483</span><span id="line-483"> // it's the one we have just pushed or just a no-op</span>
<span class="source-line-no">484</span><span id="line-484"> waitingAckQueue.removeFirst();</span>
<span class="source-line-no">485</span><span id="line-485"> }</span>
<span class="source-line-no">486</span><span id="line-486"> } else {</span>
<span class="source-line-no">487</span><span id="line-487"> // we must have acked all the data so the ackedBlockLength must be same with</span>
<span class="source-line-no">488</span><span id="line-488"> // lengthAfterFlush</span>
<span class="source-line-no">489</span><span id="line-489"> future.complete(lengthAfterFlush);</span>
<span class="source-line-no">490</span><span id="line-490"> }</span>
<span class="source-line-no">491</span><span id="line-491"> return;</span>
<span class="source-line-no">492</span><span id="line-492"> }</span>
<span class="source-line-no">493</span><span id="line-493"></span>
<span class="source-line-no">494</span><span id="line-494"> if (encryptor != null) {</span>
<span class="source-line-no">495</span><span id="line-495"> ByteBuf encryptBuf = alloc.directBuffer(dataLen);</span>
<span class="source-line-no">496</span><span id="line-496"> buf.readBytes(encryptBuf, trailingPartialChunkLength);</span>
<span class="source-line-no">497</span><span id="line-497"> int toEncryptLength = dataLen - trailingPartialChunkLength;</span>
<span class="source-line-no">498</span><span id="line-498"> try {</span>
<span class="source-line-no">499</span><span id="line-499"> encryptor.encrypt(buf.nioBuffer(trailingPartialChunkLength, toEncryptLength),</span>
<span class="source-line-no">500</span><span id="line-500"> encryptBuf.nioBuffer(trailingPartialChunkLength, toEncryptLength));</span>
<span class="source-line-no">501</span><span id="line-501"> } catch (IOException e) {</span>
<span class="source-line-no">502</span><span id="line-502"> encryptBuf.release();</span>
<span class="source-line-no">503</span><span id="line-503"> future.completeExceptionally(e);</span>
<span class="source-line-no">504</span><span id="line-504"> return;</span>
<span class="source-line-no">505</span><span id="line-505"> }</span>
<span class="source-line-no">506</span><span id="line-506"> encryptBuf.writerIndex(dataLen);</span>
<span class="source-line-no">507</span><span id="line-507"> buf.release();</span>
<span class="source-line-no">508</span><span id="line-508"> buf = encryptBuf;</span>
<span class="source-line-no">509</span><span id="line-509"> }</span>
<span class="source-line-no">510</span><span id="line-510"></span>
<span class="source-line-no">511</span><span id="line-511"> if (dataLen &gt; maxDataLen) {</span>
<span class="source-line-no">512</span><span id="line-512"> // We need to write out the data by multiple packets as the max packet allowed is 16M.</span>
<span class="source-line-no">513</span><span id="line-513"> long nextSubPacketOffsetInBlock = nextPacketOffsetInBlock;</span>
<span class="source-line-no">514</span><span id="line-514"> for (int remaining = dataLen;;) {</span>
<span class="source-line-no">515</span><span id="line-515"> if (remaining &lt; maxDataLen) {</span>
<span class="source-line-no">516</span><span id="line-516"> flushBuffer(future, buf.readRetainedSlice(remaining), nextSubPacketOffsetInBlock,</span>
<span class="source-line-no">517</span><span id="line-517"> syncBlock);</span>
<span class="source-line-no">518</span><span id="line-518"> break;</span>
<span class="source-line-no">519</span><span id="line-519"> } else {</span>
<span class="source-line-no">520</span><span id="line-520"> flushBuffer(new CompletableFuture&lt;&gt;(), buf.readRetainedSlice(maxDataLen),</span>
<span class="source-line-no">521</span><span id="line-521"> nextSubPacketOffsetInBlock, syncBlock);</span>
<span class="source-line-no">522</span><span id="line-522"> remaining -= maxDataLen;</span>
<span class="source-line-no">523</span><span id="line-523"> nextSubPacketOffsetInBlock += maxDataLen;</span>
<span class="source-line-no">524</span><span id="line-524"> }</span>
<span class="source-line-no">525</span><span id="line-525"> }</span>
<span class="source-line-no">526</span><span id="line-526"> } else {</span>
<span class="source-line-no">527</span><span id="line-527"> flushBuffer(future, buf.retain(), nextPacketOffsetInBlock, syncBlock);</span>
<span class="source-line-no">528</span><span id="line-528"> }</span>
<span class="source-line-no">529</span><span id="line-529"> trailingPartialChunkLength = dataLen % summer.getBytesPerChecksum();</span>
<span class="source-line-no">530</span><span id="line-530"> ByteBuf newBuf = alloc.directBuffer(sendBufSizePRedictor.guess(dataLen))</span>
<span class="source-line-no">531</span><span id="line-531"> .ensureWritable(trailingPartialChunkLength);</span>
<span class="source-line-no">532</span><span id="line-532"> if (trailingPartialChunkLength != 0) {</span>
<span class="source-line-no">533</span><span id="line-533"> buf.readerIndex(dataLen - trailingPartialChunkLength).readBytes(newBuf,</span>
<span class="source-line-no">534</span><span id="line-534"> trailingPartialChunkLength);</span>
<span class="source-line-no">535</span><span id="line-535"> }</span>
<span class="source-line-no">536</span><span id="line-536"> buf.release();</span>
<span class="source-line-no">537</span><span id="line-537"> this.buf = newBuf;</span>
<span class="source-line-no">538</span><span id="line-538"> nextPacketOffsetInBlock += dataLen - trailingPartialChunkLength;</span>
<span class="source-line-no">539</span><span id="line-539"> }</span>
<span class="source-line-no">540</span><span id="line-540"></span>
<span class="source-line-no">541</span><span id="line-541"> /**</span>
<span class="source-line-no">542</span><span id="line-542"> * Flush the buffer out to datanodes.</span>
<span class="source-line-no">543</span><span id="line-543"> * @param syncBlock will call hsync if true, otherwise hflush.</span>
<span class="source-line-no">544</span><span id="line-544"> * @return A CompletableFuture that hold the acked length after flushing.</span>
<span class="source-line-no">545</span><span id="line-545"> */</span>
<span class="source-line-no">546</span><span id="line-546"> @Override</span>
<span class="source-line-no">547</span><span id="line-547"> public CompletableFuture&lt;Long&gt; flush(boolean syncBlock) {</span>
<span class="source-line-no">548</span><span id="line-548"> CompletableFuture&lt;Long&gt; future = new CompletableFuture&lt;&gt;();</span>
<span class="source-line-no">549</span><span id="line-549"> flush0(future, syncBlock);</span>
<span class="source-line-no">550</span><span id="line-550"> return future;</span>
<span class="source-line-no">551</span><span id="line-551"> }</span>
<span class="source-line-no">552</span><span id="line-552"></span>
<span class="source-line-no">553</span><span id="line-553"> private void endBlock() throws IOException {</span>
<span class="source-line-no">554</span><span id="line-554"> Preconditions.checkState(waitingAckQueue.isEmpty(),</span>
<span class="source-line-no">555</span><span id="line-555"> "should call flush first before calling close");</span>
<span class="source-line-no">556</span><span id="line-556"> if (state != State.STREAMING) {</span>
<span class="source-line-no">557</span><span id="line-557"> throw new IOException("stream already broken");</span>
<span class="source-line-no">558</span><span id="line-558"> }</span>
<span class="source-line-no">559</span><span id="line-559"> state = State.CLOSING;</span>
<span class="source-line-no">560</span><span id="line-560"> long finalizedLength = ackedBlockLength;</span>
<span class="source-line-no">561</span><span id="line-561"> PacketHeader header = new PacketHeader(4, finalizedLength, nextPacketSeqno, true, 0, false);</span>
<span class="source-line-no">562</span><span id="line-562"> buf.release();</span>
<span class="source-line-no">563</span><span id="line-563"> buf = null;</span>
<span class="source-line-no">564</span><span id="line-564"> int headerLen = header.getSerializedSize();</span>
<span class="source-line-no">565</span><span id="line-565"> ByteBuf headerBuf = alloc.directBuffer(headerLen);</span>
<span class="source-line-no">566</span><span id="line-566"> header.putInBuffer(headerBuf.nioBuffer(0, headerLen));</span>
<span class="source-line-no">567</span><span id="line-567"> headerBuf.writerIndex(headerLen);</span>
<span class="source-line-no">568</span><span id="line-568"> CompletableFuture&lt;Long&gt; future = new CompletableFuture&lt;&gt;();</span>
<span class="source-line-no">569</span><span id="line-569"> waitingAckQueue.add(new Callback(future, finalizedLength, datanodeInfoMap.keySet(), 0));</span>
<span class="source-line-no">570</span><span id="line-570"> datanodeInfoMap.keySet().forEach(ch -&gt; safeWriteAndFlush(ch, headerBuf.retainedDuplicate()));</span>
<span class="source-line-no">571</span><span id="line-571"> headerBuf.release();</span>
<span class="source-line-no">572</span><span id="line-572"> FutureUtils.get(future);</span>
<span class="source-line-no">573</span><span id="line-573"> }</span>
<span class="source-line-no">574</span><span id="line-574"></span>
<span class="source-line-no">575</span><span id="line-575"> private void closeDataNodeChannelsAndAwait() {</span>
<span class="source-line-no">576</span><span id="line-576"> List&lt;ChannelFuture&gt; futures = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">577</span><span id="line-577"> for (Channel ch : datanodeInfoMap.keySet()) {</span>
<span class="source-line-no">578</span><span id="line-578"> futures.add(ch.close());</span>
<span class="source-line-no">579</span><span id="line-579"> }</span>
<span class="source-line-no">580</span><span id="line-580"> for (ChannelFuture future : futures) {</span>
<span class="source-line-no">581</span><span id="line-581"> consume(future.awaitUninterruptibly());</span>
<span class="source-line-no">582</span><span id="line-582"> }</span>
<span class="source-line-no">583</span><span id="line-583"> }</span>
<span class="source-line-no">584</span><span id="line-584"></span>
<span class="source-line-no">585</span><span id="line-585"> /**</span>
<span class="source-line-no">586</span><span id="line-586"> * The close method when error occurred. Now we just call recoverFileLease.</span>
<span class="source-line-no">587</span><span id="line-587"> */</span>
<span class="source-line-no">588</span><span id="line-588"> @Override</span>
<span class="source-line-no">589</span><span id="line-589"> public void recoverAndClose(CancelableProgressable reporter) throws IOException {</span>
<span class="source-line-no">590</span><span id="line-590"> if (buf != null) {</span>
<span class="source-line-no">591</span><span id="line-591"> buf.release();</span>
<span class="source-line-no">592</span><span id="line-592"> buf = null;</span>
<span class="source-line-no">593</span><span id="line-593"> }</span>
<span class="source-line-no">594</span><span id="line-594"> closeDataNodeChannelsAndAwait();</span>
<span class="source-line-no">595</span><span id="line-595"> endFileLease(client, fileId);</span>
<span class="source-line-no">596</span><span id="line-596"> RecoverLeaseFSUtils.recoverFileLease(dfs, new Path(src), conf,</span>
<span class="source-line-no">597</span><span id="line-597"> reporter == null ? new CancelOnClose(client) : reporter);</span>
<span class="source-line-no">598</span><span id="line-598"> }</span>
<span class="source-line-no">599</span><span id="line-599"></span>
<span class="source-line-no">600</span><span id="line-600"> /**</span>
<span class="source-line-no">601</span><span id="line-601"> * End the current block and complete file at namenode. You should call</span>
<span class="source-line-no">602</span><span id="line-602"> * {@link #recoverAndClose(CancelableProgressable)} if this method throws an exception.</span>
<span class="source-line-no">603</span><span id="line-603"> */</span>
<span class="source-line-no">604</span><span id="line-604"> @Override</span>
<span class="source-line-no">605</span><span id="line-605"> public void close() throws IOException {</span>
<span class="source-line-no">606</span><span id="line-606"> endBlock();</span>
<span class="source-line-no">607</span><span id="line-607"> state = State.CLOSED;</span>
<span class="source-line-no">608</span><span id="line-608"> closeDataNodeChannelsAndAwait();</span>
<span class="source-line-no">609</span><span id="line-609"> block.setNumBytes(ackedBlockLength);</span>
<span class="source-line-no">610</span><span id="line-610"> completeFile(client, namenode, src, clientName, block, fileId);</span>
<span class="source-line-no">611</span><span id="line-611"> }</span>
<span class="source-line-no">612</span><span id="line-612"></span>
<span class="source-line-no">613</span><span id="line-613"> @Override</span>
<span class="source-line-no">614</span><span id="line-614"> public boolean isBroken() {</span>
<span class="source-line-no">615</span><span id="line-615"> return state == State.BROKEN;</span>
<span class="source-line-no">616</span><span id="line-616"> }</span>
<span class="source-line-no">617</span><span id="line-617"></span>
<span class="source-line-no">618</span><span id="line-618"> @Override</span>
<span class="source-line-no">619</span><span id="line-619"> public long getSyncedLength() {</span>
<span class="source-line-no">620</span><span id="line-620"> return this.ackedBlockLength;</span>
<span class="source-line-no">621</span><span id="line-621"> }</span>
<span class="source-line-no">622</span><span id="line-622"></span>
<span class="source-line-no">623</span><span id="line-623"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">624</span><span id="line-624"> allowedOnPath = ".*/src/test/.*")</span>
<span class="source-line-no">625</span><span id="line-625"> Map&lt;Channel, DatanodeInfo&gt; getDatanodeInfoMap() {</span>
<span class="source-line-no">626</span><span id="line-626"> return this.datanodeInfoMap;</span>
<span class="source-line-no">627</span><span id="line-627"> }</span>
<span class="source-line-no">628</span><span id="line-628">}</span>
</pre>
</div>
</main>
</body>
</html>