Github user StephanEwen commented on a diff in the pull request:
https://github.com/apache/incubator-flink/pull/254#discussion_r21539247
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java
---
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.netty;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import io.netty.handler.codec.MessageToMessageDecoder;
+import org.apache.flink.runtime.event.task.TaskEvent;
+import
org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import
org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * A simple and generic interface to serialize messages to Netty's buffer
space.
+ */
+interface NettyMessage {
+
+ //
------------------------------------------------------------------------
+ // Note: Every NettyMessage subtype needs to have a public 0-argument
+ // constructor in order to work with the generic deserializer.
+ //
------------------------------------------------------------------------
+
+ byte getId();
+
+ int getLength();
+
+ void writeTo(ByteBuf outboundBuffer);
+
+ void readFrom(ByteBuf inboundBuffer) throws Exception;
+
+ interface ClientRequest extends NettyMessage {
+
+ InputChannelID getReceiverId();
+
+ }
+
+ //
------------------------------------------------------------------------
+ // Generic NettyMessage encoder and decoder
+ //
------------------------------------------------------------------------
+
+ @ChannelHandler.Sharable
+ static class NettyMessageEncoder extends ChannelOutboundHandlerAdapter {
+
+ static final int HEADER_LENGTH = 4 + 4 + 1; // frame length
(4), magic number (4), msg ID (1)
+
+ static final int MAGIC_NUMBER = 0xBADC0FFE;
+
+ @Override
+ public void write(ChannelHandlerContext ctx, Object msg,
ChannelPromise promise) throws Exception {
+ if (msg instanceof NettyMessage) {
+ try {
+ NettyMessage nettyMsg = (NettyMessage)
msg;
+ // frame length (4), magic number (4),
message id (1), message (var length)
+ int frameLength = 4 + 4 + 1 +
nettyMsg.getLength();
+
+ final ByteBuf outboundBuffer =
ctx.alloc().directBuffer(frameLength);
+
+ outboundBuffer.writeInt(frameLength);
+ outboundBuffer.writeInt(MAGIC_NUMBER);
+
outboundBuffer.writeByte(nettyMsg.getId());
+ nettyMsg.writeTo(outboundBuffer);
+
+ ctx.write(outboundBuffer, promise);
+ }
+ catch (Throwable t) {
+ throw new IOException("Error while
serializing message: " + msg, t);
+ }
+ }
+ else {
+ ctx.write(msg, promise);
+ }
+ }
+
+ // Create the frame length decoder here as it depends on the
encoder
+ static LengthFieldBasedFrameDecoder createFrameLengthDecoder() {
+ return new
LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, -4, 4);
+ }
+ }
+
+ @ChannelHandler.Sharable
+ static class NettyMessageDecoder extends
MessageToMessageDecoder<ByteBuf> {
+
+ @Override
+ protected void decode(ChannelHandlerContext ctx, ByteBuf msg,
List<Object> out) throws Exception {
+ int magicNumber = msg.readInt();
+
+ if (magicNumber != NettyMessageEncoder.MAGIC_NUMBER) {
+ throw new IllegalStateException("Network stream
corrupted: received incorrect magic number.");
+ }
+
+ byte msgId = msg.readByte();
+
+ NettyMessage decodedMsg = null;
+
+ if (msgId == BufferResponse.ID) {
+ decodedMsg = new BufferResponse();
+ }
+ else if (msgId == PartitionRequest.ID) {
+ decodedMsg = new PartitionRequest();
+ }
+ else if (msgId == TaskEventRequest.ID) {
+ decodedMsg = new TaskEventRequest();
+ }
+ else if (msgId == ErrorResponse.ID) {
+ decodedMsg = new ErrorResponse();
+ }
+ else {
+ // Exception: unknown message
--- End diff --
Really throw an exception here? ;-)
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---