You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/06/08 13:18:26 UTC

[GitHub] [ozone] elek opened a new pull request #2315: HDDS-5303. Add SSL support to the Ozone streming API

elek opened a new pull request #2315:
URL: https://github.com/apache/ozone/pull/2315


   ## What changes were proposed in this pull request?
   
   HDDS-5142 will introduce a new streaming API for closed container replication / snapshot download and other data movement.
   
   For server2server communication we need to support mTLS. We should configure pure mTLS on the netty server
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5237
   
   ## How was this patch tested?
   
   New unit test is added to test SSL option with self-signed certificate. During the tests I also found some intermittent issues in the DirstreamerClient. I added more unit tests there with minor changes.
   


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] adoroszlai commented on a change in pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on a change in pull request #2315:
URL: https://github.com/apache/ozone/pull/2315#discussion_r647729493



##########
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);

Review comment:
       ```suggestion
         LOG.info("Started streaming server on {}", port);
   ```

##########
File path: hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/stream/package-info.java
##########
@@ -0,0 +1,18 @@
+/**

Review comment:
       ```suggestion
   /*
   ```

##########
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:
       Can we keep `throws InterruptedException` instead?  I think it's better for clarity.

##########
File path: hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/stream/package-info.java
##########
@@ -0,0 +1,18 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.container.stream;
+// Unit tests for streaming.

Review comment:
       ```suggestion
   /**
    * Unit tests for streaming.
    */
   package org.apache.hadoop.ozone.container.stream;
   ```




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] elek commented on a change in pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
elek commented on a change in pull request #2315:
URL: https://github.com/apache/ozone/pull/2315#discussion_r651623809



##########
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:
       I just pushed a new commit (0448a984d) with another approach based on our offline discussion. 
   
   I created a `StreamingException` (runtime) which can be used instead of the generic `RuntimeException` but still unchecked.
   
   Furthermore, I also updated existing streaming API to avoid using raw `new RuntimeException` everywhere in the streaming package.
   
   Please let me know what do you think....




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] elek merged pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
elek merged pull request #2315:
URL: https://github.com/apache/ozone/pull/2315


   


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] elek commented on pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
elek commented on pull request #2315:
URL: https://github.com/apache/ozone/pull/2315#issuecomment-864875716


   Thanks for the review @adoroszlai. I am merging it now...


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] elek commented on a change in pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] adoroszlai commented on a change in pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on a change in pull request #2315:
URL: https://github.com/apache/ozone/pull/2315#discussion_r650965387



##########
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);

Review comment:
       Thanks for updating the patch with the rest of the suggestions.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] adoroszlai commented on a change in pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on a change in pull request #2315:
URL: https://github.com/apache/ozone/pull/2315#discussion_r650964633



##########
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:
       So if I understand correctly, the current patch itself does not need to handle `InterruptedException`, but a future change (replacing the current replication server implementation with the streaming one) will.  So this is in anticipation of the next change.
   
   If that's correct, I'm OK with converting it.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] adoroszlai commented on a change in pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on a change in pull request #2315:
URL: https://github.com/apache/ozone/pull/2315#discussion_r655161534



##########
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:
       Thanks for updating the patch.  I think it's much cleaner.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] adoroszlai commented on a change in pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on a change in pull request #2315:
URL: https://github.com/apache/ozone/pull/2315#discussion_r655161534



##########
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:
       Thanks for updating the patch.  I think it's much cleaner.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] elek merged pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
elek merged pull request #2315:
URL: https://github.com/apache/ozone/pull/2315


   


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] elek commented on pull request #2315: HDDS-5237. Add SSL support to the Ozone streaming API

Posted by GitBox <gi...@apache.org>.
elek commented on pull request #2315:
URL: https://github.com/apache/ozone/pull/2315#issuecomment-864875716


   Thanks for the review @adoroszlai. I am merging it now...


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org