Skip to content
Closed
Show file tree
Hide file tree
Changes from 8 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,8 @@
import java.util.List;

import io.netty.channel.Channel;
import io.netty.channel.ChannelPipeline;
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 +34,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 +81,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;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there any special reason that this must be a global one? I have not yet looked the details. But looks like this may cause ChunkFetchIntegrationSuite flaky as there is no isolation between tests.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I haven't been able to reproduce this but the number of threads used for this tests are 2* number of cores or spark.shuffle.io.serverThreads.


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

synchronized(this.getClass()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think synchronized(this.getClass()) is not recommended due to it not handling inheritance and such. Use synchronized(TransportContext.class)

if (chunkFetchWorkers == null && conf.getModuleName() != null &&
conf.getModuleName().equalsIgnoreCase("shuffle")) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fix spacing here, line up conf.getModuleName with the chunkFetchWorkers

chunkFetchWorkers = NettyUtils.createEventLoop(
IOMode.valueOf(conf.ioMode()),
conf.chunkFetchHandlerThreads(),
"chunk-fetch-handler");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

for consistency perhaps name thread shuffle-chunk-fetch-handler

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

like you mention if we can not create the event loop when on the client side that would be best

}
}
}

/**
Expand Down Expand Up @@ -144,14 +163,21 @@ public TransportChannelHandler initializePipeline(
RpcHandler channelRpcHandler) {
try {
TransportChannelHandler channelHandler = createChannelHandler(channel, channelRpcHandler);
channel.pipeline()
ChunkFetchRequestHandler chunkFetchHandler =
createChunkFetchHandler(channelHandler, channelRpcHandler);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fix spacing, only indented 2 spaces

ChannelPipeline pipeline = channel.pipeline()
.addLast("encoder", ENCODER)
.addLast(TransportFrameDecoder.HANDLER_NAME, NettyUtils.createFrameDecoder())
.addLast("decoder", DECODER)
.addLast("idleStateHandler", new IdleStateHandler(0, 0, conf.connectionTimeoutMs() / 1000))
.addLast("idleStateHandler",
new IdleStateHandler(0, 0, conf.connectionTimeoutMs() / 1000))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fix indentation

// 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);
// Use a separate EventLoopGroup to handle ChunkFetchRequest messages for shuffle rpcs.
if (conf.getModuleName() != null && conf.getModuleName().equalsIgnoreCase("shuffle")) {
pipeline.addLast(chunkFetchWorkers, "chunkFetchHandler", chunkFetchHandler);
}
return channelHandler;
} catch (RuntimeException e) {
logger.error("Error while initializing Netty pipeline", e);
Expand All @@ -173,5 +199,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());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

identation 2 spaces inside return

}

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


Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove extra empty line

/**
* 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);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

spacing 2, fix throughout the file

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this can actually be unwrapped here

ctx.close();
}

@Override
protected void channelRead0(ChannelHandlerContext ctx,
final ChunkFetchRequest msg) throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fix wrapping

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,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fix wrapping and sapcing

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 {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fix indentation

final SocketAddress remoteAddress = channel.remoteAddress();
return channel.writeAndFlush(result).sync().addListener((ChannelFutureListener) future -> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we actually want to use await() here instead of sync(). Sync says "Waits for this future until it is done, and rethrows the cause of the failure if this future failed."

I'm not sure we want a rethrow here since we have the addListener to handle the future failure.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes await can be used as well... i will test it out and let you know its ramifications if any, thanks

Copy link
Author

@redsanket redsanket Sep 17, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok i figured the following... i think it better to rethrow a.k.a. use sync() instead of quitely logging the exception via await(). The reason being as follows....

The respond call made here https://github.com/apache/spark/pull/22173/files#diff-a37b07d454be4d6cd26edb294661d4e3R106 waits for sending the failed or success response back. Hence, it is a blocking call. If an exception is thrown using await() we log quitely and do no rethrow, the underlying rpc handler will not know about the request status here https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java#L183, rethrowing the exception allows it to handle and throw an rpc failure here https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java#L194 else we have to wait for a timeout exception instead of an Interrupt exception. Either cases this seems fine but using sync() we will fail fast...

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

so it doesn't go through https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java#L183 since that is hte rpchandler . The TransportChannelHandler won't handle it at all, it will go to the pipeline which calls the ChunkFetchRequestHandler. So whatever is calling that is what would get the exception propogated. The reason I said this is because previously when this was handled in TransportChannelHandler, it wasn't throwing the exception up. The TrasnportRequestHandler.respond is calling hte writeAndFlush async and then just adding a listener which would simply log an error if the future wasn't successful.
I want to make sure if an exception is thrown here it doesn't kill the entire external shuffle service for instance.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok i had chunkFetchHandler as an instance of rpcHandler in my mind...

Copy link
Author

@redsanket redsanket Sep 18, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It justs logs the exception but the behaviour is same… as in the first case an additional warning from netty side is passed onto the listener and the channel is closed… in the later we quitely log error and the channel is closed… the executors deal with the closed channel and retry or schedule the task on different executor… and eventually the job seems to succeed… in the example i ran… i expect the retry to go through in such scenarios… i will just use await() as the ERROR is logged and we dont have to be more verbose with the warning from the netty side...

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 @@ -18,14 +18,16 @@
package org.apache.spark.network.server;

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 +49,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 +114,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 @@ -24,15 +24,27 @@
import com.google.common.base.Throwables;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NioManagedBuffer;
import org.apache.spark.network.client.*;
import org.apache.spark.network.protocol.*;
import org.apache.spark.network.client.RpcResponseCallback;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

revert the imports to be .*

import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.client.StreamCallbackWithID;
import org.apache.spark.network.client.StreamInterceptor;
import org.apache.spark.network.protocol.Encodable;
import org.apache.spark.network.protocol.OneWayMessage;
import org.apache.spark.network.protocol.RequestMessage;
import org.apache.spark.network.protocol.RpcFailure;
import org.apache.spark.network.protocol.RpcRequest;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

these don't seem like should be necessary? and in wrong location

import org.apache.spark.network.protocol.RpcResponse;
import org.apache.spark.network.protocol.StreamFailure;
import org.apache.spark.network.protocol.StreamRequest;
import org.apache.spark.network.protocol.StreamResponse;
import org.apache.spark.network.protocol.UploadStream;
import org.apache.spark.network.util.TransportFrameDecoder;

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

put extra line back to keep separation

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

/**
Expand Down Expand Up @@ -97,9 +109,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 @@ -112,36 +122,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 @@ -21,6 +21,7 @@
import java.util.Properties;

import com.google.common.primitives.Ints;
import io.netty.util.NettyRuntime;

/**
* A central location that tracks all the settings we expose to users.
Expand Down Expand Up @@ -281,4 +282,31 @@ public Properties cryptoConf() {
public long maxChunksBeingTransferred() {
return conf.getLong("spark.shuffle.maxChunksBeingTransferred", Long.MAX_VALUE);
}

/**
* Percentage of io.serverThreads used by netty 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
* or io.serverThreads. 90 would mean 90% of 2*#cores or 90% of io.serverThreads
* which equals 0.9 * 2*#cores or 0.9 * io.serverThreads.
*/
public int chunkFetchHandlerThreads() {
if (!this.getModuleName().equalsIgnoreCase("shuffle")) {
return 0;
}
int chunkFetchHandlerThreadsPercent =
conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 0);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fix spacing

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes it is documented above... if it is 0 or 100 it is 2*#cores or io.serverThreads

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fix indentation

return this.serverThreads() > 0 ? (this.serverThreads() * chunkFetchHandlerThreadsPercent)/100:
(2 * NettyRuntime.availableProcessors() * chunkFetchHandlerThreadsPercent)/100;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fix indentation

}
}
Loading