|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.network.shuffle; |
| 19 | + |
| 20 | +import java.nio.ByteBuffer; |
| 21 | +import java.util.List; |
| 22 | + |
| 23 | +import io.netty.buffer.ByteBuf; |
| 24 | +import io.netty.buffer.Unpooled; |
| 25 | +import io.netty.channel.ChannelHandlerContext; |
| 26 | +import io.netty.channel.EventLoopGroup; |
| 27 | +import io.netty.channel.SimpleChannelInboundHandler; |
| 28 | +import io.netty.channel.socket.SocketChannel; |
| 29 | +import io.netty.handler.codec.MessageToMessageDecoder; |
| 30 | + |
| 31 | +import org.slf4j.Logger; |
| 32 | +import org.slf4j.LoggerFactory; |
| 33 | + |
| 34 | +import org.apache.spark.network.TransportContext; |
| 35 | +import org.apache.spark.network.protocol.Message; |
| 36 | +import org.apache.spark.network.protocol.MessageDecoder; |
| 37 | +import org.apache.spark.network.protocol.RpcRequest; |
| 38 | +import org.apache.spark.network.server.RpcHandler; |
| 39 | +import org.apache.spark.network.server.TransportChannelHandler; |
| 40 | +import org.apache.spark.network.server.TransportRequestHandler; |
| 41 | +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; |
| 42 | +import org.apache.spark.network.util.IOMode; |
| 43 | +import org.apache.spark.network.util.NettyUtils; |
| 44 | +import org.apache.spark.network.util.TransportConf; |
| 45 | + |
| 46 | +import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; |
| 47 | + |
| 48 | +/** |
| 49 | + * Extends {@link TransportContext} to support customized shuffle service. Specifically, we |
| 50 | + * modified the Netty Channel Pipeline so that IO expensive messages such as FINALIZE_SHUFFLE_MERGE |
| 51 | + * are processed in the separate handlers. |
| 52 | + * */ |
| 53 | +public class ShuffleTransportContext extends TransportContext { |
| 54 | + private static final Logger logger = LoggerFactory.getLogger(ShuffleTransportContext.class); |
| 55 | + private static final ShuffleMessageDecoder SHUFFLE_DECODER = |
| 56 | + new ShuffleMessageDecoder(MessageDecoder.INSTANCE); |
| 57 | + private final EventLoopGroup finalizeWorkers; |
| 58 | + |
| 59 | + public ShuffleTransportContext( |
| 60 | + TransportConf conf, |
| 61 | + ExternalBlockHandler rpcHandler, |
| 62 | + boolean closeIdleConnections) { |
| 63 | + this(conf, rpcHandler, closeIdleConnections, false); |
| 64 | + } |
| 65 | + |
| 66 | + public ShuffleTransportContext(TransportConf conf, |
| 67 | + RpcHandler rpcHandler, |
| 68 | + boolean closeIdleConnections, |
| 69 | + boolean isClientOnly) { |
| 70 | + super(conf, rpcHandler, closeIdleConnections, isClientOnly); |
| 71 | + |
| 72 | + if ("shuffle".equalsIgnoreCase(conf.getModuleName()) && conf.separateFinalizeShuffleMerge()) { |
| 73 | + finalizeWorkers = NettyUtils.createEventLoop( |
| 74 | + IOMode.valueOf(conf.ioMode()), |
| 75 | + conf.finalizeShuffleMergeHandlerThreads(), |
| 76 | + "shuffle-finalize-merge-handler"); |
| 77 | + logger.info("finalize shuffle merged workers created"); |
| 78 | + } else { |
| 79 | + finalizeWorkers = null; |
| 80 | + } |
| 81 | + } |
| 82 | + |
| 83 | + @Override |
| 84 | + public TransportChannelHandler initializePipeline(SocketChannel channel) { |
| 85 | + TransportChannelHandler ch = super.initializePipeline(channel); |
| 86 | + addHandlerToPipeline(channel, ch); |
| 87 | + return ch; |
| 88 | + } |
| 89 | + |
| 90 | + @Override |
| 91 | + public TransportChannelHandler initializePipeline(SocketChannel channel, |
| 92 | + RpcHandler channelRpcHandler) { |
| 93 | + TransportChannelHandler ch = super.initializePipeline(channel, channelRpcHandler); |
| 94 | + addHandlerToPipeline(channel, ch); |
| 95 | + return ch; |
| 96 | + } |
| 97 | + |
| 98 | + /** |
| 99 | + * Add finalize handler to pipeline if needed. This is needed only when |
| 100 | + * separateFinalizeShuffleMerge is enabled. |
| 101 | + */ |
| 102 | + private void addHandlerToPipeline(SocketChannel channel, |
| 103 | + TransportChannelHandler transportChannelHandler) { |
| 104 | + if (finalizeWorkers != null) { |
| 105 | + channel.pipeline().addLast(finalizeWorkers, FinalizedHandler.HANDLER_NAME, |
| 106 | + new FinalizedHandler(transportChannelHandler.getRequestHandler())); |
| 107 | + } |
| 108 | + } |
| 109 | + |
| 110 | + @Override |
| 111 | + protected MessageToMessageDecoder<ByteBuf> getDecoder() { |
| 112 | + return finalizeWorkers == null ? super.getDecoder() : SHUFFLE_DECODER; |
| 113 | + } |
| 114 | + |
| 115 | + static class ShuffleMessageDecoder extends MessageToMessageDecoder<ByteBuf> { |
| 116 | + |
| 117 | + private final MessageDecoder delegate; |
| 118 | + ShuffleMessageDecoder(MessageDecoder delegate) { |
| 119 | + super(); |
| 120 | + this.delegate = delegate; |
| 121 | + } |
| 122 | + |
| 123 | + /** |
| 124 | + * Decode the message and check if it is a finalize merge request. If yes, then create an |
| 125 | + * internal rpc request message and add it to the list of messages to be handled by |
| 126 | + * {@link TransportChannelHandler} |
| 127 | + */ |
| 128 | + @Override |
| 129 | + protected void decode(ChannelHandlerContext channelHandlerContext, |
| 130 | + ByteBuf byteBuf, |
| 131 | + List<Object> list) throws Exception { |
| 132 | + delegate.decode(channelHandlerContext, byteBuf, list); |
| 133 | + Object msg = list.get(list.size() - 1); |
| 134 | + if (msg instanceof RpcRequest) { |
| 135 | + RpcRequest req = (RpcRequest) msg; |
| 136 | + ByteBuffer buffer = req.body().nioByteBuffer(); |
| 137 | + byte type = Unpooled.wrappedBuffer(buffer).readByte(); |
| 138 | + if (type == BlockTransferMessage.Type.FINALIZE_SHUFFLE_MERGE.id()) { |
| 139 | + list.remove(list.size() - 1); |
| 140 | + RpcRequestInternal rpcRequestInternal = |
| 141 | + new RpcRequestInternal(BlockTransferMessage.Type.FINALIZE_SHUFFLE_MERGE, req); |
| 142 | + logger.trace("Created internal rpc request msg with rpcId {} for finalize merge req", |
| 143 | + req.requestId); |
| 144 | + list.add(rpcRequestInternal); |
| 145 | + } |
| 146 | + } |
| 147 | + } |
| 148 | + } |
| 149 | + |
| 150 | + /** |
| 151 | + * Internal message to handle rpc requests that is not accepted by |
| 152 | + * {@link TransportChannelHandler} as this message doesn't extend {@link Message}. It will be |
| 153 | + * accepted by {@link FinalizedHandler} instead, which is configured to execute in a separate |
| 154 | + * EventLoopGroup. |
| 155 | + */ |
| 156 | + static class RpcRequestInternal { |
| 157 | + public final BlockTransferMessage.Type messageType; |
| 158 | + public final RpcRequest rpcRequest; |
| 159 | + |
| 160 | + RpcRequestInternal(BlockTransferMessage.Type messageType, |
| 161 | + RpcRequest rpcRequest) { |
| 162 | + this.messageType = messageType; |
| 163 | + this.rpcRequest = rpcRequest; |
| 164 | + } |
| 165 | + } |
| 166 | + |
| 167 | + static class FinalizedHandler extends SimpleChannelInboundHandler<RpcRequestInternal> { |
| 168 | + private static final Logger logger = LoggerFactory.getLogger(FinalizedHandler.class); |
| 169 | + public static final String HANDLER_NAME = "finalizeHandler"; |
| 170 | + private final TransportRequestHandler transportRequestHandler; |
| 171 | + |
| 172 | + @Override |
| 173 | + public boolean acceptInboundMessage(Object msg) throws Exception { |
| 174 | + if (msg instanceof RpcRequestInternal) { |
| 175 | + RpcRequestInternal rpcRequestInternal = (RpcRequestInternal) msg; |
| 176 | + return rpcRequestInternal.messageType == BlockTransferMessage.Type.FINALIZE_SHUFFLE_MERGE; |
| 177 | + } |
| 178 | + return false; |
| 179 | + } |
| 180 | + |
| 181 | + FinalizedHandler(TransportRequestHandler transportRequestHandler) { |
| 182 | + this.transportRequestHandler = transportRequestHandler; |
| 183 | + } |
| 184 | + |
| 185 | + @Override |
| 186 | + protected void channelRead0(ChannelHandlerContext channelHandlerContext, |
| 187 | + RpcRequestInternal req) throws Exception { |
| 188 | + if (logger.isTraceEnabled()) { |
| 189 | + logger.trace("Finalize shuffle req from {} for rpc request {}", |
| 190 | + getRemoteAddress(channelHandlerContext.channel()), req.rpcRequest.requestId); |
| 191 | + } |
| 192 | + this.transportRequestHandler.handle(req.rpcRequest); |
| 193 | + } |
| 194 | + } |
| 195 | +} |
0 commit comments