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); } }
@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); } }
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();
/** * 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; }
/** * 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; }
/** * 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; }
private byte[] prepareFakePacket(byte[] data, byte[] sums) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(baos); int packetLen = data.length + sums.length + Ints.BYTES; PacketHeader header = new PacketHeader( packetLen, OFFSET_IN_BLOCK, SEQNO, false, data.length, false); header.write(dos); dos.write(sums); dos.write(data); dos.flush(); return baos.toByteArray(); }
curPacketBuf.get(headerBuf); if (curHeader == null) { curHeader = new PacketHeader();
curPacketBuf.get(headerBuf); if (curHeader == null) { curHeader = new PacketHeader();
curPacketBuf.get(headerBuf); if (curHeader == null) { curHeader = new PacketHeader();
PacketHeader header = new PacketHeader(); header.readFields(in); PacketHeader hdr = new PacketHeader(); hdr.readFields(in);
PacketHeader header = new PacketHeader(); header.readFields(in); PacketHeader hdr = new PacketHeader(); hdr.readFields(in);
@Test public void testPacketHeader() throws IOException { PacketHeader hdr = new PacketHeader( 4, // size of packet 1024, // OffsetInBlock 100, // sequencenumber false, // lastPacketInBlock 4096, // chunk length false); ByteArrayOutputStream baos = new ByteArrayOutputStream(); hdr.write(new DataOutputStream(baos)); // Read back using DataInput PacketHeader readBack = new PacketHeader(); ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); readBack.readFields(new DataInputStream(bais)); assertEquals(hdr, readBack); // Read back using ByteBuffer readBack = new PacketHeader(); readBack.readFields(ByteBuffer.wrap(baos.toByteArray())); assertEquals(hdr, readBack); assertTrue(hdr.sanityCheck(99)); assertFalse(hdr.sanityCheck(100)); }
final int pktLen = HdfsConstants.BYTES_IN_INTEGER + dataLen + checksumLen; PacketHeader header = new PacketHeader( pktLen, offsetInBlock, seqno, lastPacketInBlock, dataLen, syncBlock);
final int pktLen = HdfsConstants.BYTES_IN_INTEGER + dataLen + checksumLen; PacketHeader header = new PacketHeader( pktLen, offsetInBlock, seqno, lastPacketInBlock, dataLen, syncBlock);
final int pktLen = HdfsConstants.BYTES_IN_INTEGER + dataLen + checksumLen; PacketHeader header = new PacketHeader( pktLen, offsetInBlock, seqno, lastPacketInBlock, dataLen, syncBlock);
private void writeZeroLengthPacket(ExtendedBlock block, String description) throws IOException { PacketHeader hdr = new PacketHeader( 8, // size of packet block.getNumBytes(), // OffsetInBlock 100, // sequencenumber true, // lastPacketInBlock 0, // chunk length false); // sync block hdr.write(sendOut); sendOut.writeInt(0); // zero checksum //ok finally write a block with 0 len sendResponse(Status.SUCCESS, "", null, recvOut); new PipelineAck(100, new int[] {PipelineAck.combineHeader (PipelineAck.ECN.DISABLED, Status.SUCCESS)}).write (recvOut); sendRecvData(description, false); }
writeBlock(poolId, ++newBlockId, DEFAULT_CHECKSUM); PacketHeader hdr = new PacketHeader( 4, // size of packet 0, // offset in block, writeBlock(poolId, ++newBlockId, DEFAULT_CHECKSUM); hdr = new PacketHeader( 8, // size of packet 0, // OffsetInBlock