/** Provides a canonical set of Encoders for simple types. */publicclassEncoders {/** Strings are encoded with their length followed by UTF-8 bytes. */publicstaticclassStrings {publicstaticintencodedLength(String s) {
return4 + s.getBytes(StandardCharsets.UTF_8).length;
}
publicstaticvoidencode(ByteBuf buf, String s) {
byte[] bytes = s.getBytes(StandardCharsets.UTF_8);
buf.writeInt(bytes.length);
buf.writeBytes(bytes);
}
publicstatic String decode(ByteBuf buf) {
int length = buf.readInt();
byte[] bytes = newbyte[length];
buf.readBytes(bytes);
returnnew String(bytes, StandardCharsets.UTF_8);
}
}
/** Byte arrays are encoded with their length followed by bytes. */publicstaticclassByteArrays {publicstaticintencodedLength(byte[] arr) {
return4 + arr.length;
}
publicstaticvoidencode(ByteBuf buf, byte[] arr) {
buf.writeInt(arr.length);
buf.writeBytes(arr);
}
publicstaticbyte[] decode(ByteBuf buf) {
int length = buf.readInt();
byte[] bytes = newbyte[length];
buf.readBytes(bytes);
return bytes;
}
}
/** String arrays are encoded with the number of strings followed by per-String encoding. */publicstaticclassStringArrays {publicstaticintencodedLength(String[] strings) {
int totalLength = 4;
for (String s : strings) {
totalLength += Strings.encodedLength(s);
}
return totalLength;
}
publicstaticvoidencode(ByteBuf buf, String[] strings) {
buf.writeInt(strings.length);
for (String s : strings) {
Strings.encode(buf, s);
}
}
publicstatic String[] decode(ByteBuf buf) {
int numStrings = buf.readInt();
String[] strings = new String[numStrings];
for (int i = 0; i < strings.length; i ++) {
strings[i] = Strings.decode(buf);
}
return strings;
}
}
}
Encodable
/**
* Interface for an object which can be encoded into a ByteBuf. Multiple Encodable objects are
* stored in a single, pre-allocated ByteBuf, so Encodables must also provide their length.
*
* Encodable objects should provide a static "decode(ByteBuf)" method which is invoked by
* {@link MessageDecoder}. During decoding, if the object uses the ByteBuf as its data (rather than
* just copying data from it), then you must retain() the ByteBuf.
*
* Additionally, when adding a new Encodable Message, add it to {@link Message.Type}.
*/publicinterfaceEncodable {/** Number of bytes of the encoded form of this object. */int encodedLength();
/**
* Serializes this object by writing into the given ByteBuf.
* This method must write exactly encodedLength() bytes.
*/void encode(ByteBuf buf);
}
BlockTransferMessage
/**
* Messages handled by the {@link org.apache.spark.network.shuffle.ExternalShuffleBlockHandler}, or
* by Spark's NettyBlockTransferService.
*
* At a high level:
* - OpenBlock is handled by both services, but only services shuffle files for the external
* shuffle service. It returns a StreamHandle.
* - UploadBlock is only handled by the NettyBlockTransferService.
* - RegisterExecutor is only handled by the external shuffle service.
*/publicabstractclassBlockTransferMessageimplementsEncodable {protectedabstract Type type();
/** Preceding every serialized message is its type, which allows us to deserialize it. */publicenum Type {
OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3), REGISTER_DRIVER(4),
HEARTBEAT(5);
privatefinalbyte id;
Type(int id) {
assert id < 128 : "Cannot have more than 128 message types";
this.id = (byte) id;
}
publicbyteid() { return id; }
}
// NB: Java does not support static methods in interfaces, so we must put this in a static class.publicstaticclassDecoder {/** Deserializes the 'type' byte followed by the message itself. */publicstatic BlockTransferMessage fromByteBuffer(ByteBuffer msg) {
ByteBuf buf = Unpooled.wrappedBuffer(msg);
byte type = buf.readByte();
switch (type) {
case0: return OpenBlocks.decode(buf);
case1: return UploadBlock.decode(buf);
case2: return RegisterExecutor.decode(buf);
case3: return StreamHandle.decode(buf);
case4: return RegisterDriver.decode(buf);
case5: return ShuffleServiceHeartbeat.decode(buf);
default: thrownew IllegalArgumentException("Unknown message type: " + type);
}
}
}
/** Serializes the 'type' byte followed by the message itself. */public ByteBuffer toByteBuffer() {
// Allow room for encoded message, plus the type byte
ByteBuf buf = Unpooled.buffer(encodedLength() + 1);
buf.writeByte(type().id);
encode(buf);
assert buf.writableBytes() == 0 : "Writable bytes remain: " + buf.writableBytes();
return buf.nioBuffer();
}
}
UploadBlock
/** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */publicclassUploadBlockextendsBlockTransferMessage {publicfinal String appId;
publicfinal String execId;
publicfinal String blockId;
// TODO: StorageLevel is serialized separately in here because StorageLevel is not available in// this package. We should avoid this hack.publicfinalbyte[] metadata;
publicfinalbyte[] blockData;
/**
* @param metadata Meta-information about block, typically StorageLevel.
* @param blockData The actual block's bytes.
*/publicUploadBlock(
String appId,
String execId,
String blockId,
byte[] metadata,
byte[] blockData) {
this.appId = appId;
this.execId = execId;
this.blockId = blockId;
this.metadata = metadata;
this.blockData = blockData;
}
@Overrideprotected Type type() { return Type.UPLOAD_BLOCK; }
@OverridepublicinthashCode() {
int objectsHashCode = Objects.hashCode(appId, execId, blockId);
return (objectsHashCode * 41 + Arrays.hashCode(metadata)) * 41 + Arrays.hashCode(blockData);
}
@Overridepublic String toString() {
return Objects.toStringHelper(this)
.add("appId", appId)
.add("execId", execId)
.add("blockId", blockId)
.add("metadata size", metadata.length)
.add("block size", blockData.length)
.toString();
}
@Overridepublicbooleanequals(Object other) {
if (other != null && other instanceof UploadBlock) {
UploadBlock o = (UploadBlock) other;
return Objects.equal(appId, o.appId)
&& Objects.equal(execId, o.execId)
&& Objects.equal(blockId, o.blockId)
&& Arrays.equals(metadata, o.metadata)
&& Arrays.equals(blockData, o.blockData);
}
returnfalse;
}
@OverridepublicintencodedLength() {
return Encoders.Strings.encodedLength(appId)
+ Encoders.Strings.encodedLength(execId)
+ Encoders.Strings.encodedLength(blockId)
+ Encoders.ByteArrays.encodedLength(metadata)
+ Encoders.ByteArrays.encodedLength(blockData);
}
@Overridepublicvoidencode(ByteBuf buf) {
Encoders.Strings.encode(buf, appId);
Encoders.Strings.encode(buf, execId);
Encoders.Strings.encode(buf, blockId);
Encoders.ByteArrays.encode(buf, metadata);
Encoders.ByteArrays.encode(buf, blockData);
}
publicstatic UploadBlock decode(ByteBuf buf) {
String appId = Encoders.Strings.decode(buf);
String execId = Encoders.Strings.decode(buf);
String blockId = Encoders.Strings.decode(buf);
byte[] metadata = Encoders.ByteArrays.decode(buf);
byte[] blockData = Encoders.ByteArrays.decode(buf);
returnnew UploadBlock(appId, execId, blockId, metadata, blockData);
}
}