private void check(DataTransferEncryptorMessageProto proto) throws IOException { if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) { dfsClient.clearDataEncryptionKey(); throw new InvalidEncryptionKeyException(proto.getMessage()); } else if (proto.getStatus() == DataTransferEncryptorStatus.ERROR) { throw new IOException(proto.getMessage()); } }
@Override public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { if (evt instanceof IdleStateEvent) { IdleStateEvent e = (IdleStateEvent) evt; if (e.state() == READER_IDLE) { failed(ctx.channel(), () -> new IOException("Timeout(" + timeoutMs + "ms) waiting for response")); } else if (e.state() == WRITER_IDLE) { PacketHeader heartbeat = new PacketHeader(4, 0, HEART_BEAT_SEQNO, false, 0, false); int len = heartbeat.getSerializedSize(); ByteBuf buf = alloc.buffer(len); heartbeat.putInBuffer(buf.nioBuffer(0, len)); buf.writerIndex(len); ctx.channel().writeAndFlush(buf); } return; } super.userEventTriggered(ctx, evt); } }
Sender createSender(IOStreamPair pair) { DataOutputStream out = (DataOutputStream) pair.out; return new Sender(out); }
/** Send a reportedBlock replace request to the output stream. */ private static void sendRequest(DataOutputStream out, ExtendedBlock eb, Token<BlockTokenIdentifier> accessToken, DatanodeInfo source, StorageType targetStorageType) throws IOException { new Sender(out).replaceBlock(eb, targetStorageType, accessToken, source.getDatanodeUuid(), source, null); }
.setToken(PB_HELPER.convert(locatedBlock.getBlockToken()))) .setClientName(clientName).build(); ChecksumProto checksumProto = DataTransferProtoUtil.toProto(summer); OpWriteBlockProto.Builder writeBlockProtoBuilder = OpWriteBlockProto.newBuilder() .setHeader(header).setStage(OpWriteBlockProto.BlockConstructionStage.valueOf(stage.name())) .setPipelineSize(1).setMinBytesRcvd(locatedBlock.getBlock().getNumBytes()) .setMaxBytesRcvd(maxBytesRcvd).setLatestGenerationStamp(latestGS)
@Override protected void channelRead0(ChannelHandlerContext ctx, PipelineAckProto ack) throws Exception { Status reply = getStatus(ack); if (reply != Status.SUCCESS) { failed(ctx.channel(), () -> new IOException("Bad response " + reply + " for block " + block + " from datanode " + ctx.channel().remoteAddress())); return; } if (PipelineAck.isRestartOOBStatus(reply)) { failed(ctx.channel(), () -> new IOException("Restart response " + reply + " for block " + block + " from datanode " + ctx.channel().remoteAddress())); return; } if (ack.getSeqno() == HEART_BEAT_SEQNO) { return; } completed(ctx.channel()); }
private TraceScope continueTraceSpan(DataTransferTraceInfoProto proto, String description) { TraceScope scope = null; SpanId spanId = fromProto(proto); if (spanId != null) { scope = tracer.newScope(description, spanId); } return scope; }
/** Receive a reportedBlock copy response from the input stream. */ private static void receiveResponse(DataInputStream in) throws IOException { BlockOpResponseProto response = BlockOpResponseProto .parseFrom(vintPrefixed(in)); while (response.getStatus() == Status.IN_PROGRESS) { // read intermediate responses response = BlockOpResponseProto.parseFrom(vintPrefixed(in)); } String logInfo = "reportedBlock move is failed"; DataTransferProtoUtil.checkBlockOpStatus(response, logInfo); } }
/** Read an Op. It also checks protocol version. */ protected final Op readOp() throws IOException { final short version = in.readShort(); if (version != DataTransferProtocol.DATA_TRANSFER_VERSION) { throw new IOException( "Version Mismatch (Expected: " + DataTransferProtocol.DATA_TRANSFER_VERSION + ", Received: " + version + " )"); } return Op.read(in); }
@Override public void setConf(Configuration conf) { super.setConf(conf); String fixedFile = conf.get(DFS_DATATRANSFER_SERVER_FIXEDWHITELIST_FILE, FIXEDWHITELIST_DEFAULT_LOCATION); String variableFile = null; long expiryTime = 0; if (conf.getBoolean(DFS_DATATRANSFER_SERVER_VARIABLEWHITELIST_ENABLE, false)) { variableFile = conf.get(DFS_DATATRANSFER_SERVER_VARIABLEWHITELIST_FILE, VARIABLEWHITELIST_DEFAULT_LOCATION); expiryTime = conf.getLong(DFS_DATATRANSFER_SERVER_VARIABLEWHITELIST_CACHE_SECS,3600) * 1000; } whiteListForServer = new CombinedIPWhiteList(fixedFile,variableFile,expiryTime); fixedFile = conf.get(DFS_DATATRANSFER_CLIENT_FIXEDWHITELIST_FILE, fixedFile); expiryTime = 0; if (conf.getBoolean(DFS_DATATRANSFER_CLIENT_VARIABLEWHITELIST_ENABLE, false)) { variableFile = conf.get(DFS_DATATRANSFER_CLIENT_VARIABLEWHITELIST_FILE,variableFile); expiryTime = conf.getLong(DFS_DATATRANSFER_CLIENT_VARIABLEWHITELIST_CACHE_SECS,3600) * 1000; } whitelistForClient = new CombinedIPWhiteList(fixedFile,variableFile,expiryTime); }
public void sendOOB() throws IOException, InterruptedException { if (isDatanode) { return; } ((PacketResponder) responder.getRunnable()).sendOOBResponse(PipelineAck .getRestartOOBStatus()); }
private TraceScope continueTraceSpan(BaseHeaderProto header, String description) { return continueTraceSpan(header.getTraceInfo(), description); }
private void endBlock() throws IOException { Preconditions.checkState(waitingAckQueue.isEmpty(), "should call flush first before calling close"); if (state != State.STREAMING) { throw new IOException("stream already broken"); } state = State.CLOSING; long finalizedLength = ackedBlockLength; PacketHeader header = new PacketHeader(4, finalizedLength, nextPacketSeqno, true, 0, false); buf.release(); buf = null; int headerLen = header.getSerializedSize(); ByteBuf headerBuf = alloc.directBuffer(headerLen); header.putInBuffer(headerBuf.nioBuffer(0, headerLen)); headerBuf.writerIndex(headerLen); CompletableFuture<Long> future = new CompletableFuture<>(); waitingAckQueue.add(new Callback(future, finalizedLength, datanodeList)); datanodeList.forEach(ch -> ch.writeAndFlush(headerBuf.retainedDuplicate())); headerBuf.release(); try { future.get(); } catch (InterruptedException e) { throw (IOException) new InterruptedIOException().initCause(e); } catch (ExecutionException e) { Throwable cause = e.getCause(); Throwables.propagateIfPossible(cause, IOException.class); throw new IOException(cause); } }
/** Send a reportedBlock replace request to the output stream */ private void sendRequest(DataOutputStream out, ExtendedBlock eb, Token<BlockTokenIdentifier> accessToken) throws IOException { new Sender(out).replaceBlock(eb, target.storageType, accessToken, source.getDatanodeInfo().getDatanodeUuid(), proxySource.datanode, null); }
Sender createSender(IOStreamPair pair) { DataOutputStream out = (DataOutputStream) pair.out; return new Sender(out); }
throws Exception { Status pipelineStatus = resp.getStatus(); if (PipelineAck.isRestartOOBStatus(pipelineStatus)) { throw new IOException("datanode " + dnInfo + " is restarting");
private TraceScope continueTraceSpan(ClientOperationHeaderProto header, String description) { return continueTraceSpan(header.getBaseHeader(), description); }
/** * Write packet header into {@code pkt}, * return the length of the header written. */ private int writePacketHeader(ByteBuffer pkt, int dataLen, int packetLen) { pkt.clear(); // both syncBlock and syncPacket are false PacketHeader header = new PacketHeader(packetLen, offset, seqno, (dataLen == 0), dataLen, false); int size = header.getSerializedSize(); pkt.position(PacketHeader.PKT_MAX_HEADER_LEN - size); header.putInBuffer(pkt); return size; }
summer.calculateChunkedSums(dataBuf.nioBuffer(), checksumBuf.nioBuffer(0, checksumLen)); checksumBuf.writerIndex(checksumLen); PacketHeader header = new PacketHeader(4 + checksumLen + dataLen, nextPacketOffsetInBlock, nextPacketSeqno, false, dataLen, syncBlock); int headerLen = header.getSerializedSize(); ByteBuf headerBuf = alloc.buffer(headerLen); header.putInBuffer(headerBuf.nioBuffer(0, headerLen)); headerBuf.writerIndex(headerLen); Callback c = new Callback(future, nextPacketOffsetInBlock + dataLen, datanodeList);