elek commented on a change in pull request #2315:
URL: https://github.com/apache/ozone/pull/2315#discussion_r650922219
##########
File path:
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/StreamingServer.java
##########
@@ -47,35 +48,54 @@
private EventLoopGroup workerGroup;
+ private SslContext sslContext;
+
public StreamingServer(
- StreamingSource source, int port
+ StreamingSource source, int port
+ ) {
+ this(source, port, null);
+ }
+
+ public StreamingServer(
+ StreamingSource source, int port, SslContext sslContext
) {
this.port = port;
this.source = source;
+ this.sslContext = sslContext;
}
- public void start() throws InterruptedException {
- ServerBootstrap b = new ServerBootstrap();
- bossGroup = new NioEventLoopGroup(100);
- workerGroup = new NioEventLoopGroup(100);
-
- b.group(bossGroup, workerGroup)
- .channel(NioServerSocketChannel.class)
- .option(ChannelOption.SO_BACKLOG, 100)
- .childHandler(new ChannelInitializer<SocketChannel>() {
- @Override
- public void initChannel(SocketChannel ch) throws Exception {
- ch.pipeline().addLast(
- new ChunkedWriteHandler(),
- new DirstreamServerHandler(source));
+ public void start() {
+ try {
+ ServerBootstrap b = new ServerBootstrap();
+ bossGroup = new NioEventLoopGroup(100);
+ workerGroup = new NioEventLoopGroup(100);
+
+ b.group(bossGroup, workerGroup)
+ .channel(NioServerSocketChannel.class)
+ .option(ChannelOption.SO_BACKLOG, 100)
+
+ .childHandler(new ChannelInitializer<SocketChannel>() {
+ @Override
+ public void initChannel(SocketChannel ch) throws Exception {
+ if (sslContext != null) {
+ ch.pipeline().addLast(sslContext.newHandler(ch.alloc()));
}
- });
+ ch.pipeline().addLast(
+ new ChunkedWriteHandler(),
+ new DirstreamServerHandler(source));
+
+
+ }
+ });
- ChannelFuture f = b.bind(port).sync();
- final InetSocketAddress socketAddress =
- (InetSocketAddress) f.channel().localAddress();
- port = socketAddress.getPort();
- LOG.info("Started streaming server on " + port);
+ ChannelFuture f = b.bind(port).sync();
+ final InetSocketAddress socketAddress =
+ (InetSocketAddress) f.channel().localAddress();
+ port = socketAddress.getPort();
+ LOG.info("Started streaming server on " + port);
+ } catch (InterruptedException ex) {
+ throw new RuntimeException(ex);
Review comment:
`InterrputedException` is a checked exception what I tried to avoid,
especially as we don't see any added value in differentiating between this
error or any other runtime error during the startup.
I would be happy to replace it with any more specific runtime exception --
if you have any suggestion -- but not sure how would it be more clear with
keeping checked execption,
I tried how would it look like with keeping the checked
`InterruptedException` here, but it doesn't look *more clear* to me, it
requires same conversation (but later) plus maintaining additional checked
execption in the method signature.
Looks to be more clear for me simple use runtime exception instead of
checked one (BTW, the current practice to convert everything to `IOException`
is also more closed to use `RuntimeExceptions` everywhere and do the
differentating only if it's required.
Uploaded the experiment to here:
https://github.com/elek/ozone/commit/767b5fb346178ef27075a5a3413392abe60075c9,
I can add it to this PR, if this is your strong preference...
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]