blob: 65cc465e5a1484acf4b6ef9f46c598f03a88e248 [file] [log] [blame]
<!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.FanOutOneBlockAsyncDFSOutputSaslHelper.createEncryptor;<a name="line.20"></a>
<span class="sourceLineNo">021</span>import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper.trySaslNegotiate;<a name="line.21"></a>
<span class="sourceLineNo">022</span>import static org.apache.hadoop.hbase.util.LocatedBlockHelper.getLocatedBlockLocations;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import static org.apache.hadoop.hbase.util.NettyFutureUtils.addListener;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import static org.apache.hadoop.hbase.util.NettyFutureUtils.safeClose;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import static org.apache.hadoop.hbase.util.NettyFutureUtils.safeWriteAndFlush;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import static org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage.PIPELINE_SETUP_CREATE;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import static org.apache.hbase.thirdparty.io.netty.channel.ChannelOption.CONNECT_TIMEOUT_MILLIS;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.READER_IDLE;<a name="line.31"></a>
<span class="sourceLineNo">032</span><a name="line.32"></a>
<span class="sourceLineNo">033</span>import com.google.protobuf.CodedOutputStream;<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.io.InterruptedIOException;<a name="line.35"></a>
<span class="sourceLineNo">036</span>import java.lang.reflect.InvocationTargetException;<a name="line.36"></a>
<span class="sourceLineNo">037</span>import java.lang.reflect.Method;<a name="line.37"></a>
<span class="sourceLineNo">038</span>import java.util.ArrayList;<a name="line.38"></a>
<span class="sourceLineNo">039</span>import java.util.EnumSet;<a name="line.39"></a>
<span class="sourceLineNo">040</span>import java.util.HashSet;<a name="line.40"></a>
<span class="sourceLineNo">041</span>import java.util.IdentityHashMap;<a name="line.41"></a>
<span class="sourceLineNo">042</span>import java.util.List;<a name="line.42"></a>
<span class="sourceLineNo">043</span>import java.util.Map;<a name="line.43"></a>
<span class="sourceLineNo">044</span>import java.util.Set;<a name="line.44"></a>
<span class="sourceLineNo">045</span>import java.util.concurrent.TimeUnit;<a name="line.45"></a>
<span class="sourceLineNo">046</span>import org.apache.hadoop.conf.Configuration;<a name="line.46"></a>
<span class="sourceLineNo">047</span>import org.apache.hadoop.crypto.CryptoProtocolVersion;<a name="line.47"></a>
<span class="sourceLineNo">048</span>import org.apache.hadoop.crypto.Encryptor;<a name="line.48"></a>
<span class="sourceLineNo">049</span>import org.apache.hadoop.fs.CreateFlag;<a name="line.49"></a>
<span class="sourceLineNo">050</span>import org.apache.hadoop.fs.FileSystem;<a name="line.50"></a>
<span class="sourceLineNo">051</span>import org.apache.hadoop.fs.FileSystemLinkResolver;<a name="line.51"></a>
<span class="sourceLineNo">052</span>import org.apache.hadoop.fs.Path;<a name="line.52"></a>
<span class="sourceLineNo">053</span>import org.apache.hadoop.fs.StorageType;<a name="line.53"></a>
<span class="sourceLineNo">054</span>import org.apache.hadoop.fs.UnresolvedLinkException;<a name="line.54"></a>
<span class="sourceLineNo">055</span>import org.apache.hadoop.fs.permission.FsPermission;<a name="line.55"></a>
<span class="sourceLineNo">056</span>import org.apache.hadoop.hbase.client.ConnectionUtils;<a name="line.56"></a>
<span class="sourceLineNo">057</span>import org.apache.hadoop.hbase.io.asyncfs.monitor.ExcludeDatanodeManager;<a name="line.57"></a>
<span class="sourceLineNo">058</span>import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor;<a name="line.58"></a>
<span class="sourceLineNo">059</span>import org.apache.hadoop.hbase.util.CancelableProgressable;<a name="line.59"></a>
<span class="sourceLineNo">060</span>import org.apache.hadoop.hdfs.DFSClient;<a name="line.60"></a>
<span class="sourceLineNo">061</span>import org.apache.hadoop.hdfs.DFSOutputStream;<a name="line.61"></a>
<span class="sourceLineNo">062</span>import org.apache.hadoop.hdfs.DistributedFileSystem;<a name="line.62"></a>
<span class="sourceLineNo">063</span>import org.apache.hadoop.hdfs.protocol.ClientProtocol;<a name="line.63"></a>
<span class="sourceLineNo">064</span>import org.apache.hadoop.hdfs.protocol.DatanodeInfo;<a name="line.64"></a>
<span class="sourceLineNo">065</span>import org.apache.hadoop.hdfs.protocol.ExtendedBlock;<a name="line.65"></a>
<span class="sourceLineNo">066</span>import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;<a name="line.66"></a>
<span class="sourceLineNo">067</span>import org.apache.hadoop.hdfs.protocol.LocatedBlock;<a name="line.67"></a>
<span class="sourceLineNo">068</span>import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;<a name="line.68"></a>
<span class="sourceLineNo">069</span>import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;<a name="line.69"></a>
<span class="sourceLineNo">070</span>import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;<a name="line.70"></a>
<span class="sourceLineNo">071</span>import org.apache.hadoop.hdfs.protocol.datatransfer.Op;<a name="line.71"></a>
<span class="sourceLineNo">072</span>import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;<a name="line.72"></a>
<span class="sourceLineNo">073</span>import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck.ECN;<a name="line.73"></a>
<span class="sourceLineNo">074</span>import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;<a name="line.74"></a>
<span class="sourceLineNo">075</span>import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;<a name="line.75"></a>
<span class="sourceLineNo">076</span>import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto;<a name="line.76"></a>
<span class="sourceLineNo">077</span>import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto;<a name="line.77"></a>
<span class="sourceLineNo">078</span>import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;<a name="line.78"></a>
<span class="sourceLineNo">079</span>import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;<a name="line.79"></a>
<span class="sourceLineNo">080</span>import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;<a name="line.80"></a>
<span class="sourceLineNo">081</span>import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;<a name="line.81"></a>
<span class="sourceLineNo">082</span>import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;<a name="line.82"></a>
<span class="sourceLineNo">083</span>import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;<a name="line.83"></a>
<span class="sourceLineNo">084</span>import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;<a name="line.84"></a>
<span class="sourceLineNo">085</span>import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;<a name="line.85"></a>
<span class="sourceLineNo">086</span>import org.apache.hadoop.io.EnumSetWritable;<a name="line.86"></a>
<span class="sourceLineNo">087</span>import org.apache.hadoop.ipc.RemoteException;<a name="line.87"></a>
<span class="sourceLineNo">088</span>import org.apache.hadoop.net.NetUtils;<a name="line.88"></a>
<span class="sourceLineNo">089</span>import org.apache.hadoop.security.token.Token;<a name="line.89"></a>
<span class="sourceLineNo">090</span>import org.apache.hadoop.util.DataChecksum;<a name="line.90"></a>
<span class="sourceLineNo">091</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.91"></a>
<span class="sourceLineNo">092</span>import org.slf4j.Logger;<a name="line.92"></a>
<span class="sourceLineNo">093</span>import org.slf4j.LoggerFactory;<a name="line.93"></a>
<span class="sourceLineNo">094</span><a name="line.94"></a>
<span class="sourceLineNo">095</span>import org.apache.hbase.thirdparty.io.netty.bootstrap.Bootstrap;<a name="line.95"></a>
<span class="sourceLineNo">096</span>import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;<a name="line.96"></a>
<span class="sourceLineNo">097</span>import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufAllocator;<a name="line.97"></a>
<span class="sourceLineNo">098</span>import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream;<a name="line.98"></a>
<span class="sourceLineNo">099</span>import org.apache.hbase.thirdparty.io.netty.buffer.PooledByteBufAllocator;<a name="line.99"></a>
<span class="sourceLineNo">100</span>import org.apache.hbase.thirdparty.io.netty.channel.Channel;<a name="line.100"></a>
<span class="sourceLineNo">101</span>import org.apache.hbase.thirdparty.io.netty.channel.ChannelFuture;<a name="line.101"></a>
<span class="sourceLineNo">102</span>import org.apache.hbase.thirdparty.io.netty.channel.ChannelFutureListener;<a name="line.102"></a>
<span class="sourceLineNo">103</span>import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandler;<a name="line.103"></a>
<span class="sourceLineNo">104</span>import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;<a name="line.104"></a>
<span class="sourceLineNo">105</span>import org.apache.hbase.thirdparty.io.netty.channel.ChannelInitializer;<a name="line.105"></a>
<span class="sourceLineNo">106</span>import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline;<a name="line.106"></a>
<span class="sourceLineNo">107</span>import org.apache.hbase.thirdparty.io.netty.channel.EventLoop;<a name="line.107"></a>
<span class="sourceLineNo">108</span>import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;<a name="line.108"></a>
<span class="sourceLineNo">109</span>import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler;<a name="line.109"></a>
<span class="sourceLineNo">110</span>import org.apache.hbase.thirdparty.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;<a name="line.110"></a>
<span class="sourceLineNo">111</span>import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateEvent;<a name="line.111"></a>
<span class="sourceLineNo">112</span>import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler;<a name="line.112"></a>
<span class="sourceLineNo">113</span>import org.apache.hbase.thirdparty.io.netty.util.concurrent.Future;<a name="line.113"></a>
<span class="sourceLineNo">114</span>import org.apache.hbase.thirdparty.io.netty.util.concurrent.FutureListener;<a name="line.114"></a>
<span class="sourceLineNo">115</span>import org.apache.hbase.thirdparty.io.netty.util.concurrent.Promise;<a name="line.115"></a>
<span class="sourceLineNo">116</span><a name="line.116"></a>
<span class="sourceLineNo">117</span>/**<a name="line.117"></a>
<span class="sourceLineNo">118</span> * Helper class for implementing {@link FanOutOneBlockAsyncDFSOutput}.<a name="line.118"></a>
<span class="sourceLineNo">119</span> */<a name="line.119"></a>
<span class="sourceLineNo">120</span>@InterfaceAudience.Private<a name="line.120"></a>
<span class="sourceLineNo">121</span>public final class FanOutOneBlockAsyncDFSOutputHelper {<a name="line.121"></a>
<span class="sourceLineNo">122</span> private static final Logger LOG =<a name="line.122"></a>
<span class="sourceLineNo">123</span> LoggerFactory.getLogger(FanOutOneBlockAsyncDFSOutputHelper.class);<a name="line.123"></a>
<span class="sourceLineNo">124</span><a name="line.124"></a>
<span class="sourceLineNo">125</span> private FanOutOneBlockAsyncDFSOutputHelper() {<a name="line.125"></a>
<span class="sourceLineNo">126</span> }<a name="line.126"></a>
<span class="sourceLineNo">127</span><a name="line.127"></a>
<span class="sourceLineNo">128</span> public static final String ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES = "hbase.fs.async.create.retries";<a name="line.128"></a>
<span class="sourceLineNo">129</span><a name="line.129"></a>
<span class="sourceLineNo">130</span> public static final int DEFAULT_ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES = 10;<a name="line.130"></a>
<span class="sourceLineNo">131</span> // use pooled allocator for performance.<a name="line.131"></a>
<span class="sourceLineNo">132</span> private static final ByteBufAllocator ALLOC = PooledByteBufAllocator.DEFAULT;<a name="line.132"></a>
<span class="sourceLineNo">133</span><a name="line.133"></a>
<span class="sourceLineNo">134</span> // copied from DFSPacket since it is package private.<a name="line.134"></a>
<span class="sourceLineNo">135</span> public static final long HEART_BEAT_SEQNO = -1L;<a name="line.135"></a>
<span class="sourceLineNo">136</span><a name="line.136"></a>
<span class="sourceLineNo">137</span> // Timeouts for communicating with DataNode for streaming writes/reads<a name="line.137"></a>
<span class="sourceLineNo">138</span> public static final int READ_TIMEOUT = 60 * 1000;<a name="line.138"></a>
<span class="sourceLineNo">139</span><a name="line.139"></a>
<span class="sourceLineNo">140</span> private interface LeaseManager {<a name="line.140"></a>
<span class="sourceLineNo">141</span><a name="line.141"></a>
<span class="sourceLineNo">142</span> void begin(DFSClient client, long inodeId);<a name="line.142"></a>
<span class="sourceLineNo">143</span><a name="line.143"></a>
<span class="sourceLineNo">144</span> void end(DFSClient client, long inodeId);<a name="line.144"></a>
<span class="sourceLineNo">145</span> }<a name="line.145"></a>
<span class="sourceLineNo">146</span><a name="line.146"></a>
<span class="sourceLineNo">147</span> private static final LeaseManager LEASE_MANAGER;<a name="line.147"></a>
<span class="sourceLineNo">148</span><a name="line.148"></a>
<span class="sourceLineNo">149</span> // helper class for creating files.<a name="line.149"></a>
<span class="sourceLineNo">150</span> private interface FileCreator {<a name="line.150"></a>
<span class="sourceLineNo">151</span> default HdfsFileStatus create(ClientProtocol instance, String src, FsPermission masked,<a name="line.151"></a>
<span class="sourceLineNo">152</span> String clientName, EnumSetWritable&lt;CreateFlag&gt; flag, boolean createParent, short replication,<a name="line.152"></a>
<span class="sourceLineNo">153</span> long blockSize, CryptoProtocolVersion[] supportedVersions) throws Exception {<a name="line.153"></a>
<span class="sourceLineNo">154</span> try {<a name="line.154"></a>
<span class="sourceLineNo">155</span> return (HdfsFileStatus) createObject(instance, src, masked, clientName, flag, createParent,<a name="line.155"></a>
<span class="sourceLineNo">156</span> replication, blockSize, supportedVersions);<a name="line.156"></a>
<span class="sourceLineNo">157</span> } catch (InvocationTargetException e) {<a name="line.157"></a>
<span class="sourceLineNo">158</span> if (e.getCause() instanceof Exception) {<a name="line.158"></a>
<span class="sourceLineNo">159</span> throw (Exception) e.getCause();<a name="line.159"></a>
<span class="sourceLineNo">160</span> } else {<a name="line.160"></a>
<span class="sourceLineNo">161</span> throw new RuntimeException(e.getCause());<a name="line.161"></a>
<span class="sourceLineNo">162</span> }<a name="line.162"></a>
<span class="sourceLineNo">163</span> }<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> Object createObject(ClientProtocol instance, String src, FsPermission masked, String clientName,<a name="line.166"></a>
<span class="sourceLineNo">167</span> EnumSetWritable&lt;CreateFlag&gt; flag, boolean createParent, short replication, long blockSize,<a name="line.167"></a>
<span class="sourceLineNo">168</span> CryptoProtocolVersion[] supportedVersions) throws Exception;<a name="line.168"></a>
<span class="sourceLineNo">169</span> }<a name="line.169"></a>
<span class="sourceLineNo">170</span><a name="line.170"></a>
<span class="sourceLineNo">171</span> private static final FileCreator FILE_CREATOR;<a name="line.171"></a>
<span class="sourceLineNo">172</span><a name="line.172"></a>
<span class="sourceLineNo">173</span> private static LeaseManager createLeaseManager() throws NoSuchMethodException {<a name="line.173"></a>
<span class="sourceLineNo">174</span> Method beginFileLeaseMethod =<a name="line.174"></a>
<span class="sourceLineNo">175</span> DFSClient.class.getDeclaredMethod("beginFileLease", long.class, DFSOutputStream.class);<a name="line.175"></a>
<span class="sourceLineNo">176</span> beginFileLeaseMethod.setAccessible(true);<a name="line.176"></a>
<span class="sourceLineNo">177</span> Method endFileLeaseMethod = DFSClient.class.getDeclaredMethod("endFileLease", long.class);<a name="line.177"></a>
<span class="sourceLineNo">178</span> endFileLeaseMethod.setAccessible(true);<a name="line.178"></a>
<span class="sourceLineNo">179</span> return new LeaseManager() {<a name="line.179"></a>
<span class="sourceLineNo">180</span><a name="line.180"></a>
<span class="sourceLineNo">181</span> @Override<a name="line.181"></a>
<span class="sourceLineNo">182</span> public void begin(DFSClient client, long inodeId) {<a name="line.182"></a>
<span class="sourceLineNo">183</span> try {<a name="line.183"></a>
<span class="sourceLineNo">184</span> beginFileLeaseMethod.invoke(client, inodeId, null);<a name="line.184"></a>
<span class="sourceLineNo">185</span> } catch (IllegalAccessException | InvocationTargetException e) {<a name="line.185"></a>
<span class="sourceLineNo">186</span> throw new RuntimeException(e);<a name="line.186"></a>
<span class="sourceLineNo">187</span> }<a name="line.187"></a>
<span class="sourceLineNo">188</span> }<a name="line.188"></a>
<span class="sourceLineNo">189</span><a name="line.189"></a>
<span class="sourceLineNo">190</span> @Override<a name="line.190"></a>
<span class="sourceLineNo">191</span> public void end(DFSClient client, long inodeId) {<a name="line.191"></a>
<span class="sourceLineNo">192</span> try {<a name="line.192"></a>
<span class="sourceLineNo">193</span> endFileLeaseMethod.invoke(client, inodeId);<a name="line.193"></a>
<span class="sourceLineNo">194</span> } catch (IllegalAccessException | InvocationTargetException e) {<a name="line.194"></a>
<span class="sourceLineNo">195</span> throw new RuntimeException(e);<a name="line.195"></a>
<span class="sourceLineNo">196</span> }<a name="line.196"></a>
<span class="sourceLineNo">197</span> }<a name="line.197"></a>
<span class="sourceLineNo">198</span> };<a name="line.198"></a>
<span class="sourceLineNo">199</span> }<a name="line.199"></a>
<span class="sourceLineNo">200</span><a name="line.200"></a>
<span class="sourceLineNo">201</span> private static FileCreator createFileCreator3_3() throws NoSuchMethodException {<a name="line.201"></a>
<span class="sourceLineNo">202</span> Method createMethod = ClientProtocol.class.getMethod("create", String.class, FsPermission.class,<a name="line.202"></a>
<span class="sourceLineNo">203</span> String.class, EnumSetWritable.class, boolean.class, short.class, long.class,<a name="line.203"></a>
<span class="sourceLineNo">204</span> CryptoProtocolVersion[].class, String.class, String.class);<a name="line.204"></a>
<span class="sourceLineNo">205</span><a name="line.205"></a>
<span class="sourceLineNo">206</span> return (instance, src, masked, clientName, flag, createParent, replication, blockSize,<a name="line.206"></a>
<span class="sourceLineNo">207</span> supportedVersions) -&gt; {<a name="line.207"></a>
<span class="sourceLineNo">208</span> return (HdfsFileStatus) createMethod.invoke(instance, src, masked, clientName, flag,<a name="line.208"></a>
<span class="sourceLineNo">209</span> createParent, replication, blockSize, supportedVersions, null, null);<a name="line.209"></a>
<span class="sourceLineNo">210</span> };<a name="line.210"></a>
<span class="sourceLineNo">211</span> }<a name="line.211"></a>
<span class="sourceLineNo">212</span><a name="line.212"></a>
<span class="sourceLineNo">213</span> private static FileCreator createFileCreator3() throws NoSuchMethodException {<a name="line.213"></a>
<span class="sourceLineNo">214</span> Method createMethod = ClientProtocol.class.getMethod("create", String.class, FsPermission.class,<a name="line.214"></a>
<span class="sourceLineNo">215</span> String.class, EnumSetWritable.class, boolean.class, short.class, long.class,<a name="line.215"></a>
<span class="sourceLineNo">216</span> CryptoProtocolVersion[].class, String.class);<a name="line.216"></a>
<span class="sourceLineNo">217</span><a name="line.217"></a>
<span class="sourceLineNo">218</span> return (instance, src, masked, clientName, flag, createParent, replication, blockSize,<a name="line.218"></a>
<span class="sourceLineNo">219</span> supportedVersions) -&gt; {<a name="line.219"></a>
<span class="sourceLineNo">220</span> return (HdfsFileStatus) createMethod.invoke(instance, src, masked, clientName, flag,<a name="line.220"></a>
<span class="sourceLineNo">221</span> createParent, replication, blockSize, supportedVersions, null);<a name="line.221"></a>
<span class="sourceLineNo">222</span> };<a name="line.222"></a>
<span class="sourceLineNo">223</span> }<a name="line.223"></a>
<span class="sourceLineNo">224</span><a name="line.224"></a>
<span class="sourceLineNo">225</span> private static FileCreator createFileCreator() throws NoSuchMethodException {<a name="line.225"></a>
<span class="sourceLineNo">226</span> try {<a name="line.226"></a>
<span class="sourceLineNo">227</span> return createFileCreator3_3();<a name="line.227"></a>
<span class="sourceLineNo">228</span> } catch (NoSuchMethodException e) {<a name="line.228"></a>
<span class="sourceLineNo">229</span> LOG.debug("ClientProtocol::create wrong number of arguments, should be hadoop 3.2 or below");<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> return createFileCreator3();<a name="line.232"></a>
<span class="sourceLineNo">233</span> }<a name="line.233"></a>
<span class="sourceLineNo">234</span><a name="line.234"></a>
<span class="sourceLineNo">235</span> // cancel the processing if DFSClient is already closed.<a name="line.235"></a>
<span class="sourceLineNo">236</span> static final class CancelOnClose implements CancelableProgressable {<a name="line.236"></a>
<span class="sourceLineNo">237</span><a name="line.237"></a>
<span class="sourceLineNo">238</span> private final DFSClient client;<a name="line.238"></a>
<span class="sourceLineNo">239</span><a name="line.239"></a>
<span class="sourceLineNo">240</span> public CancelOnClose(DFSClient client) {<a name="line.240"></a>
<span class="sourceLineNo">241</span> this.client = client;<a name="line.241"></a>
<span class="sourceLineNo">242</span> }<a name="line.242"></a>
<span class="sourceLineNo">243</span><a name="line.243"></a>
<span class="sourceLineNo">244</span> @Override<a name="line.244"></a>
<span class="sourceLineNo">245</span> public boolean progress() {<a name="line.245"></a>
<span class="sourceLineNo">246</span> return client.isClientRunning();<a name="line.246"></a>
<span class="sourceLineNo">247</span> }<a name="line.247"></a>
<span class="sourceLineNo">248</span> }<a name="line.248"></a>
<span class="sourceLineNo">249</span><a name="line.249"></a>
<span class="sourceLineNo">250</span> static {<a name="line.250"></a>
<span class="sourceLineNo">251</span> try {<a name="line.251"></a>
<span class="sourceLineNo">252</span> LEASE_MANAGER = createLeaseManager();<a name="line.252"></a>
<span class="sourceLineNo">253</span> FILE_CREATOR = createFileCreator();<a name="line.253"></a>
<span class="sourceLineNo">254</span> } catch (Exception e) {<a name="line.254"></a>
<span class="sourceLineNo">255</span> String msg = "Couldn't properly initialize access to HDFS internals. Please "<a name="line.255"></a>
<span class="sourceLineNo">256</span> + "update your WAL Provider to not make use of the 'asyncfs' provider. See "<a name="line.256"></a>
<span class="sourceLineNo">257</span> + "HBASE-16110 for more information.";<a name="line.257"></a>
<span class="sourceLineNo">258</span> LOG.error(msg, e);<a name="line.258"></a>
<span class="sourceLineNo">259</span> throw new Error(msg, e);<a name="line.259"></a>
<span class="sourceLineNo">260</span> }<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> static void beginFileLease(DFSClient client, long inodeId) {<a name="line.263"></a>
<span class="sourceLineNo">264</span> LEASE_MANAGER.begin(client, inodeId);<a name="line.264"></a>
<span class="sourceLineNo">265</span> }<a name="line.265"></a>
<span class="sourceLineNo">266</span><a name="line.266"></a>
<span class="sourceLineNo">267</span> static void endFileLease(DFSClient client, long inodeId) {<a name="line.267"></a>
<span class="sourceLineNo">268</span> LEASE_MANAGER.end(client, inodeId);<a name="line.268"></a>
<span class="sourceLineNo">269</span> }<a name="line.269"></a>
<span class="sourceLineNo">270</span><a name="line.270"></a>
<span class="sourceLineNo">271</span> static DataChecksum createChecksum(DFSClient client) {<a name="line.271"></a>
<span class="sourceLineNo">272</span> return client.getConf().createChecksum(null);<a name="line.272"></a>
<span class="sourceLineNo">273</span> }<a name="line.273"></a>
<span class="sourceLineNo">274</span><a name="line.274"></a>
<span class="sourceLineNo">275</span> static Status getStatus(PipelineAckProto ack) {<a name="line.275"></a>
<span class="sourceLineNo">276</span> List&lt;Integer&gt; flagList = ack.getFlagList();<a name="line.276"></a>
<span class="sourceLineNo">277</span> Integer headerFlag;<a name="line.277"></a>
<span class="sourceLineNo">278</span> if (flagList.isEmpty()) {<a name="line.278"></a>
<span class="sourceLineNo">279</span> Status reply = ack.getReply(0);<a name="line.279"></a>
<span class="sourceLineNo">280</span> headerFlag = PipelineAck.combineHeader(ECN.DISABLED, reply);<a name="line.280"></a>
<span class="sourceLineNo">281</span> } else {<a name="line.281"></a>
<span class="sourceLineNo">282</span> headerFlag = flagList.get(0);<a name="line.282"></a>
<span class="sourceLineNo">283</span> }<a name="line.283"></a>
<span class="sourceLineNo">284</span> return PipelineAck.getStatusFromHeader(headerFlag);<a name="line.284"></a>
<span class="sourceLineNo">285</span> }<a name="line.285"></a>
<span class="sourceLineNo">286</span><a name="line.286"></a>
<span class="sourceLineNo">287</span> private static void processWriteBlockResponse(Channel channel, DatanodeInfo dnInfo,<a name="line.287"></a>
<span class="sourceLineNo">288</span> Promise&lt;Channel&gt; promise, int timeoutMs) {<a name="line.288"></a>
<span class="sourceLineNo">289</span> channel.pipeline().addLast(new IdleStateHandler(timeoutMs, 0, 0, TimeUnit.MILLISECONDS),<a name="line.289"></a>
<span class="sourceLineNo">290</span> new ProtobufVarint32FrameDecoder(),<a name="line.290"></a>
<span class="sourceLineNo">291</span> new ProtobufDecoder(BlockOpResponseProto.getDefaultInstance()),<a name="line.291"></a>
<span class="sourceLineNo">292</span> new SimpleChannelInboundHandler&lt;BlockOpResponseProto&gt;() {<a name="line.292"></a>
<span class="sourceLineNo">293</span><a name="line.293"></a>
<span class="sourceLineNo">294</span> @Override<a name="line.294"></a>
<span class="sourceLineNo">295</span> protected void channelRead0(ChannelHandlerContext ctx, BlockOpResponseProto resp)<a name="line.295"></a>
<span class="sourceLineNo">296</span> throws Exception {<a name="line.296"></a>
<span class="sourceLineNo">297</span> Status pipelineStatus = resp.getStatus();<a name="line.297"></a>
<span class="sourceLineNo">298</span> if (PipelineAck.isRestartOOBStatus(pipelineStatus)) {<a name="line.298"></a>
<span class="sourceLineNo">299</span> throw new IOException("datanode " + dnInfo + " is restarting");<a name="line.299"></a>
<span class="sourceLineNo">300</span> }<a name="line.300"></a>
<span class="sourceLineNo">301</span> String logInfo = "ack with firstBadLink as " + resp.getFirstBadLink();<a name="line.301"></a>
<span class="sourceLineNo">302</span> if (resp.getStatus() != Status.SUCCESS) {<a name="line.302"></a>
<span class="sourceLineNo">303</span> if (resp.getStatus() == Status.ERROR_ACCESS_TOKEN) {<a name="line.303"></a>
<span class="sourceLineNo">304</span> throw new InvalidBlockTokenException("Got access token error" + ", status message "<a name="line.304"></a>
<span class="sourceLineNo">305</span> + resp.getMessage() + ", " + logInfo);<a name="line.305"></a>
<span class="sourceLineNo">306</span> } else {<a name="line.306"></a>
<span class="sourceLineNo">307</span> throw new IOException("Got error" + ", status=" + resp.getStatus().name()<a name="line.307"></a>
<span class="sourceLineNo">308</span> + ", status message " + resp.getMessage() + ", " + logInfo);<a name="line.308"></a>
<span class="sourceLineNo">309</span> }<a name="line.309"></a>
<span class="sourceLineNo">310</span> }<a name="line.310"></a>
<span class="sourceLineNo">311</span> // success<a name="line.311"></a>
<span class="sourceLineNo">312</span> ChannelPipeline p = ctx.pipeline();<a name="line.312"></a>
<span class="sourceLineNo">313</span> for (ChannelHandler handler; (handler = p.removeLast()) != null;) {<a name="line.313"></a>
<span class="sourceLineNo">314</span> // do not remove all handlers because we may have wrap or unwrap handlers at the header<a name="line.314"></a>
<span class="sourceLineNo">315</span> // of pipeline.<a name="line.315"></a>
<span class="sourceLineNo">316</span> if (handler instanceof IdleStateHandler) {<a name="line.316"></a>
<span class="sourceLineNo">317</span> break;<a name="line.317"></a>
<span class="sourceLineNo">318</span> }<a name="line.318"></a>
<span class="sourceLineNo">319</span> }<a name="line.319"></a>
<span class="sourceLineNo">320</span> // Disable auto read here. Enable it after we setup the streaming pipeline in<a name="line.320"></a>
<span class="sourceLineNo">321</span> // FanOutOneBLockAsyncDFSOutput.<a name="line.321"></a>
<span class="sourceLineNo">322</span> ctx.channel().config().setAutoRead(false);<a name="line.322"></a>
<span class="sourceLineNo">323</span> promise.trySuccess(ctx.channel());<a name="line.323"></a>
<span class="sourceLineNo">324</span> }<a name="line.324"></a>
<span class="sourceLineNo">325</span><a name="line.325"></a>
<span class="sourceLineNo">326</span> @Override<a name="line.326"></a>
<span class="sourceLineNo">327</span> public void channelInactive(ChannelHandlerContext ctx) throws Exception {<a name="line.327"></a>
<span class="sourceLineNo">328</span> promise.tryFailure(new IOException("connection to " + dnInfo + " is closed"));<a name="line.328"></a>
<span class="sourceLineNo">329</span> }<a name="line.329"></a>
<span class="sourceLineNo">330</span><a name="line.330"></a>
<span class="sourceLineNo">331</span> @Override<a name="line.331"></a>
<span class="sourceLineNo">332</span> public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {<a name="line.332"></a>
<span class="sourceLineNo">333</span> if (evt instanceof IdleStateEvent &amp;&amp; ((IdleStateEvent) evt).state() == READER_IDLE) {<a name="line.333"></a>
<span class="sourceLineNo">334</span> promise<a name="line.334"></a>
<span class="sourceLineNo">335</span> .tryFailure(new IOException("Timeout(" + timeoutMs + "ms) waiting for response"));<a name="line.335"></a>
<span class="sourceLineNo">336</span> } else {<a name="line.336"></a>
<span class="sourceLineNo">337</span> super.userEventTriggered(ctx, evt);<a name="line.337"></a>
<span class="sourceLineNo">338</span> }<a name="line.338"></a>
<span class="sourceLineNo">339</span> }<a name="line.339"></a>
<span class="sourceLineNo">340</span><a name="line.340"></a>
<span class="sourceLineNo">341</span> @Override<a name="line.341"></a>
<span class="sourceLineNo">342</span> public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {<a name="line.342"></a>
<span class="sourceLineNo">343</span> promise.tryFailure(cause);<a name="line.343"></a>
<span class="sourceLineNo">344</span> }<a name="line.344"></a>
<span class="sourceLineNo">345</span> });<a name="line.345"></a>
<span class="sourceLineNo">346</span> }<a name="line.346"></a>
<span class="sourceLineNo">347</span><a name="line.347"></a>
<span class="sourceLineNo">348</span> private static void requestWriteBlock(Channel channel, StorageType storageType,<a name="line.348"></a>
<span class="sourceLineNo">349</span> OpWriteBlockProto.Builder writeBlockProtoBuilder) throws IOException {<a name="line.349"></a>
<span class="sourceLineNo">350</span> OpWriteBlockProto proto =<a name="line.350"></a>
<span class="sourceLineNo">351</span> writeBlockProtoBuilder.setStorageType(PBHelperClient.convertStorageType(storageType)).build();<a name="line.351"></a>
<span class="sourceLineNo">352</span> int protoLen = proto.getSerializedSize();<a name="line.352"></a>
<span class="sourceLineNo">353</span> ByteBuf buffer =<a name="line.353"></a>
<span class="sourceLineNo">354</span> channel.alloc().buffer(3 + CodedOutputStream.computeRawVarint32Size(protoLen) + protoLen);<a name="line.354"></a>
<span class="sourceLineNo">355</span> buffer.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);<a name="line.355"></a>
<span class="sourceLineNo">356</span> buffer.writeByte(Op.WRITE_BLOCK.code);<a name="line.356"></a>
<span class="sourceLineNo">357</span> proto.writeDelimitedTo(new ByteBufOutputStream(buffer));<a name="line.357"></a>
<span class="sourceLineNo">358</span> safeWriteAndFlush(channel, buffer);<a name="line.358"></a>
<span class="sourceLineNo">359</span> }<a name="line.359"></a>
<span class="sourceLineNo">360</span><a name="line.360"></a>
<span class="sourceLineNo">361</span> private static void initialize(Configuration conf, Channel channel, DatanodeInfo dnInfo,<a name="line.361"></a>
<span class="sourceLineNo">362</span> StorageType storageType, OpWriteBlockProto.Builder writeBlockProtoBuilder, int timeoutMs,<a name="line.362"></a>
<span class="sourceLineNo">363</span> DFSClient client, Token&lt;BlockTokenIdentifier&gt; accessToken, Promise&lt;Channel&gt; promise)<a name="line.363"></a>
<span class="sourceLineNo">364</span> throws IOException {<a name="line.364"></a>
<span class="sourceLineNo">365</span> Promise&lt;Void&gt; saslPromise = channel.eventLoop().newPromise();<a name="line.365"></a>
<span class="sourceLineNo">366</span> trySaslNegotiate(conf, channel, dnInfo, timeoutMs, client, accessToken, saslPromise);<a name="line.366"></a>
<span class="sourceLineNo">367</span> addListener(saslPromise, new FutureListener&lt;Void&gt;() {<a name="line.367"></a>
<span class="sourceLineNo">368</span><a name="line.368"></a>
<span class="sourceLineNo">369</span> @Override<a name="line.369"></a>
<span class="sourceLineNo">370</span> public void operationComplete(Future&lt;Void&gt; future) throws Exception {<a name="line.370"></a>
<span class="sourceLineNo">371</span> if (future.isSuccess()) {<a name="line.371"></a>
<span class="sourceLineNo">372</span> // setup response processing pipeline first, then send request.<a name="line.372"></a>
<span class="sourceLineNo">373</span> processWriteBlockResponse(channel, dnInfo, promise, timeoutMs);<a name="line.373"></a>
<span class="sourceLineNo">374</span> requestWriteBlock(channel, storageType, writeBlockProtoBuilder);<a name="line.374"></a>
<span class="sourceLineNo">375</span> } else {<a name="line.375"></a>
<span class="sourceLineNo">376</span> promise.tryFailure(future.cause());<a name="line.376"></a>
<span class="sourceLineNo">377</span> }<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> }<a name="line.380"></a>
<span class="sourceLineNo">381</span><a name="line.381"></a>
<span class="sourceLineNo">382</span> private static List&lt;Future&lt;Channel&gt;&gt; connectToDataNodes(Configuration conf, DFSClient client,<a name="line.382"></a>
<span class="sourceLineNo">383</span> String clientName, LocatedBlock locatedBlock, long maxBytesRcvd, long latestGS,<a name="line.383"></a>
<span class="sourceLineNo">384</span> BlockConstructionStage stage, DataChecksum summer, EventLoopGroup eventLoopGroup,<a name="line.384"></a>
<span class="sourceLineNo">385</span> Class&lt;? extends Channel&gt; channelClass) {<a name="line.385"></a>
<span class="sourceLineNo">386</span> StorageType[] storageTypes = locatedBlock.getStorageTypes();<a name="line.386"></a>
<span class="sourceLineNo">387</span> DatanodeInfo[] datanodeInfos = getLocatedBlockLocations(locatedBlock);<a name="line.387"></a>
<span class="sourceLineNo">388</span> boolean connectToDnViaHostname =<a name="line.388"></a>
<span class="sourceLineNo">389</span> conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME, DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);<a name="line.389"></a>
<span class="sourceLineNo">390</span> int timeoutMs = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT);<a name="line.390"></a>
<span class="sourceLineNo">391</span> ExtendedBlock blockCopy = new ExtendedBlock(locatedBlock.getBlock());<a name="line.391"></a>
<span class="sourceLineNo">392</span> blockCopy.setNumBytes(locatedBlock.getBlockSize());<a name="line.392"></a>
<span class="sourceLineNo">393</span> ClientOperationHeaderProto header = ClientOperationHeaderProto.newBuilder()<a name="line.393"></a>
<span class="sourceLineNo">394</span> .setBaseHeader(BaseHeaderProto.newBuilder().setBlock(PBHelperClient.convert(blockCopy))<a name="line.394"></a>
<span class="sourceLineNo">395</span> .setToken(PBHelperClient.convert(locatedBlock.getBlockToken())))<a name="line.395"></a>
<span class="sourceLineNo">396</span> .setClientName(clientName).build();<a name="line.396"></a>
<span class="sourceLineNo">397</span> ChecksumProto checksumProto = DataTransferProtoUtil.toProto(summer);<a name="line.397"></a>
<span class="sourceLineNo">398</span> OpWriteBlockProto.Builder writeBlockProtoBuilder =<a name="line.398"></a>
<span class="sourceLineNo">399</span> OpWriteBlockProto.newBuilder().setHeader(header)<a name="line.399"></a>
<span class="sourceLineNo">400</span> .setStage(OpWriteBlockProto.BlockConstructionStage.valueOf(stage.name())).setPipelineSize(1)<a name="line.400"></a>
<span class="sourceLineNo">401</span> .setMinBytesRcvd(locatedBlock.getBlock().getNumBytes()).setMaxBytesRcvd(maxBytesRcvd)<a name="line.401"></a>
<span class="sourceLineNo">402</span> .setLatestGenerationStamp(latestGS).setRequestedChecksum(checksumProto)<a name="line.402"></a>
<span class="sourceLineNo">403</span> .setCachingStrategy(CachingStrategyProto.newBuilder().setDropBehind(true).build());<a name="line.403"></a>
<span class="sourceLineNo">404</span> List&lt;Future&lt;Channel&gt;&gt; futureList = new ArrayList&lt;&gt;(datanodeInfos.length);<a name="line.404"></a>
<span class="sourceLineNo">405</span> for (int i = 0; i &lt; datanodeInfos.length; i++) {<a name="line.405"></a>
<span class="sourceLineNo">406</span> DatanodeInfo dnInfo = datanodeInfos[i];<a name="line.406"></a>
<span class="sourceLineNo">407</span> StorageType storageType = storageTypes[i];<a name="line.407"></a>
<span class="sourceLineNo">408</span> Promise&lt;Channel&gt; promise = eventLoopGroup.next().newPromise();<a name="line.408"></a>
<span class="sourceLineNo">409</span> futureList.add(promise);<a name="line.409"></a>
<span class="sourceLineNo">410</span> String dnAddr = dnInfo.getXferAddr(connectToDnViaHostname);<a name="line.410"></a>
<span class="sourceLineNo">411</span> addListener(new Bootstrap().group(eventLoopGroup).channel(channelClass)<a name="line.411"></a>
<span class="sourceLineNo">412</span> .option(CONNECT_TIMEOUT_MILLIS, timeoutMs).handler(new ChannelInitializer&lt;Channel&gt;() {<a name="line.412"></a>
<span class="sourceLineNo">413</span><a name="line.413"></a>
<span class="sourceLineNo">414</span> @Override<a name="line.414"></a>
<span class="sourceLineNo">415</span> protected void initChannel(Channel ch) throws Exception {<a name="line.415"></a>
<span class="sourceLineNo">416</span> // we need to get the remote address of the channel so we can only move on after<a name="line.416"></a>
<span class="sourceLineNo">417</span> // channel connected. Leave an empty implementation here because netty does not allow<a name="line.417"></a>
<span class="sourceLineNo">418</span> // a null handler.<a name="line.418"></a>
<span class="sourceLineNo">419</span> }<a name="line.419"></a>
<span class="sourceLineNo">420</span> }).connect(NetUtils.createSocketAddr(dnAddr)), new ChannelFutureListener() {<a name="line.420"></a>
<span class="sourceLineNo">421</span><a name="line.421"></a>
<span class="sourceLineNo">422</span> @Override<a name="line.422"></a>
<span class="sourceLineNo">423</span> public void operationComplete(ChannelFuture future) throws Exception {<a name="line.423"></a>
<span class="sourceLineNo">424</span> if (future.isSuccess()) {<a name="line.424"></a>
<span class="sourceLineNo">425</span> initialize(conf, future.channel(), dnInfo, storageType, writeBlockProtoBuilder,<a name="line.425"></a>
<span class="sourceLineNo">426</span> timeoutMs, client, locatedBlock.getBlockToken(), promise);<a name="line.426"></a>
<span class="sourceLineNo">427</span> } else {<a name="line.427"></a>
<span class="sourceLineNo">428</span> promise.tryFailure(future.cause());<a name="line.428"></a>
<span class="sourceLineNo">429</span> }<a name="line.429"></a>
<span class="sourceLineNo">430</span> }<a name="line.430"></a>
<span class="sourceLineNo">431</span> });<a name="line.431"></a>
<span class="sourceLineNo">432</span> }<a name="line.432"></a>
<span class="sourceLineNo">433</span> return futureList;<a name="line.433"></a>
<span class="sourceLineNo">434</span> }<a name="line.434"></a>
<span class="sourceLineNo">435</span><a name="line.435"></a>
<span class="sourceLineNo">436</span> /**<a name="line.436"></a>
<span class="sourceLineNo">437</span> * Exception other than RemoteException thrown when calling create on namenode<a name="line.437"></a>
<span class="sourceLineNo">438</span> */<a name="line.438"></a>
<span class="sourceLineNo">439</span> public static class NameNodeException extends IOException {<a name="line.439"></a>
<span class="sourceLineNo">440</span><a name="line.440"></a>
<span class="sourceLineNo">441</span> private static final long serialVersionUID = 3143237406477095390L;<a name="line.441"></a>
<span class="sourceLineNo">442</span><a name="line.442"></a>
<span class="sourceLineNo">443</span> public NameNodeException(Throwable cause) {<a name="line.443"></a>
<span class="sourceLineNo">444</span> super(cause);<a name="line.444"></a>
<span class="sourceLineNo">445</span> }<a name="line.445"></a>
<span class="sourceLineNo">446</span> }<a name="line.446"></a>
<span class="sourceLineNo">447</span><a name="line.447"></a>
<span class="sourceLineNo">448</span> private static EnumSetWritable&lt;CreateFlag&gt; getCreateFlags(boolean overwrite,<a name="line.448"></a>
<span class="sourceLineNo">449</span> boolean noLocalWrite) {<a name="line.449"></a>
<span class="sourceLineNo">450</span> List&lt;CreateFlag&gt; flags = new ArrayList&lt;&gt;();<a name="line.450"></a>
<span class="sourceLineNo">451</span> flags.add(CreateFlag.CREATE);<a name="line.451"></a>
<span class="sourceLineNo">452</span> if (overwrite) {<a name="line.452"></a>
<span class="sourceLineNo">453</span> flags.add(CreateFlag.OVERWRITE);<a name="line.453"></a>
<span class="sourceLineNo">454</span> }<a name="line.454"></a>
<span class="sourceLineNo">455</span> if (noLocalWrite) {<a name="line.455"></a>
<span class="sourceLineNo">456</span> flags.add(CreateFlag.NO_LOCAL_WRITE);<a name="line.456"></a>
<span class="sourceLineNo">457</span> }<a name="line.457"></a>
<span class="sourceLineNo">458</span> flags.add(CreateFlag.SHOULD_REPLICATE);<a name="line.458"></a>
<span class="sourceLineNo">459</span> return new EnumSetWritable&lt;&gt;(EnumSet.copyOf(flags));<a name="line.459"></a>
<span class="sourceLineNo">460</span> }<a name="line.460"></a>
<span class="sourceLineNo">461</span><a name="line.461"></a>
<span class="sourceLineNo">462</span> private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem dfs, String src,<a name="line.462"></a>
<span class="sourceLineNo">463</span> boolean overwrite, boolean createParent, short replication, long blockSize,<a name="line.463"></a>
<span class="sourceLineNo">464</span> EventLoopGroup eventLoopGroup, Class&lt;? extends Channel&gt; channelClass, StreamSlowMonitor monitor,<a name="line.464"></a>
<span class="sourceLineNo">465</span> boolean noLocalWrite) throws IOException {<a name="line.465"></a>
<span class="sourceLineNo">466</span> Configuration conf = dfs.getConf();<a name="line.466"></a>
<span class="sourceLineNo">467</span> DFSClient client = dfs.getClient();<a name="line.467"></a>
<span class="sourceLineNo">468</span> String clientName = client.getClientName();<a name="line.468"></a>
<span class="sourceLineNo">469</span> ClientProtocol namenode = client.getNamenode();<a name="line.469"></a>
<span class="sourceLineNo">470</span> int createMaxRetries =<a name="line.470"></a>
<span class="sourceLineNo">471</span> conf.getInt(ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES, DEFAULT_ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES);<a name="line.471"></a>
<span class="sourceLineNo">472</span> ExcludeDatanodeManager excludeDatanodeManager = monitor.getExcludeDatanodeManager();<a name="line.472"></a>
<span class="sourceLineNo">473</span> Set&lt;DatanodeInfo&gt; toExcludeNodes =<a name="line.473"></a>
<span class="sourceLineNo">474</span> new HashSet&lt;&gt;(excludeDatanodeManager.getExcludeDNs().keySet());<a name="line.474"></a>
<span class="sourceLineNo">475</span> for (int retry = 0;; retry++) {<a name="line.475"></a>
<span class="sourceLineNo">476</span> LOG.debug("When create output stream for {}, exclude list is {}, retry={}", src,<a name="line.476"></a>
<span class="sourceLineNo">477</span> toExcludeNodes, retry);<a name="line.477"></a>
<span class="sourceLineNo">478</span> HdfsFileStatus stat;<a name="line.478"></a>
<span class="sourceLineNo">479</span> try {<a name="line.479"></a>
<span class="sourceLineNo">480</span> stat = FILE_CREATOR.create(namenode, src,<a name="line.480"></a>
<span class="sourceLineNo">481</span> FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,<a name="line.481"></a>
<span class="sourceLineNo">482</span> getCreateFlags(overwrite, noLocalWrite), createParent, replication, blockSize,<a name="line.482"></a>
<span class="sourceLineNo">483</span> CryptoProtocolVersion.supported());<a name="line.483"></a>
<span class="sourceLineNo">484</span> } catch (Exception e) {<a name="line.484"></a>
<span class="sourceLineNo">485</span> if (e instanceof RemoteException) {<a name="line.485"></a>
<span class="sourceLineNo">486</span> throw (RemoteException) e;<a name="line.486"></a>
<span class="sourceLineNo">487</span> } else {<a name="line.487"></a>
<span class="sourceLineNo">488</span> throw new NameNodeException(e);<a name="line.488"></a>
<span class="sourceLineNo">489</span> }<a name="line.489"></a>
<span class="sourceLineNo">490</span> }<a name="line.490"></a>
<span class="sourceLineNo">491</span> beginFileLease(client, stat.getFileId());<a name="line.491"></a>
<span class="sourceLineNo">492</span> boolean succ = false;<a name="line.492"></a>
<span class="sourceLineNo">493</span> LocatedBlock locatedBlock = null;<a name="line.493"></a>
<span class="sourceLineNo">494</span> List&lt;Future&lt;Channel&gt;&gt; futureList = null;<a name="line.494"></a>
<span class="sourceLineNo">495</span> try {<a name="line.495"></a>
<span class="sourceLineNo">496</span> DataChecksum summer = createChecksum(client);<a name="line.496"></a>
<span class="sourceLineNo">497</span> locatedBlock = namenode.addBlock(src, client.getClientName(), null,<a name="line.497"></a>
<span class="sourceLineNo">498</span> toExcludeNodes.toArray(new DatanodeInfo[0]), stat.getFileId(), null, null);<a name="line.498"></a>
<span class="sourceLineNo">499</span> Map&lt;Channel, DatanodeInfo&gt; datanodes = new IdentityHashMap&lt;&gt;();<a name="line.499"></a>
<span class="sourceLineNo">500</span> futureList = connectToDataNodes(conf, client, clientName, locatedBlock, 0L, 0L,<a name="line.500"></a>
<span class="sourceLineNo">501</span> PIPELINE_SETUP_CREATE, summer, eventLoopGroup, channelClass);<a name="line.501"></a>
<span class="sourceLineNo">502</span> for (int i = 0, n = futureList.size(); i &lt; n; i++) {<a name="line.502"></a>
<span class="sourceLineNo">503</span> DatanodeInfo datanodeInfo = getLocatedBlockLocations(locatedBlock)[i];<a name="line.503"></a>
<span class="sourceLineNo">504</span> try {<a name="line.504"></a>
<span class="sourceLineNo">505</span> datanodes.put(futureList.get(i).syncUninterruptibly().getNow(), datanodeInfo);<a name="line.505"></a>
<span class="sourceLineNo">506</span> } catch (Exception e) {<a name="line.506"></a>
<span class="sourceLineNo">507</span> // exclude the broken DN next time<a name="line.507"></a>
<span class="sourceLineNo">508</span> toExcludeNodes.add(datanodeInfo);<a name="line.508"></a>
<span class="sourceLineNo">509</span> excludeDatanodeManager.tryAddExcludeDN(datanodeInfo, "connect error");<a name="line.509"></a>
<span class="sourceLineNo">510</span> throw e;<a name="line.510"></a>
<span class="sourceLineNo">511</span> }<a name="line.511"></a>
<span class="sourceLineNo">512</span> }<a name="line.512"></a>
<span class="sourceLineNo">513</span> Encryptor encryptor = createEncryptor(conf, stat, client);<a name="line.513"></a>
<span class="sourceLineNo">514</span> FanOutOneBlockAsyncDFSOutput output =<a name="line.514"></a>
<span class="sourceLineNo">515</span> new FanOutOneBlockAsyncDFSOutput(conf, dfs, client, namenode, clientName, src,<a name="line.515"></a>
<span class="sourceLineNo">516</span> stat.getFileId(), locatedBlock, encryptor, datanodes, summer, ALLOC, monitor);<a name="line.516"></a>
<span class="sourceLineNo">517</span> succ = true;<a name="line.517"></a>
<span class="sourceLineNo">518</span> return output;<a name="line.518"></a>
<span class="sourceLineNo">519</span> } catch (RemoteException e) {<a name="line.519"></a>
<span class="sourceLineNo">520</span> LOG.warn("create fan-out dfs output {} failed, retry = {}", src, retry, e);<a name="line.520"></a>
<span class="sourceLineNo">521</span> if (shouldRetryCreate(e)) {<a name="line.521"></a>
<span class="sourceLineNo">522</span> if (retry &gt;= createMaxRetries) {<a name="line.522"></a>
<span class="sourceLineNo">523</span> throw e.unwrapRemoteException();<a name="line.523"></a>
<span class="sourceLineNo">524</span> }<a name="line.524"></a>
<span class="sourceLineNo">525</span> } else {<a name="line.525"></a>
<span class="sourceLineNo">526</span> throw e.unwrapRemoteException();<a name="line.526"></a>
<span class="sourceLineNo">527</span> }<a name="line.527"></a>
<span class="sourceLineNo">528</span> } catch (IOException e) {<a name="line.528"></a>
<span class="sourceLineNo">529</span> LOG.warn("create fan-out dfs output {} failed, retry = {}", src, retry, e);<a name="line.529"></a>
<span class="sourceLineNo">530</span> if (retry &gt;= createMaxRetries) {<a name="line.530"></a>
<span class="sourceLineNo">531</span> throw e;<a name="line.531"></a>
<span class="sourceLineNo">532</span> }<a name="line.532"></a>
<span class="sourceLineNo">533</span> // overwrite the old broken file.<a name="line.533"></a>
<span class="sourceLineNo">534</span> overwrite = true;<a name="line.534"></a>
<span class="sourceLineNo">535</span> try {<a name="line.535"></a>
<span class="sourceLineNo">536</span> Thread.sleep(ConnectionUtils.getPauseTime(100, retry));<a name="line.536"></a>
<span class="sourceLineNo">537</span> } catch (InterruptedException ie) {<a name="line.537"></a>
<span class="sourceLineNo">538</span> throw new InterruptedIOException();<a name="line.538"></a>
<span class="sourceLineNo">539</span> }<a name="line.539"></a>
<span class="sourceLineNo">540</span> } finally {<a name="line.540"></a>
<span class="sourceLineNo">541</span> if (!succ) {<a name="line.541"></a>
<span class="sourceLineNo">542</span> if (futureList != null) {<a name="line.542"></a>
<span class="sourceLineNo">543</span> for (Future&lt;Channel&gt; f : futureList) {<a name="line.543"></a>
<span class="sourceLineNo">544</span> addListener(f, new FutureListener&lt;Channel&gt;() {<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 void operationComplete(Future&lt;Channel&gt; future) throws Exception {<a name="line.547"></a>
<span class="sourceLineNo">548</span> if (future.isSuccess()) {<a name="line.548"></a>
<span class="sourceLineNo">549</span> safeClose(future.getNow());<a name="line.549"></a>
<span class="sourceLineNo">550</span> }<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> }<a name="line.553"></a>
<span class="sourceLineNo">554</span> }<a name="line.554"></a>
<span class="sourceLineNo">555</span> endFileLease(client, stat.getFileId());<a name="line.555"></a>
<span class="sourceLineNo">556</span> }<a name="line.556"></a>
<span class="sourceLineNo">557</span> }<a name="line.557"></a>
<span class="sourceLineNo">558</span> }<a name="line.558"></a>
<span class="sourceLineNo">559</span> }<a name="line.559"></a>
<span class="sourceLineNo">560</span><a name="line.560"></a>
<span class="sourceLineNo">561</span> /**<a name="line.561"></a>
<span class="sourceLineNo">562</span> * Create a {@link FanOutOneBlockAsyncDFSOutput}. The method maybe blocked so do not call it<a name="line.562"></a>
<span class="sourceLineNo">563</span> * inside an {@link EventLoop}.<a name="line.563"></a>
<span class="sourceLineNo">564</span> */<a name="line.564"></a>
<span class="sourceLineNo">565</span> public static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem dfs, Path f,<a name="line.565"></a>
<span class="sourceLineNo">566</span> boolean overwrite, boolean createParent, short replication, long blockSize,<a name="line.566"></a>
<span class="sourceLineNo">567</span> EventLoopGroup eventLoopGroup, Class&lt;? extends Channel&gt; channelClass,<a name="line.567"></a>
<span class="sourceLineNo">568</span> final StreamSlowMonitor monitor, boolean noLocalWrite) throws IOException {<a name="line.568"></a>
<span class="sourceLineNo">569</span> return new FileSystemLinkResolver&lt;FanOutOneBlockAsyncDFSOutput&gt;() {<a name="line.569"></a>
<span class="sourceLineNo">570</span><a name="line.570"></a>
<span class="sourceLineNo">571</span> @Override<a name="line.571"></a>
<span class="sourceLineNo">572</span> public FanOutOneBlockAsyncDFSOutput doCall(Path p)<a name="line.572"></a>
<span class="sourceLineNo">573</span> throws IOException, UnresolvedLinkException {<a name="line.573"></a>
<span class="sourceLineNo">574</span> return createOutput(dfs, p.toUri().getPath(), overwrite, createParent, replication,<a name="line.574"></a>
<span class="sourceLineNo">575</span> blockSize, eventLoopGroup, channelClass, monitor, noLocalWrite);<a name="line.575"></a>
<span class="sourceLineNo">576</span> }<a name="line.576"></a>
<span class="sourceLineNo">577</span><a name="line.577"></a>
<span class="sourceLineNo">578</span> @Override<a name="line.578"></a>
<span class="sourceLineNo">579</span> public FanOutOneBlockAsyncDFSOutput next(FileSystem fs, Path p) throws IOException {<a name="line.579"></a>
<span class="sourceLineNo">580</span> throw new UnsupportedOperationException();<a name="line.580"></a>
<span class="sourceLineNo">581</span> }<a name="line.581"></a>
<span class="sourceLineNo">582</span> }.resolve(dfs, f);<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> public static boolean shouldRetryCreate(RemoteException e) {<a name="line.585"></a>
<span class="sourceLineNo">586</span> // RetryStartFileException is introduced in HDFS 2.6+, so here we can only use the class name.<a name="line.586"></a>
<span class="sourceLineNo">587</span> // For exceptions other than this, we just throw it out. This is same with<a name="line.587"></a>
<span class="sourceLineNo">588</span> // DFSOutputStream.newStreamForCreate.<a name="line.588"></a>
<span class="sourceLineNo">589</span> return e.getClassName().endsWith("RetryStartFileException");<a name="line.589"></a>
<span class="sourceLineNo">590</span> }<a name="line.590"></a>
<span class="sourceLineNo">591</span><a name="line.591"></a>
<span class="sourceLineNo">592</span> static void completeFile(DFSClient client, ClientProtocol namenode, String src, String clientName,<a name="line.592"></a>
<span class="sourceLineNo">593</span> ExtendedBlock block, long fileId) {<a name="line.593"></a>
<span class="sourceLineNo">594</span> for (int retry = 0;; retry++) {<a name="line.594"></a>
<span class="sourceLineNo">595</span> try {<a name="line.595"></a>
<span class="sourceLineNo">596</span> if (namenode.complete(src, clientName, block, fileId)) {<a name="line.596"></a>
<span class="sourceLineNo">597</span> endFileLease(client, fileId);<a name="line.597"></a>
<span class="sourceLineNo">598</span> return;<a name="line.598"></a>
<span class="sourceLineNo">599</span> } else {<a name="line.599"></a>
<span class="sourceLineNo">600</span> LOG.warn("complete file " + src + " not finished, retry = " + retry);<a name="line.600"></a>
<span class="sourceLineNo">601</span> }<a name="line.601"></a>
<span class="sourceLineNo">602</span> } catch (RemoteException e) {<a name="line.602"></a>
<span class="sourceLineNo">603</span> IOException ioe = e.unwrapRemoteException();<a name="line.603"></a>
<span class="sourceLineNo">604</span> if (ioe instanceof LeaseExpiredException) {<a name="line.604"></a>
<span class="sourceLineNo">605</span> LOG.warn("lease for file " + src + " is expired, give up", e);<a name="line.605"></a>
<span class="sourceLineNo">606</span> return;<a name="line.606"></a>
<span class="sourceLineNo">607</span> } else {<a name="line.607"></a>
<span class="sourceLineNo">608</span> LOG.warn("complete file " + src + " failed, retry = " + retry, e);<a name="line.608"></a>
<span class="sourceLineNo">609</span> }<a name="line.609"></a>
<span class="sourceLineNo">610</span> } catch (Exception e) {<a name="line.610"></a>
<span class="sourceLineNo">611</span> LOG.warn("complete file " + src + " failed, retry = " + retry, e);<a name="line.611"></a>
<span class="sourceLineNo">612</span> }<a name="line.612"></a>
<span class="sourceLineNo">613</span> sleepIgnoreInterrupt(retry);<a name="line.613"></a>
<span class="sourceLineNo">614</span> }<a name="line.614"></a>
<span class="sourceLineNo">615</span> }<a name="line.615"></a>
<span class="sourceLineNo">616</span><a name="line.616"></a>
<span class="sourceLineNo">617</span> static void sleepIgnoreInterrupt(int retry) {<a name="line.617"></a>
<span class="sourceLineNo">618</span> try {<a name="line.618"></a>
<span class="sourceLineNo">619</span> Thread.sleep(ConnectionUtils.getPauseTime(100, retry));<a name="line.619"></a>
<span class="sourceLineNo">620</span> } catch (InterruptedException e) {<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>}<a name="line.623"></a>
</pre>
</div>
</body>
</html>