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