Return-Path: X-Original-To: apmail-flink-issues-archive@minotaur.apache.org Delivered-To: apmail-flink-issues-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 334C59048 for ; Tue, 9 Dec 2014 20:30:38 +0000 (UTC) Received: (qmail 31745 invoked by uid 500); 9 Dec 2014 20:30:38 -0000 Delivered-To: apmail-flink-issues-archive@flink.apache.org Received: (qmail 31705 invoked by uid 500); 9 Dec 2014 20:30:38 -0000 Mailing-List: contact issues-help@flink.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.incubator.apache.org Delivered-To: mailing list issues@flink.incubator.apache.org Received: (qmail 31696 invoked by uid 99); 9 Dec 2014 20:30:38 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 09 Dec 2014 20:30:38 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Tue, 09 Dec 2014 20:30:15 +0000 Received: (qmail 29936 invoked by uid 99); 9 Dec 2014 20:30:13 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 09 Dec 2014 20:30:13 +0000 Date: Tue, 9 Dec 2014 20:30:12 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.incubator.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-986) Add intermediate results to distributed runtime MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 X-Virus-Checked: Checked by ClamAV on apache.org [ https://issues.apache.org/jira/browse/FLINK-986?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14240009#comment-14240009 ] ASF GitHub Bot commented on FLINK-986: -------------------------------------- Github user uce commented on a diff in the pull request: https://github.com/apache/incubator-flink/pull/254#discussion_r21559670 --- 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 { + + @Override + protected void decode(ChannelHandlerContext ctx, ByteBuf msg, List 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 -- Yes ;) > Add intermediate results to distributed runtime > ----------------------------------------------- > > Key: FLINK-986 > URL: https://issues.apache.org/jira/browse/FLINK-986 > Project: Flink > Issue Type: New Feature > Components: Distributed Runtime > Reporter: Ufuk Celebi > Assignee: Ufuk Celebi > Priority: Blocker > > Support for intermediate results in the runtime is currently blocking different efforts like fault tolerance or result collection at the client. -- This message was sent by Atlassian JIRA (v6.3.4#6332)