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