[
https://issues.apache.org/jira/browse/FLINK-7411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16152655#comment-16152655
]
ASF GitHub Bot commented on FLINK-7411:
---------------------------------------
Github user pnowojski commented on a diff in the pull request:
https://github.com/apache/flink/pull/4517#discussion_r136830945
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java
---
@@ -242,21 +239,22 @@ ByteBuf write(ByteBufAllocator allocator) throws
IOException {
throw new IOException(t);
}
finally {
- if (buffer != null) {
- buffer.recycle();
- }
+ buffer.recycle();
}
}
- @Override
- void readFrom(ByteBuf buffer) {
- receiverId = InputChannelID.fromByteBuf(buffer);
- sequenceNumber = buffer.readInt();
- isBuffer = buffer.readBoolean();
- size = buffer.readInt();
-
- retainedSlice = buffer.readSlice(size);
- retainedSlice.retain();
+ static BufferResponse readFrom(ByteBuf buffer) {
+ BufferResponse result = new BufferResponse();
--- End diff --
this is somehow strange construct, with this default empty constructor and
manually modifying fields afterwards. Why not:
```
InputChannelID receiverId = InputChannelID.fromByteBuf(buffer);
int sequenceNumber = buffer.readInt();
boolean isBuffer = buffer.readBoolean();
int size = buffer.readInt();
return new BufferResponse(
receivedId,
sequenceNumber,
isBuffer,
size,
buffer.readSlice(size).retain());
```
would be more natural, would be easier for the finding usages of the fields
and that way you could make all of the fields `final`, which is nice feature on
its own.
> minor performance improvements in NettyMessage
> ----------------------------------------------
>
> Key: FLINK-7411
> URL: https://issues.apache.org/jira/browse/FLINK-7411
> Project: Flink
> Issue Type: Sub-task
> Components: Network
> Affects Versions: 1.4.0
> Reporter: Nico Kruber
> Assignee: Nico Kruber
> Priority: Minor
>
> {{NettyMessage}} may be improved slightly performance-wise in these regards:
> - in {{NettyMessage.NettyMessageDecoder#decode()}}: instead of having
> multiple if-elseif-... use a switch to cycle through the message ID
> - use a static {{NettyMessage}} subtype {{readFrom(ByteBuf buffer)}} - we do
> not really need to have a virtual function here
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)