Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.util.List;

import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.socket.SocketChannel;
import io.netty.handler.timeout.IdleStateHandler;
import org.slf4j.Logger;
Expand All @@ -32,11 +33,13 @@
import org.apache.spark.network.client.TransportResponseHandler;
import org.apache.spark.network.protocol.MessageDecoder;
import org.apache.spark.network.protocol.MessageEncoder;
import org.apache.spark.network.server.ChunkFetchRequestHandler;
import org.apache.spark.network.server.RpcHandler;
import org.apache.spark.network.server.TransportChannelHandler;
import org.apache.spark.network.server.TransportRequestHandler;
import org.apache.spark.network.server.TransportServer;
import org.apache.spark.network.server.TransportServerBootstrap;
import org.apache.spark.network.util.IOMode;
import org.apache.spark.network.util.NettyUtils;
import org.apache.spark.network.util.TransportConf;
import org.apache.spark.network.util.TransportFrameDecoder;
Expand Down Expand Up @@ -77,6 +80,11 @@ public class TransportContext {
private static final MessageEncoder ENCODER = MessageEncoder.INSTANCE;
private static final MessageDecoder DECODER = MessageDecoder.INSTANCE;

// Separate thread pool for handling ChunkFetchRequest. This helps to enable throttling
// max number of TransportServer worker threads that are blocked on writing response
// of ChunkFetchRequest message back to the client via the underlying channel.
private static EventLoopGroup chunkFetchWorkers;

public TransportContext(TransportConf conf, RpcHandler rpcHandler) {
this(conf, rpcHandler, false);
}
Expand All @@ -88,6 +96,15 @@ public TransportContext(
this.conf = conf;
this.rpcHandler = rpcHandler;
this.closeIdleConnections = closeIdleConnections;

synchronized(this.getClass()) {
if (chunkFetchWorkers == null) {
chunkFetchWorkers = NettyUtils.createEventLoop(
IOMode.valueOf(conf.ioMode()),
conf.chunkFetchHandlerThreads(),
"chunk-fetch-handler");
}
}
}

/**
Expand Down Expand Up @@ -144,14 +161,17 @@ public TransportChannelHandler initializePipeline(
RpcHandler channelRpcHandler) {
try {
TransportChannelHandler channelHandler = createChannelHandler(channel, channelRpcHandler);
ChunkFetchRequestHandler chunkFetchHandler = createChunkFetchHandler(channelHandler, channelRpcHandler);
channel.pipeline()
.addLast("encoder", ENCODER)
.addLast(TransportFrameDecoder.HANDLER_NAME, NettyUtils.createFrameDecoder())
.addLast("decoder", DECODER)
.addLast("idleStateHandler", new IdleStateHandler(0, 0, conf.connectionTimeoutMs() / 1000))
// NOTE: Chunks are currently guaranteed to be returned in the order of request, but this
// would require more logic to guarantee if this were not part of the same event loop.
.addLast("handler", channelHandler);
.addLast("handler", channelHandler)
// Use a separate EventLoopGroup to handle ChunkFetchRequest messages.
.addLast(chunkFetchWorkers, "chunkFetchHandler", chunkFetchHandler);
return channelHandler;
} catch (RuntimeException e) {
logger.error("Error while initializing Netty pipeline", e);
Expand All @@ -173,5 +193,14 @@ private TransportChannelHandler createChannelHandler(Channel channel, RpcHandler
conf.connectionTimeoutMs(), closeIdleConnections);
}

/**
* Creates the dedicated ChannelHandler for ChunkFetchRequest messages.
*/
private ChunkFetchRequestHandler createChunkFetchHandler(TransportChannelHandler channelHandler,
RpcHandler rpcHandler) {
return new ChunkFetchRequestHandler(channelHandler.getClient(),
rpcHandler.getStreamManager(), conf.maxChunksBeingTransferred());
}

public TransportConf getConf() { return conf; }
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
/*
* 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.spark.network.server;

import java.net.SocketAddress;

import com.google.common.base.Throwables;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.protocol.ChunkFetchFailure;
import org.apache.spark.network.protocol.ChunkFetchRequest;
import org.apache.spark.network.protocol.ChunkFetchSuccess;
import org.apache.spark.network.protocol.Encodable;

import static org.apache.spark.network.util.NettyUtils.*;


/**
* A dedicated ChannelHandler for processing ChunkFetchRequest messages. When sending response
* of ChunkFetchRequest messages to the clients, the thread performing the I/O on the underlying
* channel could potentially be blocked due to disk contentions. If several hundreds of clients
* send ChunkFetchRequest to the server at the same time, it could potentially occupying all
* threads from TransportServer's default EventLoopGroup for waiting for disk reads before it
* can send the block data back to the client as part of the ChunkFetchSuccess messages. As a
* result, it would leave no threads left to process other RPC messages, which takes much less
* time to process, and could lead to client timing out on either performing SASL authentication,
* registering executors, or waiting for response for an OpenBlocks messages.
*/
public class ChunkFetchRequestHandler extends SimpleChannelInboundHandler<ChunkFetchRequest> {
private static final Logger logger = LoggerFactory.getLogger(ChunkFetchRequestHandler.class);

private final TransportClient client;
private final StreamManager streamManager;
/** The max number of chunks being transferred and not finished yet. */
private final long maxChunksBeingTransferred;

public ChunkFetchRequestHandler(
TransportClient client,
StreamManager streamManager,
Long maxChunksBeingTransferred) {
this.client = client;
this.streamManager = streamManager;
this.maxChunksBeingTransferred = maxChunksBeingTransferred;
}

@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
logger.warn("Exception in connection from " + getRemoteAddress(ctx.channel()),
cause);
ctx.close();
}

@Override
protected void channelRead0(ChannelHandlerContext ctx, final ChunkFetchRequest msg) throws Exception {
Channel channel = ctx.channel();
if (logger.isTraceEnabled()) {
logger.trace("Received req from {} to fetch block {}", getRemoteAddress(channel),
msg.streamChunkId);
}
long chunksBeingTransferred = streamManager.chunksBeingTransferred();
if (chunksBeingTransferred >= maxChunksBeingTransferred) {
logger.warn("The number of chunks being transferred {} is above {}, close the connection.",
chunksBeingTransferred, maxChunksBeingTransferred);
channel.close();
return;
}
ManagedBuffer buf;
try {
streamManager.checkAuthorization(client, msg.streamChunkId.streamId);
streamManager.registerChannel(channel, msg.streamChunkId.streamId);
buf = streamManager.getChunk(msg.streamChunkId.streamId, msg.streamChunkId.chunkIndex);
} catch (Exception e) {
logger.error(String.format("Error opening block %s for request from %s",
msg.streamChunkId, getRemoteAddress(channel)), e);
respond(channel, new ChunkFetchFailure(msg.streamChunkId, Throwables.getStackTraceAsString(e)));
return;
}

streamManager.chunkBeingSent(msg.streamChunkId.streamId);
respond(channel, new ChunkFetchSuccess(msg.streamChunkId, buf)).addListener(
(ChannelFutureListener) future -> streamManager.chunkSent(msg.streamChunkId.streamId));
}

/**
* The invocation to channel.writeAndFlush is async, and the actual I/O on the channel will be handled
* by the EventLoop the channel is registered to. So even though we are processing the ChunkFetchRequest
* in a separate thread pool, the actual I/O, which is the potentially blocking call that could deplete
* server handler threads, is still being processed by TransportServer's default EventLoopGroup. In order
* to throttle the max number of threads that channel I/O for sending response to ChunkFetchRequest,
* the thread calling channel.writeAndFlush will wait for the completion of sending response back to client
* by invoking sync(). This will throttle the rate at which threads from ChunkFetchRequest dedicated
* EventLoopGroup submit channel I/O requests to TransportServer's default EventLoopGroup, thus making sure
* that we can reserve some threads in TransportServer's default EventLoopGroup for handling other RPC
* messages.
*/
private ChannelFuture respond(final Channel channel, final Encodable result) throws InterruptedException {
final SocketAddress remoteAddress = channel.remoteAddress();
return channel.writeAndFlush(result).sync().addListener((ChannelFutureListener) future -> {
if (future.isSuccess()) {
logger.trace("Sent result {} to client {}", result, remoteAddress);
} else {
logger.error(String.format("Error sending result %s to %s; closing connection",
result, remoteAddress), future.cause());
channel.close();
}
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,16 @@

import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.timeout.IdleState;
import io.netty.handler.timeout.IdleStateEvent;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.client.TransportResponseHandler;
import org.apache.spark.network.protocol.ChunkFetchRequest;
import org.apache.spark.network.protocol.Message;
import org.apache.spark.network.protocol.RequestMessage;
import org.apache.spark.network.protocol.ResponseMessage;
import static org.apache.spark.network.util.NettyUtils.getRemoteAddress;
Expand All @@ -47,7 +50,7 @@
* on the channel for at least `requestTimeoutMs`. Note that this is duplex traffic; we will not
* timeout if the client is continuously sending but getting no responses, for simplicity.
*/
public class TransportChannelHandler extends ChannelInboundHandlerAdapter {
public class TransportChannelHandler extends SimpleChannelInboundHandler<Message> {
private static final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class);

private final TransportClient client;
Expand Down Expand Up @@ -112,8 +115,21 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception {
super.channelInactive(ctx);
}

/**
* Overwrite acceptInboundMessage to properly delegate ChunkFetchRequest messages
* to ChunkFetchRequestHandler.
*/
@Override
public void channelRead(ChannelHandlerContext ctx, Object request) throws Exception {
public boolean acceptInboundMessage(Object msg) throws Exception {
if (msg instanceof ChunkFetchRequest) {
return false;
} else {
return super.acceptInboundMessage(msg);
}
}

@Override
public void channelRead0(ChannelHandlerContext ctx, Message request) throws Exception {
if (request instanceof RequestMessage) {
requestHandler.handle((RequestMessage) request);
} else if (request instanceof ResponseMessage) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,6 @@
import org.apache.spark.network.buffer.NioManagedBuffer;
import org.apache.spark.network.client.RpcResponseCallback;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.protocol.ChunkFetchRequest;
import org.apache.spark.network.protocol.ChunkFetchFailure;
import org.apache.spark.network.protocol.ChunkFetchSuccess;
import org.apache.spark.network.protocol.Encodable;
import org.apache.spark.network.protocol.OneWayMessage;
import org.apache.spark.network.protocol.RequestMessage;
Expand Down Expand Up @@ -105,9 +102,7 @@ public void channelInactive() {

@Override
public void handle(RequestMessage request) {
if (request instanceof ChunkFetchRequest) {
processFetchRequest((ChunkFetchRequest) request);
} else if (request instanceof RpcRequest) {
if (request instanceof RpcRequest) {
processRpcRequest((RpcRequest) request);
} else if (request instanceof OneWayMessage) {
processOneWayMessage((OneWayMessage) request);
Expand All @@ -118,36 +113,6 @@ public void handle(RequestMessage request) {
}
}

private void processFetchRequest(final ChunkFetchRequest req) {
if (logger.isTraceEnabled()) {
logger.trace("Received req from {} to fetch block {}", getRemoteAddress(channel),
req.streamChunkId);
}
long chunksBeingTransferred = streamManager.chunksBeingTransferred();
if (chunksBeingTransferred >= maxChunksBeingTransferred) {
logger.warn("The number of chunks being transferred {} is above {}, close the connection.",
chunksBeingTransferred, maxChunksBeingTransferred);
channel.close();
return;
}
ManagedBuffer buf;
try {
streamManager.checkAuthorization(reverseClient, req.streamChunkId.streamId);
streamManager.registerChannel(channel, req.streamChunkId.streamId);
buf = streamManager.getChunk(req.streamChunkId.streamId, req.streamChunkId.chunkIndex);
} catch (Exception e) {
logger.error(String.format("Error opening block %s for request from %s",
req.streamChunkId, getRemoteAddress(channel)), e);
respond(new ChunkFetchFailure(req.streamChunkId, Throwables.getStackTraceAsString(e)));
return;
}

streamManager.chunkBeingSent(req.streamChunkId.streamId);
respond(new ChunkFetchSuccess(req.streamChunkId, buf)).addListener(future -> {
streamManager.chunkSent(req.streamChunkId.streamId);
});
}

private void processStreamRequest(final StreamRequest req) {
if (logger.isTraceEnabled()) {
logger.trace("Received req from {} to fetch stream {}", getRemoteAddress(channel),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -281,4 +281,23 @@ public Properties cryptoConf() {
public long maxChunksBeingTransferred() {
return conf.getLong("spark.shuffle.maxChunksBeingTransferred", Long.MAX_VALUE);
}

/**
* Number of threads to process ChunkFetchRequest. Shuffle server will use a separate
* EventLoopGroup to process ChunkFetchRequest messages. Although when calling the
* async writeAndFlush on the underlying channel to send response back to client,
* the I/O on the channel is still being handled by
* {@link org.apache.spark.network.server.TransportServer}'s default EventLoopGroup
* that's registered with the Channel, by waiting inside the ChunkFetchRequest handler
* threads for the completion of sending back responses, we are able to put a limit on
* the max number of threads from TransportServer's default EventLoopGroup that are
* going to be consumed by writing response to ChunkFetchRequest, which are I/O intensive
* and could take long time to process due to disk contentions. By configuring a slightly
* higher number of shuffler server threads, we are able to reserve some threads for
* handling other RPC messages, thus making the Client less likely to experience timeout
* when sending RPC messages to the shuffle server. Default to 0, which is 2*#cores.
*/
public int chunkFetchHandlerThreads() {
return conf.getInt("spark.shuffle.server.chunkFetchHandlerThreads", 0);
}
}
Loading