|
| 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.netty |
| 19 | + |
| 20 | +import java.net.InetSocketAddress |
| 21 | + |
| 22 | +import io.netty.bootstrap.ServerBootstrap |
| 23 | +import io.netty.buffer.PooledByteBufAllocator |
| 24 | +import io.netty.channel.{ChannelFuture, ChannelInitializer, ChannelOption} |
| 25 | +import io.netty.channel.epoll.{EpollEventLoopGroup, EpollServerSocketChannel} |
| 26 | +import io.netty.channel.nio.NioEventLoopGroup |
| 27 | +import io.netty.channel.oio.OioEventLoopGroup |
| 28 | +import io.netty.channel.socket.SocketChannel |
| 29 | +import io.netty.channel.socket.nio.NioServerSocketChannel |
| 30 | +import io.netty.channel.socket.oio.OioServerSocketChannel |
| 31 | +import io.netty.handler.codec.LineBasedFrameDecoder |
| 32 | +import io.netty.handler.codec.string.StringDecoder |
| 33 | +import io.netty.util.CharsetUtil |
| 34 | + |
| 35 | +import org.apache.spark.{Logging, SparkConf} |
| 36 | +import org.apache.spark.util.Utils |
| 37 | + |
| 38 | +/** |
| 39 | + * Server for serving Spark data blocks. This should be used together with [[BlockFetchingClient]]. |
| 40 | + * |
| 41 | + * Protocol for requesting blocks: specify one block id per line. |
| 42 | + * |
| 43 | + * Protocol for sending blocks: for each block, |
| 44 | + */ |
| 45 | +private[spark] |
| 46 | +class BlockServer(conf: SparkConf, pResolver: PathResolver) extends Logging { |
| 47 | + |
| 48 | + // TODO: Allow random port selection |
| 49 | + val port: Int = conf.getInt("spark.shuffle.io.port", 12345) |
| 50 | + |
| 51 | + private var bootstrap: ServerBootstrap = _ |
| 52 | + private var channelFuture: ChannelFuture = _ |
| 53 | + |
| 54 | + /** Initialize the server. */ |
| 55 | + def init(): Unit = { |
| 56 | + bootstrap = new ServerBootstrap |
| 57 | + val bossThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-boss") |
| 58 | + val workerThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-worker") |
| 59 | + |
| 60 | + def initNio(): Unit = { |
| 61 | + val bossGroup = new NioEventLoopGroup(0, bossThreadFactory) |
| 62 | + val workerGroup = new NioEventLoopGroup(0, workerThreadFactory) |
| 63 | + bootstrap.group(bossGroup, workerGroup).channel(classOf[NioServerSocketChannel]) |
| 64 | + } |
| 65 | + def initOio(): Unit = { |
| 66 | + val bossGroup = new OioEventLoopGroup(0, bossThreadFactory) |
| 67 | + val workerGroup = new OioEventLoopGroup(0, workerThreadFactory) |
| 68 | + bootstrap.group(bossGroup, workerGroup).channel(classOf[OioServerSocketChannel]) |
| 69 | + } |
| 70 | + def initEpoll(): Unit = { |
| 71 | + val bossGroup = new EpollEventLoopGroup(0, bossThreadFactory) |
| 72 | + val workerGroup = new EpollEventLoopGroup(0, workerThreadFactory) |
| 73 | + bootstrap.group(bossGroup, workerGroup).channel(classOf[EpollServerSocketChannel]) |
| 74 | + } |
| 75 | + |
| 76 | + conf.get("spark.shuffle.io.mode", "auto").toLowerCase match { |
| 77 | + case "nio" => initNio() |
| 78 | + case "oio" => initOio() |
| 79 | + case "epoll" => initEpoll() |
| 80 | + case "auto" => |
| 81 | + // For auto mode, first try epoll (only available on Linux), then nio. |
| 82 | + try { |
| 83 | + initEpoll() |
| 84 | + } catch { |
| 85 | + case e: Throwable => initNio() |
| 86 | + } |
| 87 | + } |
| 88 | + |
| 89 | + // Use pooled buffers to reduce temporary buffer allocation |
| 90 | + bootstrap.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) |
| 91 | + bootstrap.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) |
| 92 | + |
| 93 | + // Various (advanced) user-configured settings. |
| 94 | + conf.getOption("spark.shuffle.io.backLog").foreach { backLog => |
| 95 | + bootstrap.option[java.lang.Integer](ChannelOption.SO_BACKLOG, backLog.toInt) |
| 96 | + } |
| 97 | + // Note: the optimal size for receive buffer and send buffer should be |
| 98 | + // latency * network_bandwidth. |
| 99 | + // Assuming latency = 1ms, network_bandwidth = 10Gbps |
| 100 | + // buffer size should be ~ 1.25MB |
| 101 | + conf.getOption("spark.shuffle.io.receiveBuffer").foreach { receiveBuf => |
| 102 | + bootstrap.option[java.lang.Integer](ChannelOption.SO_RCVBUF, receiveBuf.toInt) |
| 103 | + } |
| 104 | + conf.getOption("spark.shuffle.io.sendBuffer").foreach { sendBuf => |
| 105 | + bootstrap.option[java.lang.Integer](ChannelOption.SO_SNDBUF, sendBuf.toInt) |
| 106 | + } |
| 107 | + |
| 108 | + bootstrap.childHandler(new ChannelInitializer[SocketChannel] { |
| 109 | + override def initChannel(ch: SocketChannel): Unit = { |
| 110 | + ch.pipeline |
| 111 | + .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 |
| 112 | + .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) |
| 113 | + |
| 114 | + ch.pipeline |
| 115 | + .addLast("handler", new BlockServerHandler(pResolver)) |
| 116 | + } |
| 117 | + }) |
| 118 | + |
| 119 | + channelFuture = bootstrap.bind(new InetSocketAddress(port)) |
| 120 | + channelFuture.sync() |
| 121 | + |
| 122 | + val addr = channelFuture.channel.localAddress.asInstanceOf[InetSocketAddress] |
| 123 | + println("address: " + addr.getAddress + " port: " + addr.getPort) |
| 124 | + } |
| 125 | + |
| 126 | + /** Shutdown the server. */ |
| 127 | + def stop(): Unit = { |
| 128 | + if (channelFuture != null) { |
| 129 | + channelFuture.channel().close().awaitUninterruptibly() |
| 130 | + channelFuture = null |
| 131 | + } |
| 132 | + if (bootstrap != null && bootstrap.group() != null) { |
| 133 | + bootstrap.group().shutdownGracefully() |
| 134 | + } |
| 135 | + if (bootstrap != null && bootstrap.childGroup() != null) { |
| 136 | + bootstrap.childGroup().shutdownGracefully() |
| 137 | + } |
| 138 | + bootstrap = null |
| 139 | + } |
| 140 | +} |
| 141 | + |
| 142 | + |
| 143 | +object BlockServer { |
| 144 | + def main(args: Array[String]): Unit = { |
| 145 | + new BlockServer(new SparkConf, null).init() |
| 146 | + Thread.sleep(100000) |
| 147 | + } |
| 148 | +} |
0 commit comments