aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/java/spark/network/netty/FileServer.java
blob: dd3f12561cb655cfb14b822a0d61376550276196 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
package spark.network.netty;

import java.net.InetSocketAddress;

import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelOption;
import io.netty.channel.oio.OioEventLoopGroup;
import io.netty.channel.socket.oio.OioServerSocketChannel;

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


/**
 * Server that accept the path of a file an echo back its content.
 */
class FileServer {

  private Logger LOG = LoggerFactory.getLogger(this.getClass().getName());

  private ServerBootstrap bootstrap = null;
  private ChannelFuture channelFuture = null;
  private int port = 0;
  private Thread blockingThread = null;

  public FileServer(PathResolver pResolver, int port) {
    InetSocketAddress addr = new InetSocketAddress(port);

    // Configure the server.
    bootstrap = new ServerBootstrap();
    bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup())
        .channel(OioServerSocketChannel.class)
        .option(ChannelOption.SO_BACKLOG, 100)
        .option(ChannelOption.SO_RCVBUF, 1500)
        .childHandler(new FileServerChannelInitializer(pResolver));
    // Start the server.
    channelFuture = bootstrap.bind(addr);
    this.port = addr.getPort();
  }

  /**
   * Start the file server asynchronously in a new thread.
   */
  public void start() {
    try {
      blockingThread = new Thread() {
        public void run() {
          try {
            Channel channel = channelFuture.sync().channel();
            channel.closeFuture().sync();
          } catch (InterruptedException e) {
            LOG.error("File server start got interrupted", e);
          }
        }
      };
      blockingThread.setDaemon(true);
      blockingThread.start();
    } finally {
      bootstrap.shutdown();
    }
  }

  public int getPort() {
    return port;
  }

  public void stop() {
    if (blockingThread != null) {
      blockingThread.stop();
      blockingThread = null;
    }
    if (channelFuture != null) {
      channelFuture.channel().closeFuture();
      channelFuture = null;
    }
    if (bootstrap != null) {
      bootstrap.shutdown();
      bootstrap = null;
    }
  }
}