You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2021/03/29 09:22:20 UTC

[GitHub] [hadoop] jojochuang opened a new pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

jojochuang opened a new pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832


   ## NOTICE
   JIRA: https://issues.apache.org/jira/browse/HADOOP-11245
   
   This is a draft. It passed unit tests but need functional tests to ensure things like memory leak, performance is good. Looking for additional pairs of eyes to help with the code review.


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on a change in pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
jojochuang commented on a change in pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#discussion_r615684655



##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java
##########
@@ -95,36 +108,59 @@ RpcProgramPortmap getHandler() {
   void start(final int idleTimeMilliSeconds, final SocketAddress tcpAddress,
       final SocketAddress udpAddress) {
 
-    tcpServer = new ServerBootstrap(new NioServerSocketChannelFactory(
-        Executors.newCachedThreadPool(), Executors.newCachedThreadPool()));
-    tcpServer.setPipelineFactory(new ChannelPipelineFactory() {
-      private final HashedWheelTimer timer = new HashedWheelTimer();
-      private final IdleStateHandler idleStateHandler = new IdleStateHandler(
-          timer, 0, 0, idleTimeMilliSeconds, TimeUnit.MILLISECONDS);
-
-      @Override
-      public ChannelPipeline getPipeline() throws Exception {
-        return Channels.pipeline(RpcUtil.constructRpcFrameDecoder(),
-            RpcUtil.STAGE_RPC_MESSAGE_PARSER, idleStateHandler, handler,
-            RpcUtil.STAGE_RPC_TCP_RESPONSE);
-      }
-    });
-    tcpServer.setOption("reuseAddress", true);
-    tcpServer.setOption("child.reuseAddress", true);
-
-    udpServer = new ConnectionlessBootstrap(new NioDatagramChannelFactory(
-        Executors.newCachedThreadPool()));
-
-    udpServer.setPipeline(Channels.pipeline(RpcUtil.STAGE_RPC_MESSAGE_PARSER,
-        handler, RpcUtil.STAGE_RPC_UDP_RESPONSE));
-    udpServer.setOption("reuseAddress", true);
-
-    tcpChannel = tcpServer.bind(tcpAddress);
-    udpChannel = udpServer.bind(udpAddress);
+    bossGroup = new NioEventLoopGroup();
+    workerGroup = new NioEventLoopGroup(0, Executors.newCachedThreadPool());
+
+    tcpServer = new ServerBootstrap();
+    tcpServer.group(bossGroup, workerGroup)
+        .option(ChannelOption.SO_REUSEADDR, true)
+        .childOption(ChannelOption.SO_REUSEADDR, true)
+        .channel(NioServerSocketChannel.class)
+        .childHandler(new ChannelInitializer<SocketChannel>() {
+          private final IdleStateHandler idleStateHandler = new IdleStateHandler(
+              0, 0, idleTimeMilliSeconds, TimeUnit.MILLISECONDS);
+
+          @Override
+          protected void initChannel(SocketChannel ch) throws Exception {
+            ChannelPipeline p = ch.pipeline();
+
+            p.addLast(RpcUtil.constructRpcFrameDecoder(),
+                RpcUtil.STAGE_RPC_MESSAGE_PARSER, idleStateHandler, handler,
+                RpcUtil.STAGE_RPC_TCP_RESPONSE);
+          }});
+
+    udpGroup = new NioEventLoopGroup(0, Executors.newCachedThreadPool());
+
+    udpServer = new Bootstrap();
+    udpServer.group(udpGroup)
+        .channel(NioDatagramChannel.class)
+        .handler(new ChannelInitializer<NioDatagramChannel>() {
+          @Override protected void initChannel(NioDatagramChannel ch)
+              throws Exception {
+            ChannelPipeline p = ch.pipeline();
+            p.addLast(
+                new LoggingHandler(LogLevel.DEBUG),
+                RpcUtil.STAGE_RPC_MESSAGE_PARSER, handler, RpcUtil.STAGE_RPC_UDP_RESPONSE);
+          }
+        })
+        .option(ChannelOption.SO_REUSEADDR, true);
+
+    ChannelFuture tcpChannelFuture = null;
+    try {
+      tcpChannelFuture = tcpServer.bind(tcpAddress).sync();
+      tcpChannel = tcpChannelFuture.channel();
+
+      ChannelFuture udpChannelFuture = udpServer.bind(udpAddress).sync();
+      udpChannel = udpChannelFuture.channel();
+
+    } catch (InterruptedException e) {
+      e.printStackTrace();

Review comment:
       make sense.

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java
##########
@@ -242,26 +242,26 @@ public static boolean verifyLength(XDR xdr, int len) {
    * @param last specifies last request or not
    * @return TCP buffer
    */
-  public static ChannelBuffer writeMessageTcp(XDR request, boolean last) {
+  public static ByteBuf writeMessageTcp(XDR request, boolean last) {
     Preconditions.checkState(request.state == XDR.State.WRITING);
     ByteBuffer b = request.buf.duplicate();
     b.flip();
     byte[] fragmentHeader = XDR.recordMark(b.limit(), last);
     ByteBuffer headerBuf = ByteBuffer.wrap(fragmentHeader);
 
     // TODO: Investigate whether making a copy of the buffer is necessary.
-    return ChannelBuffers.copiedBuffer(headerBuf, b);
+    return Unpooled.wrappedBuffer(headerBuf, b);

Review comment:
       Intentional. As suspected in the TODO above, it seems wrapping instead of copying the buffer does work.

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServer.java
##########
@@ -39,9 +42,11 @@
       LoggerFactory.getLogger(SimpleTcpServer.class);
   protected final int port;
   protected int boundPort = -1; // Will be set after server starts
-  protected final SimpleChannelUpstreamHandler rpcProgram;
+  protected final ChannelInboundHandlerAdapter rpcProgram;
   private ServerBootstrap server;
   private Channel ch;
+  private EventLoopGroup bossGroup;
+  private EventLoopGroup workerGroup;

Review comment:
       these are all private members. Only the shutdown() and run() methods touch them, and both methods are called by the same thread. So given the current usage volatile isn't necessary.

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/RpcProgramPortmap.java
##########
@@ -60,6 +60,8 @@
   private final ChannelGroup allChannels;
 
   RpcProgramPortmap(ChannelGroup allChannels) {
+    super(1, 1, 1);
+    // FIXME: set default idle timeout 1 second.

Review comment:
       So this one I am not 100% sure. Without calling super(), the default is no idle timeout (not sure about the default behavior in netty3), TestPortmap#testIdle() will fail. So I have to set a timeout. If I set a long timeout like 60 seconds,  TestPortmap#testIdle() will fail too. 




-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-813579388


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 57s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  14m 17s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  20m 19s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 54s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  18m  2s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   3m 43s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 54s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 26s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 52s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m  3s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 17s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 52s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 15s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |  20m 15s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m  8s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |  18m  8s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   3m 36s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/2/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 23 new + 175 unchanged - 10 fixed = 198 total (was 185)  |
   | +1 :green_heart: |  mvnsite  |   2m 59s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  4s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   2m 23s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 54s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m 42s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 34s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 58s |  |  hadoop-nfs in the patch passed.  |
   | -1 :x: |  unit  | 407m 22s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/2/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 44s |  |  hadoop-hdfs-nfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  6s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 597m 19s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.namenode.TestDecommissioningStatus |
   |   | hadoop.hdfs.TestViewDistributedFileSystemContract |
   |   | hadoop.hdfs.TestSnapshotCommands |
   |   | hadoop.hdfs.TestPersistBlocks |
   |   | hadoop.hdfs.TestDFSShell |
   |   | hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList |
   |   | hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeWithHdfsScheme |
   |   | hadoop.hdfs.TestStateAlignmentContextWithHA |
   |   | hadoop.fs.viewfs.TestViewFSOverloadSchemeWithMountTableConfigInHDFS |
   |   | hadoop.hdfs.server.namenode.TestFileTruncate |
   |   | hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor |
   |   | hadoop.hdfs.TestLeaseRecovery2 |
   |   | hadoop.hdfs.server.namenode.ha.TestEditLogTailer |
   |   | hadoop.hdfs.server.namenode.ha.TestPipelinesFailover |
   |   | hadoop.hdfs.server.datanode.TestBlockScanner |
   |   | hadoop.hdfs.server.datanode.TestIncrementalBrVariations |
   |   | hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots |
   |   | hadoop.hdfs.server.datanode.TestBlockRecovery |
   |   | hadoop.hdfs.qjournal.server.TestJournalNodeRespectsBindHostKeys |
   |   | hadoop.hdfs.server.datanode.TestDirectoryScanner |
   |   | hadoop.hdfs.TestHDFSFileSystemContract |
   |   | hadoop.hdfs.server.namenode.ha.TestBootstrapStandby |
   |   | hadoop.hdfs.web.TestWebHdfsFileSystemContract |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2832 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle |
   | uname | Linux f70f948666ef 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / d8256fe16007dcf84d152053e9de9bbab64303e8 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/2/testReport/ |
   | Max. process+thread count | 2516 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-nfs hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/2/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
jojochuang commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-813391259


   I realized this PR breaks YARN because the netty3 dependency is removed from hdfs, and then transitively YARN doesn't have the netty3 dependency. As a result, when YARN NM starts, it couldn't find netty3 required by the MR shuffle handler.
   
   I'll post a follow-up to add explicit dependency for the MR shuffle handler so we can proceed with this one first, leaving the YARN side along.


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szetszwo commented on a change in pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#discussion_r613193567



##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServer.java
##########
@@ -59,35 +64,41 @@ public SimpleTcpServer(int port, RpcProgram program, int workercount) {
 
   public void run() {
     // Configure the Server.
-    ChannelFactory factory;
+    bossGroup = new NioEventLoopGroup();
+
     if (workerCount == 0) {
       // Use default workers: 2 * the number of available processors
-      factory = new NioServerSocketChannelFactory(
-          Executors.newCachedThreadPool(), Executors.newCachedThreadPool());
+      workerGroup = new NioEventLoopGroup(0, Executors.newCachedThreadPool());
     } else {
-      factory = new NioServerSocketChannelFactory(
-          Executors.newCachedThreadPool(), Executors.newCachedThreadPool(),
-          workerCount);
+      workerGroup = new NioEventLoopGroup(workerCount, Executors.newCachedThreadPool());
     }

Review comment:
       The if-else is no longer need.  Simply pass workerCount for both cases as below.
   ```
         workerGroup = new NioEventLoopGroup(workerCount, Executors.newCachedThreadPool());
   ```

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServer.java
##########
@@ -39,9 +42,11 @@
       LoggerFactory.getLogger(SimpleTcpServer.class);
   protected final int port;
   protected int boundPort = -1; // Will be set after server starts
-  protected final SimpleChannelUpstreamHandler rpcProgram;
+  protected final ChannelInboundHandlerAdapter rpcProgram;
   private ServerBootstrap server;
   private Channel ch;
+  private EventLoopGroup bossGroup;
+  private EventLoopGroup workerGroup;

Review comment:
       Should the non-final fields (boundPort, server, ch, bossGroup, workerGroup) be volatile?  Not sure if they are synchronized in somewhere.
   

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java
##########
@@ -242,26 +242,26 @@ public static boolean verifyLength(XDR xdr, int len) {
    * @param last specifies last request or not
    * @return TCP buffer
    */
-  public static ChannelBuffer writeMessageTcp(XDR request, boolean last) {
+  public static ByteBuf writeMessageTcp(XDR request, boolean last) {
     Preconditions.checkState(request.state == XDR.State.WRITING);
     ByteBuffer b = request.buf.duplicate();
     b.flip();
     byte[] fragmentHeader = XDR.recordMark(b.limit(), last);
     ByteBuffer headerBuf = ByteBuffer.wrap(fragmentHeader);
 
     // TODO: Investigate whether making a copy of the buffer is necessary.
-    return ChannelBuffers.copiedBuffer(headerBuf, b);
+    return Unpooled.wrappedBuffer(headerBuf, b);

Review comment:
       The new code uses wrappedBuffer but old code uses copiedBuffer.  Is it intended?

##########
File path: hadoop-common-project/hadoop-nfs/src/test/resources/log4j.properties
##########
@@ -15,4 +15,4 @@ log4j.rootLogger=info,stdout
 log4j.threshold=ALL
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

Review comment:
       Let's revert this whitespace change in order to keep log4j.properties untouched.

##########
File path: hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/portmap/TestPortmap.java
##########
@@ -92,6 +92,7 @@ public void testRegistration() throws IOException, InterruptedException {
     DatagramPacket p = new DatagramPacket(reqBuf, reqBuf.length,
         pm.getUdpServerLoAddress());
     try {
+

Review comment:
       Let's revert this whitespace change.

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleUdpServer.java
##########
@@ -84,8 +97,6 @@ public void shutdown() {
     if (ch != null) {
       ch.close().awaitUninterruptibly();
     }
-    if (server != null) {
-      server.releaseExternalResources();
-    }
+    workerGroup.shutdownGracefully();

Review comment:
       Check null.

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java
##########
@@ -95,36 +108,59 @@ RpcProgramPortmap getHandler() {
   void start(final int idleTimeMilliSeconds, final SocketAddress tcpAddress,
       final SocketAddress udpAddress) {
 
-    tcpServer = new ServerBootstrap(new NioServerSocketChannelFactory(
-        Executors.newCachedThreadPool(), Executors.newCachedThreadPool()));
-    tcpServer.setPipelineFactory(new ChannelPipelineFactory() {
-      private final HashedWheelTimer timer = new HashedWheelTimer();
-      private final IdleStateHandler idleStateHandler = new IdleStateHandler(
-          timer, 0, 0, idleTimeMilliSeconds, TimeUnit.MILLISECONDS);
-
-      @Override
-      public ChannelPipeline getPipeline() throws Exception {
-        return Channels.pipeline(RpcUtil.constructRpcFrameDecoder(),
-            RpcUtil.STAGE_RPC_MESSAGE_PARSER, idleStateHandler, handler,
-            RpcUtil.STAGE_RPC_TCP_RESPONSE);
-      }
-    });
-    tcpServer.setOption("reuseAddress", true);
-    tcpServer.setOption("child.reuseAddress", true);
-
-    udpServer = new ConnectionlessBootstrap(new NioDatagramChannelFactory(
-        Executors.newCachedThreadPool()));
-
-    udpServer.setPipeline(Channels.pipeline(RpcUtil.STAGE_RPC_MESSAGE_PARSER,
-        handler, RpcUtil.STAGE_RPC_UDP_RESPONSE));
-    udpServer.setOption("reuseAddress", true);
-
-    tcpChannel = tcpServer.bind(tcpAddress);
-    udpChannel = udpServer.bind(udpAddress);
+    bossGroup = new NioEventLoopGroup();
+    workerGroup = new NioEventLoopGroup(0, Executors.newCachedThreadPool());
+
+    tcpServer = new ServerBootstrap();
+    tcpServer.group(bossGroup, workerGroup)
+        .option(ChannelOption.SO_REUSEADDR, true)
+        .childOption(ChannelOption.SO_REUSEADDR, true)
+        .channel(NioServerSocketChannel.class)
+        .childHandler(new ChannelInitializer<SocketChannel>() {
+          private final IdleStateHandler idleStateHandler = new IdleStateHandler(
+              0, 0, idleTimeMilliSeconds, TimeUnit.MILLISECONDS);
+
+          @Override
+          protected void initChannel(SocketChannel ch) throws Exception {
+            ChannelPipeline p = ch.pipeline();
+
+            p.addLast(RpcUtil.constructRpcFrameDecoder(),
+                RpcUtil.STAGE_RPC_MESSAGE_PARSER, idleStateHandler, handler,
+                RpcUtil.STAGE_RPC_TCP_RESPONSE);
+          }});
+
+    udpGroup = new NioEventLoopGroup(0, Executors.newCachedThreadPool());
+
+    udpServer = new Bootstrap();
+    udpServer.group(udpGroup)
+        .channel(NioDatagramChannel.class)
+        .handler(new ChannelInitializer<NioDatagramChannel>() {
+          @Override protected void initChannel(NioDatagramChannel ch)
+              throws Exception {
+            ChannelPipeline p = ch.pipeline();
+            p.addLast(
+                new LoggingHandler(LogLevel.DEBUG),
+                RpcUtil.STAGE_RPC_MESSAGE_PARSER, handler, RpcUtil.STAGE_RPC_UDP_RESPONSE);
+          }
+        })
+        .option(ChannelOption.SO_REUSEADDR, true);
+
+    ChannelFuture tcpChannelFuture = null;
+    try {
+      tcpChannelFuture = tcpServer.bind(tcpAddress).sync();
+      tcpChannel = tcpChannelFuture.channel();
+
+      ChannelFuture udpChannelFuture = udpServer.bind(udpAddress).sync();
+      udpChannel = udpChannelFuture.channel();

Review comment:
       Call bind for both channels first and then sync.
   ```
         ChannelFuture tcpChannelFuture = tcpServer.bind(tcpAddress);
         ChannelFuture udpChannelFuture = udpServer.bind(udpAddress);
         tcpChannel = tcpChannelFuture.sync().channel();
         udpChannel = udpChannelFuture.sync().channel();
   ```
   

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/RpcProgramPortmap.java
##########
@@ -60,6 +60,8 @@
   private final ChannelGroup allChannels;
 
   RpcProgramPortmap(ChannelGroup allChannels) {
+    super(1, 1, 1);
+    // FIXME: set default idle timeout 1 second.

Review comment:
       Why setting timeout to 1 second?  Is this for debugging?

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java
##########
@@ -95,36 +108,59 @@ RpcProgramPortmap getHandler() {
   void start(final int idleTimeMilliSeconds, final SocketAddress tcpAddress,
       final SocketAddress udpAddress) {
 
-    tcpServer = new ServerBootstrap(new NioServerSocketChannelFactory(
-        Executors.newCachedThreadPool(), Executors.newCachedThreadPool()));
-    tcpServer.setPipelineFactory(new ChannelPipelineFactory() {
-      private final HashedWheelTimer timer = new HashedWheelTimer();
-      private final IdleStateHandler idleStateHandler = new IdleStateHandler(
-          timer, 0, 0, idleTimeMilliSeconds, TimeUnit.MILLISECONDS);
-
-      @Override
-      public ChannelPipeline getPipeline() throws Exception {
-        return Channels.pipeline(RpcUtil.constructRpcFrameDecoder(),
-            RpcUtil.STAGE_RPC_MESSAGE_PARSER, idleStateHandler, handler,
-            RpcUtil.STAGE_RPC_TCP_RESPONSE);
-      }
-    });
-    tcpServer.setOption("reuseAddress", true);
-    tcpServer.setOption("child.reuseAddress", true);
-
-    udpServer = new ConnectionlessBootstrap(new NioDatagramChannelFactory(
-        Executors.newCachedThreadPool()));
-
-    udpServer.setPipeline(Channels.pipeline(RpcUtil.STAGE_RPC_MESSAGE_PARSER,
-        handler, RpcUtil.STAGE_RPC_UDP_RESPONSE));
-    udpServer.setOption("reuseAddress", true);
-
-    tcpChannel = tcpServer.bind(tcpAddress);
-    udpChannel = udpServer.bind(udpAddress);
+    bossGroup = new NioEventLoopGroup();
+    workerGroup = new NioEventLoopGroup(0, Executors.newCachedThreadPool());
+
+    tcpServer = new ServerBootstrap();
+    tcpServer.group(bossGroup, workerGroup)
+        .option(ChannelOption.SO_REUSEADDR, true)
+        .childOption(ChannelOption.SO_REUSEADDR, true)
+        .channel(NioServerSocketChannel.class)
+        .childHandler(new ChannelInitializer<SocketChannel>() {
+          private final IdleStateHandler idleStateHandler = new IdleStateHandler(
+              0, 0, idleTimeMilliSeconds, TimeUnit.MILLISECONDS);
+
+          @Override
+          protected void initChannel(SocketChannel ch) throws Exception {
+            ChannelPipeline p = ch.pipeline();
+
+            p.addLast(RpcUtil.constructRpcFrameDecoder(),
+                RpcUtil.STAGE_RPC_MESSAGE_PARSER, idleStateHandler, handler,
+                RpcUtil.STAGE_RPC_TCP_RESPONSE);
+          }});
+
+    udpGroup = new NioEventLoopGroup(0, Executors.newCachedThreadPool());
+
+    udpServer = new Bootstrap();
+    udpServer.group(udpGroup)
+        .channel(NioDatagramChannel.class)
+        .handler(new ChannelInitializer<NioDatagramChannel>() {
+          @Override protected void initChannel(NioDatagramChannel ch)
+              throws Exception {
+            ChannelPipeline p = ch.pipeline();
+            p.addLast(
+                new LoggingHandler(LogLevel.DEBUG),
+                RpcUtil.STAGE_RPC_MESSAGE_PARSER, handler, RpcUtil.STAGE_RPC_UDP_RESPONSE);
+          }
+        })
+        .option(ChannelOption.SO_REUSEADDR, true);
+
+    ChannelFuture tcpChannelFuture = null;
+    try {
+      tcpChannelFuture = tcpServer.bind(tcpAddress).sync();
+      tcpChannel = tcpChannelFuture.channel();
+
+      ChannelFuture udpChannelFuture = udpServer.bind(udpAddress).sync();
+      udpChannel = udpChannelFuture.channel();
+
+    } catch (InterruptedException e) {
+      e.printStackTrace();

Review comment:
       Should it re-throws the exception?  Otherwise, the code below referencing tcpChannel/udpChannel may throw NullPointerException.

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServer.java
##########
@@ -103,8 +114,8 @@ public void shutdown() {
     if (ch != null) {
       ch.close().awaitUninterruptibly();
     }
-    if (server != null) {
-      server.releaseExternalResources();
-    }
+
+    workerGroup.shutdownGracefully();
+    bossGroup.shutdownGracefully();

Review comment:
       Check null, i.e.
   ```
       if (workerGroup != null) {
         workerGroup.shutdownGracefully();
       }
       if (bossGroup != null) {
         bossGroup.shutdownGracefully();
       }
   ```




-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on a change in pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
jojochuang commented on a change in pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#discussion_r615585640



##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java
##########
@@ -39,27 +37,27 @@ public SimpleTcpClientHandler(XDR request) {
   }
 
   @Override
-  public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) {
+  public void channelActive(ChannelHandlerContext ctx) throws Exception {
     // Send the request
     if (LOG.isDebugEnabled()) {
       LOG.debug("sending PRC request");
     }
-    ChannelBuffer outBuf = XDR.writeMessageTcp(request, true);
-    e.getChannel().write(outBuf);
+    ByteBuf outBuf = XDR.writeMessageTcp(request, true);
+    ctx.channel().writeAndFlush(outBuf);
   }
 
   /**
    * Shutdown connection by default. Subclass can override this method to do
    * more interaction with the server.
    */
   @Override
-  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
-    e.getChannel().close();
+  public void channelRead(ChannelHandlerContext ctx, Object msg) {
+    ctx.channel().closeFuture().awaitUninterruptibly();

Review comment:
       https://netty.io/4.1/api/io/netty/channel/ChannelFuture.html
   Do not call await() inside ChannelHandler
   The event handler methods in ChannelHandler are usually called by an I/O thread. If await() is called by an event handler method, which is called by the I/O thread, the I/O operation it is waiting for might never complete because await() can block the I/O operation it is waiting for, which is a dead lock.




-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szetszwo commented on a change in pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#discussion_r612218129



##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java
##########
@@ -62,75 +64,84 @@ public static FrameDecoder constructRpcFrameDecoder() {
    * RpcFrameDecoder is a stateful pipeline stage. It has to be constructed for
    * each RPC client.
    */
-  static class RpcFrameDecoder extends FrameDecoder {
+  static class RpcFrameDecoder extends ByteToMessageDecoder {
     public static final Logger LOG =
         LoggerFactory.getLogger(RpcFrameDecoder.class);
-    private ChannelBuffer currentFrame;
+    private boolean isLast;

Review comment:
       isLast should be volatile.

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClient.java
##########
@@ -48,40 +47,42 @@ public SimpleTcpClient(String host, int port, XDR request, Boolean oneShot) {
     this.request = request;
     this.oneShot = oneShot;
   }
-  
-  protected ChannelPipelineFactory setPipelineFactory() {
-    this.pipelineFactory = new ChannelPipelineFactory() {
+
+  protected ChannelInitializer<SocketChannel> setChannelHandler() {
+    return new ChannelInitializer<SocketChannel>() {
       @Override
-      public ChannelPipeline getPipeline() {
-        return Channels.pipeline(
+      protected void initChannel(SocketChannel ch) throws Exception {
+        ChannelPipeline p = ch.pipeline();
+        p.addLast(
             RpcUtil.constructRpcFrameDecoder(),
-            new SimpleTcpClientHandler(request));
+            new SimpleTcpClientHandler(request)
+        );
       }
     };
-    return this.pipelineFactory;
   }
 
   public void run() {
     // Configure the client.
-    ChannelFactory factory = new NioClientSocketChannelFactory(
-        Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), 1, 1);
-    ClientBootstrap bootstrap = new ClientBootstrap(factory);
-
-    // Set up the pipeline factory.
-    bootstrap.setPipelineFactory(setPipelineFactory());
+    NioEventLoopGroup workerGroup = new NioEventLoopGroup();

Review comment:
       Is run() only used in unit tests?  If not, workerGroup needs to be shutdown when oneShot == false.

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClient.java
##########
@@ -48,40 +47,42 @@ public SimpleTcpClient(String host, int port, XDR request, Boolean oneShot) {
     this.request = request;
     this.oneShot = oneShot;
   }
-  
-  protected ChannelPipelineFactory setPipelineFactory() {
-    this.pipelineFactory = new ChannelPipelineFactory() {
+
+  protected ChannelInitializer<SocketChannel> setChannelHandler() {
+    return new ChannelInitializer<SocketChannel>() {
       @Override
-      public ChannelPipeline getPipeline() {
-        return Channels.pipeline(
+      protected void initChannel(SocketChannel ch) throws Exception {
+        ChannelPipeline p = ch.pipeline();
+        p.addLast(
             RpcUtil.constructRpcFrameDecoder(),
-            new SimpleTcpClientHandler(request));
+            new SimpleTcpClientHandler(request)
+        );
       }
     };
-    return this.pipelineFactory;
   }
 
   public void run() {
     // Configure the client.
-    ChannelFactory factory = new NioClientSocketChannelFactory(
-        Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), 1, 1);
-    ClientBootstrap bootstrap = new ClientBootstrap(factory);
-
-    // Set up the pipeline factory.
-    bootstrap.setPipelineFactory(setPipelineFactory());
+    NioEventLoopGroup workerGroup = new NioEventLoopGroup();
+    Bootstrap bootstrap = new Bootstrap()
+        .group(workerGroup)
+        .channel(NioSocketChannel.class);
 
-    bootstrap.setOption("tcpNoDelay", true);
-    bootstrap.setOption("keepAlive", true);
+    try {
+      ChannelFuture future = bootstrap.handler(setChannelHandler())
+          .option(ChannelOption.TCP_NODELAY, true)
+          .option(ChannelOption.SO_KEEPALIVE, true)
+          .connect(new InetSocketAddress(host, port)).sync();
 
-    // Start the connection attempt.
-    ChannelFuture future = bootstrap.connect(new InetSocketAddress(host, port));
+      if (oneShot) {
+        // Wait until the connection is closed or the connection attempt fails.
+        future.channel().closeFuture().sync();
 
-    if (oneShot) {
-      // Wait until the connection is closed or the connection attempt fails.
-      future.getChannel().getCloseFuture().awaitUninterruptibly();
-
-      // Shut down thread pools to exit.
-      bootstrap.releaseExternalResources();
+        // Shut down thread pools to exit.
+        workerGroup.shutdownGracefully();

Review comment:
       This should be moved to a finally block.  Otherwise, it won't be shutdown in case of exceptions.

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcResponse.java
##########
@@ -19,15 +19,33 @@
 
 import java.net.SocketAddress;
 
-import org.jboss.netty.buffer.ChannelBuffer;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.DefaultAddressedEnvelope;
 
 /**
  * RpcResponse encapsulates a response to a RPC request. It contains the data
  * that is going to cross the wire, as well as the information of the remote
  * peer.
  */
-public class RpcResponse {
-  private final ChannelBuffer data;
+public class RpcResponse extends
+    DefaultAddressedEnvelope<ByteBuf, SocketAddress> {
+  public RpcResponse(ByteBuf message, SocketAddress recipient) {
+    super(message, recipient, null);
+  }
+
+  public RpcResponse(ByteBuf message, SocketAddress recipient,
+      SocketAddress sender) {
+    super(message, recipient, sender);
+  }
+
+  public ByteBuf data() {
+    return this.content();
+  }
+
+  public SocketAddress remoteAddress() {
+    return this.recipient();
+  }
+  /*private final ChannelBuffer data;

Review comment:
       Let's remove the old code?

##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java
##########
@@ -39,27 +37,27 @@ public SimpleTcpClientHandler(XDR request) {
   }
 
   @Override
-  public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) {
+  public void channelActive(ChannelHandlerContext ctx) throws Exception {
     // Send the request
     if (LOG.isDebugEnabled()) {
       LOG.debug("sending PRC request");
     }
-    ChannelBuffer outBuf = XDR.writeMessageTcp(request, true);
-    e.getChannel().write(outBuf);
+    ByteBuf outBuf = XDR.writeMessageTcp(request, true);
+    ctx.channel().writeAndFlush(outBuf);
   }
 
   /**
    * Shutdown connection by default. Subclass can override this method to do
    * more interaction with the server.
    */
   @Override
-  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
-    e.getChannel().close();
+  public void channelRead(ChannelHandlerContext ctx, Object msg) {
+    ctx.channel().closeFuture().awaitUninterruptibly();

Review comment:
       If should call close() instead of closeFuture() since closeFuture() just returns the future but won't close the channel.




-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on a change in pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
jojochuang commented on a change in pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#discussion_r615587865



##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java
##########
@@ -39,27 +37,27 @@ public SimpleTcpClientHandler(XDR request) {
   }
 
   @Override
-  public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) {
+  public void channelActive(ChannelHandlerContext ctx) throws Exception {
     // Send the request
     if (LOG.isDebugEnabled()) {
       LOG.debug("sending PRC request");
     }
-    ChannelBuffer outBuf = XDR.writeMessageTcp(request, true);
-    e.getChannel().write(outBuf);
+    ByteBuf outBuf = XDR.writeMessageTcp(request, true);
+    ctx.channel().writeAndFlush(outBuf);
   }
 
   /**
    * Shutdown connection by default. Subclass can override this method to do
    * more interaction with the server.
    */
   @Override
-  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
-    e.getChannel().close();
+  public void channelRead(ChannelHandlerContext ctx, Object msg) {
+    ctx.channel().closeFuture().awaitUninterruptibly();

Review comment:
       I'll rewrite this part according to the official doc.




-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang merged pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
jojochuang merged pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832


   


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-815874732


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  3s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  14m 30s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  22m  3s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  22m 42s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  18m 45s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   3m 43s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 43s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 12s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 44s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m  4s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 57s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 57s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 38s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |  20m 38s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m 11s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |  18m 11s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   3m 42s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/5/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 23 new + 175 unchanged - 10 fixed = 198 total (was 185)  |
   | +1 :green_heart: |  mvnsite  |   2m 55s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  4s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   2m 24s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 51s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m 43s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 30s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 53s |  |  hadoop-nfs in the patch passed.  |
   | -1 :x: |  unit  | 476m 39s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/5/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  unit  |   4m 22s |  |  hadoop-hdfs-nfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m 26s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 672m 37s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.namenode.TestDecommissioningStatus |
   |   | hadoop.hdfs.TestViewDistributedFileSystemContract |
   |   | hadoop.hdfs.TestSnapshotCommands |
   |   | hadoop.hdfs.TestPersistBlocks |
   |   | hadoop.hdfs.TestDFSShell |
   |   | hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList |
   |   | hadoop.hdfs.TestReconstructStripedFile |
   |   | hadoop.hdfs.server.mover.TestMover |
   |   | hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeWithHdfsScheme |
   |   | hadoop.hdfs.TestStateAlignmentContextWithHA |
   |   | hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeHdfsFileSystemContract |
   |   | hadoop.hdfs.server.namenode.ha.TestSeveralNameNodes |
   |   | hadoop.hdfs.server.namenode.TestNamenodeStorageDirectives |
   |   | hadoop.fs.viewfs.TestViewFSOverloadSchemeWithMountTableConfigInHDFS |
   |   | hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor |
   |   | hadoop.hdfs.TestBlocksScheduledCounter |
   |   | hadoop.hdfs.TestReconstructStripedFileWithValidator |
   |   | hadoop.hdfs.server.namenode.ha.TestEditLogTailer |
   |   | hadoop.hdfs.server.datanode.TestBlockScanner |
   |   | hadoop.hdfs.server.diskbalancer.command.TestDiskBalancerCommand |
   |   | hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots |
   |   | hadoop.hdfs.qjournal.server.TestJournalNodeRespectsBindHostKeys |
   |   | hadoop.hdfs.server.datanode.TestDirectoryScanner |
   |   | hadoop.hdfs.TestHDFSFileSystemContract |
   |   | hadoop.hdfs.server.namenode.ha.TestBootstrapStandby |
   |   | hadoop.hdfs.web.TestWebHdfsFileSystemContract |
   |   | hadoop.hdfs.server.datanode.TestDataNodeErasureCodingMetrics |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2832 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle |
   | uname | Linux 9eed1d5e6a6d 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 107259e88d9d5674bca0f08801b9520b3d79675a |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/5/testReport/ |
   | Max. process+thread count | 2827 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-nfs hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/5/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
jojochuang commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-828980000


   Thanks Nicholas!


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
jojochuang commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-828041944


   @szetszwo did you get the chance to review the update? Thanks again!


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on a change in pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
jojochuang commented on a change in pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#discussion_r615609787



##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java
##########
@@ -62,75 +64,84 @@ public static FrameDecoder constructRpcFrameDecoder() {
    * RpcFrameDecoder is a stateful pipeline stage. It has to be constructed for
    * each RPC client.
    */
-  static class RpcFrameDecoder extends FrameDecoder {
+  static class RpcFrameDecoder extends ByteToMessageDecoder {
     public static final Logger LOG =
         LoggerFactory.getLogger(RpcFrameDecoder.class);
-    private ChannelBuffer currentFrame;
+    private boolean isLast;

Review comment:
       i can change it to volatile, but the RpcUtil object is created for each channel, it's not reused. I don't think this is needed.




-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-814259390


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  3s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  14m 20s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m  2s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  21m 11s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  19m 45s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   4m 15s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 44s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 13s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 52s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   4m 59s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 23s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 52s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  21m  1s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |  21m  1s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m 14s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |  18m 14s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   3m 45s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/3/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 23 new + 175 unchanged - 10 fixed = 198 total (was 185)  |
   | +1 :green_heart: |  mvnsite  |   2m 52s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  4s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   2m 14s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 41s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m 40s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 37s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 55s |  |  hadoop-nfs in the patch passed.  |
   | -1 :x: |  unit  | 434m 18s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/3/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 42s |  |  hadoop-hdfs-nfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  7s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 627m 19s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.namenode.TestDecommissioningStatus |
   |   | hadoop.hdfs.TestViewDistributedFileSystemContract |
   |   | hadoop.hdfs.TestSnapshotCommands |
   |   | hadoop.hdfs.TestPersistBlocks |
   |   | hadoop.hdfs.TestDFSShell |
   |   | hadoop.hdfs.TestLeaseRecovery |
   |   | hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList |
   |   | hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeWithHdfsScheme |
   |   | hadoop.hdfs.TestStateAlignmentContextWithHA |
   |   | hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeHdfsFileSystemContract |
   |   | hadoop.hdfs.TestViewDistributedFileSystemWithMountLinks |
   |   | hadoop.hdfs.server.namenode.ha.TestSeveralNameNodes |
   |   | hadoop.fs.viewfs.TestViewFSOverloadSchemeWithMountTableConfigInHDFS |
   |   | hadoop.hdfs.server.namenode.TestFileTruncate |
   |   | hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor |
   |   | hadoop.hdfs.TestLeaseRecovery2 |
   |   | hadoop.hdfs.TestBlocksScheduledCounter |
   |   | hadoop.hdfs.server.namenode.ha.TestEditLogTailer |
   |   | hadoop.hdfs.server.datanode.TestBlockScanner |
   |   | hadoop.hdfs.server.diskbalancer.command.TestDiskBalancerCommand |
   |   | hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots |
   |   | hadoop.hdfs.qjournal.server.TestJournalNodeRespectsBindHostKeys |
   |   | hadoop.hdfs.server.datanode.TestDirectoryScanner |
   |   | hadoop.hdfs.server.datanode.TestBlockRecovery2 |
   |   | hadoop.hdfs.TestHDFSFileSystemContract |
   |   | hadoop.hdfs.server.namenode.ha.TestBootstrapStandby |
   |   | hadoop.hdfs.web.TestWebHdfsFileSystemContract |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2832 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle |
   | uname | Linux 4a96a6bdd7b5 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / e46e5d7313e87db47c0602472aa6c51efe605a86 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/3/testReport/ |
   | Max. process+thread count | 2526 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-nfs hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/3/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-815980307


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  1s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  14m 12s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m  8s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  23m 16s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  19m 54s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   4m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   3m  2s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 31s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 58s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m 12s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 27s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 57s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  22m 27s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |  22m 27s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  19m 26s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |  19m 26s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   3m 52s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/6/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 23 new + 175 unchanged - 10 fixed = 198 total (was 185)  |
   | +1 :green_heart: |  mvnsite  |   2m 53s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  4s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   2m 22s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 50s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m 50s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  16m 12s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 58s |  |  hadoop-nfs in the patch passed.  |
   | -1 :x: |  unit  | 456m  3s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/6/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 46s |  |  hadoop-hdfs-nfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  7s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 658m 41s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.namenode.TestDecommissioningStatus |
   |   | hadoop.hdfs.TestViewDistributedFileSystemContract |
   |   | hadoop.hdfs.TestSnapshotCommands |
   |   | hadoop.hdfs.TestPersistBlocks |
   |   | hadoop.hdfs.TestDFSShell |
   |   | hadoop.hdfs.TestWriteConfigurationToDFS |
   |   | hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList |
   |   | hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeWithHdfsScheme |
   |   | hadoop.hdfs.TestStateAlignmentContextWithHA |
   |   | hadoop.hdfs.server.namenode.ha.TestSeveralNameNodes |
   |   | hadoop.hdfs.server.namenode.TestNamenodeStorageDirectives |
   |   | hadoop.fs.viewfs.TestViewFSOverloadSchemeWithMountTableConfigInHDFS |
   |   | hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor |
   |   | hadoop.hdfs.TestLeaseRecovery2 |
   |   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
   |   | hadoop.hdfs.server.namenode.ha.TestEditLogTailer |
   |   | hadoop.hdfs.server.datanode.TestBlockScanner |
   |   | hadoop.hdfs.server.diskbalancer.command.TestDiskBalancerCommand |
   |   | hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots |
   |   | hadoop.hdfs.server.namenode.TestNNThroughputBenchmark |
   |   | hadoop.hdfs.qjournal.server.TestJournalNodeRespectsBindHostKeys |
   |   | hadoop.hdfs.server.datanode.TestDirectoryScanner |
   |   | hadoop.hdfs.TestHDFSFileSystemContract |
   |   | hadoop.hdfs.server.namenode.ha.TestBootstrapStandby |
   |   | hadoop.hdfs.web.TestWebHdfsFileSystemContract |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2832 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle |
   | uname | Linux 5cd0ed6cef5c 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 790d67508cc5c3ad3659a2526fb3239e92ff9f68 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/6/testReport/ |
   | Max. process+thread count | 3066 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-nfs hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/6/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szetszwo commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-818602960


   > ... verified to contain no memory leak via -Dio.netty.leakDetectionLevel=paranoid.
   
   Hi @jojochuang , according to https://netty.io/wiki/reference-counted-objects.html , the option is renamed to `io.netty.leakDetection.Level`; see the NOTE below.
   <img width="858" alt="image" src="https://user-images.githubusercontent.com/907380/114532453-6ee1bf00-9c7f-11eb-82fe-f43bffc3cae8.png">
   Not sure if the old option name still works.


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-809528708


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  13m 16s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  14m 11s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  20m  8s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 46s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  18m  6s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   3m 51s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   3m  2s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 29s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 54s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m 10s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 10s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 56s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m  2s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |  20m  2s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  17m 58s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |  17m 58s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   3m 45s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/1/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 23 new + 173 unchanged - 10 fixed = 196 total (was 183)  |
   | +1 :green_heart: |  mvnsite  |   3m  0s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  4s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   2m 28s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 58s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m 42s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 14s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  0s |  |  hadoop-nfs in the patch passed.  |
   | -1 :x: |  unit  | 230m 28s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/1/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 10s |  |  hadoop-hdfs-nfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  7s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 431m 36s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.TestDFSClientExcludedNodes |
   |   | hadoop.hdfs.server.datanode.TestBlockRecovery |
   |   | hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots |
   |   | hadoop.hdfs.TestRollingUpgrade |
   |   | hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks |
   |   | hadoop.hdfs.server.datanode.TestDirectoryScanner |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2832 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle |
   | uname | Linux 795d2c780461 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 7b60b9945a2b0f67a35f5d94909fadba103e88d5 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/1/testReport/ |
   | Max. process+thread count | 3228 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-nfs hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/1/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on a change in pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
jojochuang commented on a change in pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#discussion_r615608993



##########
File path: hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClient.java
##########
@@ -48,40 +47,42 @@ public SimpleTcpClient(String host, int port, XDR request, Boolean oneShot) {
     this.request = request;
     this.oneShot = oneShot;
   }
-  
-  protected ChannelPipelineFactory setPipelineFactory() {
-    this.pipelineFactory = new ChannelPipelineFactory() {
+
+  protected ChannelInitializer<SocketChannel> setChannelHandler() {
+    return new ChannelInitializer<SocketChannel>() {
       @Override
-      public ChannelPipeline getPipeline() {
-        return Channels.pipeline(
+      protected void initChannel(SocketChannel ch) throws Exception {
+        ChannelPipeline p = ch.pipeline();
+        p.addLast(
             RpcUtil.constructRpcFrameDecoder(),
-            new SimpleTcpClientHandler(request));
+            new SimpleTcpClientHandler(request)
+        );
       }
     };
-    return this.pipelineFactory;
   }
 
   public void run() {
     // Configure the client.
-    ChannelFactory factory = new NioClientSocketChannelFactory(
-        Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), 1, 1);
-    ClientBootstrap bootstrap = new ClientBootstrap(factory);
-
-    // Set up the pipeline factory.
-    bootstrap.setPipelineFactory(setPipelineFactory());
+    NioEventLoopGroup workerGroup = new NioEventLoopGroup();

Review comment:
       it's actually used by UT only.  I'll add @VisibleForTesting to make it clear.
   (The only exception is TestOutOfOrderWrite class. It is a test but written in a standalone application style, not in JUnit style. So, still a test)




-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-814958017


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 40s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  14m 29s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m 24s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 51s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  18m  0s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   3m 51s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 59s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 30s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 56s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m 11s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 14s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 50s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 17s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |  20m 17s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m  5s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |  18m  5s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   3m 44s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/4/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 23 new + 173 unchanged - 10 fixed = 196 total (was 183)  |
   | +1 :green_heart: |  mvnsite  |   2m 56s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  4s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   2m 28s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   3m  0s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   5m 38s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 30s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 59s |  |  hadoop-nfs in the patch passed.  |
   | -1 :x: |  unit  | 233m 58s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/4/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m  9s |  |  hadoop-hdfs-nfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  7s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 424m 49s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hdfs.server.namenode.TestFsck |
   |   | hadoop.hdfs.server.datanode.TestBlockRecovery2 |
   |   | hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots |
   |   | hadoop.hdfs.TestRollingUpgrade |
   |   | hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks |
   |   | hadoop.hdfs.server.datanode.TestDirectoryScanner |
   |   | hadoop.hdfs.qjournal.server.TestJournalNodeRespectsBindHostKeys |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2832 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle |
   | uname | Linux bc156ffa1e84 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 2d4d4373a4c05095dc3d60abcfa4ae5687abb9b0 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/4/testReport/ |
   | Max. process+thread count | 3406 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-nfs hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2832/4/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on pull request #2832: HADOOP-11245. Update NFS gateway to use Netty4

Posted by GitBox <gi...@apache.org>.
jojochuang commented on pull request #2832:
URL: https://github.com/apache/hadoop/pull/2832#issuecomment-818384841


   As of the last commit, the code was deployed on a small cluster, verified to contain no memory leak via -Dio.netty.leakDetectionLevel=paranoid.
   
   The performance saw no noticeable change: prior to the change, the throughput is 39.3MB/s; after the change, the throughput is 38.2MB/s (writing a 1GB file)


-- 
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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org