You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2020/12/14 17:40:24 UTC

[GitHub] [hive] abstractdog opened a new pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

abstractdog opened a new pull request #1778:
URL: https://github.com/apache/hive/pull/1778


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/Hive/HowToContribute
     2. Ensure that you have created an issue on the Hive project JIRA: https://issues.apache.org/jira/projects/HIVE/summary
     3. Ensure you have added or run the appropriate tests for your PR: 
     4. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]HIVE-XXXXX:  Your PR title ...'.
     5. Be sure to keep the PR description updated to reflect all changes.
     6. Please write your PR title to summarize what this PR proposes.
     7. If possible, provide a concise example to reproduce the issue for a faster review.
   
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description, screenshot and/or a reproducable example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Hive versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   


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


[GitHub] [hive] pgaref commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
pgaref commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612347286



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
##########
@@ -797,16 +803,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
 
       Map<String, MapOutputInfo> mapOutputInfoMap =
           new HashMap<String, MapOutputInfo>();
-      Channel ch = evt.getChannel();
-
+      Channel ch = ctx.channel();
       // In case of KeepAlive, ensure that timeout handler does not close connection until entire
       // response is written (i.e, response headers + mapOutput).
-      ChannelPipeline pipeline = ch.getPipeline();
+      ChannelPipeline pipeline = ch.pipeline();
       TimeoutHandler timeoutHandler = (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER);
       timeoutHandler.setEnabledTimeout(false);
 
       String user = userRsrc.get(jobId);
-
+      if (keepAliveParam || connectionKeepAliveEnabled){

Review comment:
       Thanks Laszlo! sounds like a plan! 




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


[GitHub] [hive] abstractdog commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
abstractdog commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612343543



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
##########
@@ -797,16 +803,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
 
       Map<String, MapOutputInfo> mapOutputInfoMap =
           new HashMap<String, MapOutputInfo>();
-      Channel ch = evt.getChannel();
-
+      Channel ch = ctx.channel();
       // In case of KeepAlive, ensure that timeout handler does not close connection until entire
       // response is written (i.e, response headers + mapOutput).
-      ChannelPipeline pipeline = ch.getPipeline();
+      ChannelPipeline pipeline = ch.pipeline();
       TimeoutHandler timeoutHandler = (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER);
       timeoutHandler.setEnabledTimeout(false);
 
       String user = userRsrc.get(jobId);
-
+      if (keepAliveParam || connectionKeepAliveEnabled){

Review comment:
       okay, in this case I'll have to include some unit tests here (which might be part of tez code already) + create a simple repro to share with netty community




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


[GitHub] [hive] pgaref commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
pgaref commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612329298



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/FadvisedFileRegion.java
##########
@@ -71,15 +72,39 @@ public long transferTo(WritableByteChannel target, long position)
       throws IOException {
     if (manageOsCache && readaheadPool != null) {
       readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
-          getPosition() + position, readaheadLength,
-          getPosition() + getCount(), readaheadRequest);
+          position() + position, readaheadLength,
+          position() + count(), readaheadRequest);
     }
-    
+    long written = 0;

Review comment:
       Got 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: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] abstractdog commented on pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
abstractdog commented on pull request #1778:
URL: https://github.com/apache/hive/pull/1778#issuecomment-823871712


   merged to master, thanks @pgaref for the 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: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] abstractdog commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
abstractdog commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612326445



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/FadvisedFileRegion.java
##########
@@ -124,39 +149,33 @@ long customShuffleTransfer(WritableByteChannel target, long position)
         position += trans; 
         trans = 0;
       }
-      
+
       //write data to the target
       while(byteBuffer.hasRemaining()) {
         target.write(byteBuffer);
       }
       
       byteBuffer.clear();
     }
-    
+
     return actualCount - trans;
   }
 
-  
-  @Override
-  public void releaseExternalResources() {
-    if (readaheadRequest != null) {
-      readaheadRequest.cancel();
-    }
-    super.releaseExternalResources();
-  }
-  
   /**
    * Call when the transfer completes successfully so we can advise the OS that
    * we don't need the region to be cached anymore.
    */
   public void transferSuccessful() {
-    if (manageOsCache && getCount() > 0) {
+    if (manageOsCache && count() > 0) {
       try {
         if (canEvictAfterTransfer) {
-          LOG.debug("shuffleBufferSize: {}, path: {}", shuffleBufferSize, identifier);
-          NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
-              fd, getPosition(), getCount(),
-              NativeIO.POSIX.POSIX_FADV_DONTNEED);
+          if (fd.valid()) {

Review comment:
       hm, thought this over again, fd.valid() change was needed while I haven't been handling deallocate() stuff properly, but now, at this point fd should be valid...initially I left this check here because I thought that an invalid fd is not a problem (which is true, we won't advise to OS cache, and that's it), but as we already have try/catch, we don't need this this check (we'll have the exception in the logs anyway)




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


[GitHub] [hive] abstractdog commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
abstractdog commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612330581



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
##########
@@ -797,16 +803,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
 
       Map<String, MapOutputInfo> mapOutputInfoMap =
           new HashMap<String, MapOutputInfo>();
-      Channel ch = evt.getChannel();
-
+      Channel ch = ctx.channel();
       // In case of KeepAlive, ensure that timeout handler does not close connection until entire
       // response is written (i.e, response headers + mapOutput).
-      ChannelPipeline pipeline = ch.getPipeline();
+      ChannelPipeline pipeline = ch.pipeline();
       TimeoutHandler timeoutHandler = (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER);
       timeoutHandler.setEnabledTimeout(false);
 
       String user = userRsrc.get(jobId);
-
+      if (keepAliveParam || connectionKeepAliveEnabled){

Review comment:
       good catch :) this is an epic workaround that I haven't been able to figure out, here are some details:
   https://issues.apache.org/jira/browse/TEZ-4157?focusedCommentId=17100835&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17100835
   
   (btw: with netty3, we didn't need this)
   
   are you fine with a comment explaining this?




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


[GitHub] [hive] pgaref commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
pgaref commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612334078



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
##########
@@ -797,16 +803,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
 
       Map<String, MapOutputInfo> mapOutputInfoMap =
           new HashMap<String, MapOutputInfo>();
-      Channel ch = evt.getChannel();
-
+      Channel ch = ctx.channel();
       // In case of KeepAlive, ensure that timeout handler does not close connection until entire
       // response is written (i.e, response headers + mapOutput).
-      ChannelPipeline pipeline = ch.getPipeline();
+      ChannelPipeline pipeline = ch.pipeline();
       TimeoutHandler timeoutHandler = (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER);
       timeoutHandler.setEnabledTimeout(false);
 
       String user = userRsrc.get(jobId);
-
+      if (keepAliveParam || connectionKeepAliveEnabled){

Review comment:
       Got it, this is helpful but lets make sure this is expected from nettys' side of things before committing -- this would be helpful for the Tez change as well :) 




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


[GitHub] [hive] abstractdog merged pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
abstractdog merged pull request #1778:
URL: https://github.com/apache/hive/pull/1778


   


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


[GitHub] [hive] abstractdog commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
abstractdog commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612328280



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/FadvisedFileRegion.java
##########
@@ -71,15 +72,39 @@ public long transferTo(WritableByteChannel target, long position)
       throws IOException {
     if (manageOsCache && readaheadPool != null) {
       readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
-          getPosition() + position, readaheadLength,
-          getPosition() + getCount(), readaheadRequest);
+          position() + position, readaheadLength,
+          position() + count(), readaheadRequest);
     }
-    
+    long written = 0;

Review comment:
       looks better, but I don't think it's correct: in case of an exception during the transfer, we should not have set transferred=true




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


[GitHub] [hive] abstractdog commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
abstractdog commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612330581



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
##########
@@ -797,16 +803,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
 
       Map<String, MapOutputInfo> mapOutputInfoMap =
           new HashMap<String, MapOutputInfo>();
-      Channel ch = evt.getChannel();
-
+      Channel ch = ctx.channel();
       // In case of KeepAlive, ensure that timeout handler does not close connection until entire
       // response is written (i.e, response headers + mapOutput).
-      ChannelPipeline pipeline = ch.getPipeline();
+      ChannelPipeline pipeline = ch.pipeline();
       TimeoutHandler timeoutHandler = (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER);
       timeoutHandler.setEnabledTimeout(false);
 
       String user = userRsrc.get(jobId);
-
+      if (keepAliveParam || connectionKeepAliveEnabled){

Review comment:
       good catch :) this is an epic workaround for a problem that I haven't been able to figure out 100%, here are some details:
   https://issues.apache.org/jira/browse/TEZ-4157?focusedCommentId=17100835&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17100835
   
   (btw: with netty3, we didn't need this)
   
   are you fine with a comment explaining this?




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


[GitHub] [hive] pgaref commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
pgaref commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612289762



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/FadvisedFileRegion.java
##########
@@ -71,15 +72,39 @@ public long transferTo(WritableByteChannel target, long position)
       throws IOException {
     if (manageOsCache && readaheadPool != null) {
       readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
-          getPosition() + position, readaheadLength,
-          getPosition() + getCount(), readaheadRequest);
+          position() + position, readaheadLength,
+          position() + count(), readaheadRequest);
     }
-    
+    long written = 0;

Review comment:
       Shall we simplify this to:
   
   ```
   transferred = true;
   if (this.shuffleTransferToAllowed) {
         return super.transferTo(target, position);
   }
   return  customShuffleTransfer(target, position);
   ```
     
   
   

##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
##########
@@ -339,27 +350,60 @@ private ShuffleHandler(Configuration conf) {
 
 
   public void start() throws Exception {
-    ServerBootstrap bootstrap = new ServerBootstrap(selector);
-    // Timer is shared across entire factory and must be released separately
-    timer = new HashedWheelTimer();
-    try {
-      pipelineFact = new HttpPipelineFactory(conf, timer);
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-    bootstrap.setPipelineFactory(pipelineFact);
-    bootstrap.setOption("backlog", NetUtil.SOMAXCONN);
+    ServerBootstrap bootstrap = new ServerBootstrap()
+        .channel(NioServerSocketChannel.class)
+        .group(bossGroup, workerGroup)
+        .localAddress(port)
+        .option(ChannelOption.SO_BACKLOG, NetUtil.SOMAXCONN)
+        .childOption(ChannelOption.SO_KEEPALIVE, true);
+    initPipeline(bootstrap, conf);
+
     port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
-    Channel ch = bootstrap.bind(new InetSocketAddress(port));
+    Channel ch = bootstrap.bind().sync().channel();
     accepted.add(ch);
-    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+    port = ((InetSocketAddress)ch.localAddress()).getPort();
     conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
-    pipelineFact.SHUFFLE.setPort(port);
+    SHUFFLE.setPort(port);
     if (dirWatcher != null) {
       dirWatcher.start();
     }
-    LOG.info("LlapShuffleHandler" + " listening on port " + port + " (SOMAXCONN: " + bootstrap.getOption("backlog")
-      + ")");
+    LOG.info("LlapShuffleHandler listening on port {} (SOMAXCONN: {})", port, NetUtil.SOMAXCONN);
+  }
+
+  private void initPipeline(ServerBootstrap bootstrap, Configuration conf) throws Exception {
+    SHUFFLE = getShuffle(conf);
+    // TODO Setup SSL Shuffle

Review comment:
       I know this is copy pasted from below but do we have a ticket for this?
   Is it still needed?

##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
##########
@@ -1031,25 +1038,14 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
             info.getStartOffset(), info.getPartLength(), manageOsCache, readaheadLength,
             readaheadPool, spillfile.getAbsolutePath(), 
             shuffleBufferSize, shuffleTransferToAllowed, canEvictAfterTransfer);
-        writeFuture = ch.write(partition);
-        writeFuture.addListener(new ChannelFutureListener() {
-            // TODO error handling; distinguish IO/connection failures,
-            //      attribute to appropriate spill output
-          @Override
-          public void operationComplete(ChannelFuture future) {
-            if (future.isSuccess()) {
-              partition.transferSuccessful();
-            }
-            partition.releaseExternalResources();
-          }
-        });
+        writeFuture = ch.writeAndFlush(partition);

Review comment:
       This looks much cleaner with deallocate() call replacing completion Listeners

##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/FadvisedFileRegion.java
##########
@@ -124,39 +149,33 @@ long customShuffleTransfer(WritableByteChannel target, long position)
         position += trans; 
         trans = 0;
       }
-      
+
       //write data to the target
       while(byteBuffer.hasRemaining()) {
         target.write(byteBuffer);
       }
       
       byteBuffer.clear();
     }
-    
+
     return actualCount - trans;
   }
 
-  
-  @Override
-  public void releaseExternalResources() {
-    if (readaheadRequest != null) {
-      readaheadRequest.cancel();
-    }
-    super.releaseExternalResources();
-  }
-  
   /**
    * Call when the transfer completes successfully so we can advise the OS that
    * we don't need the region to be cached anymore.
    */
   public void transferSuccessful() {
-    if (manageOsCache && getCount() > 0) {
+    if (manageOsCache && count() > 0) {
       try {
         if (canEvictAfterTransfer) {
-          LOG.debug("shuffleBufferSize: {}, path: {}", shuffleBufferSize, identifier);
-          NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
-              fd, getPosition(), getCount(),
-              NativeIO.POSIX.POSIX_FADV_DONTNEED);
+          if (fd.valid()) {

Review comment:
       Shall we keep the original log message for advancing the file descriptor as well here?
   Why do we need the extra fd.valid check here? (maybe leave a comment?)

##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
##########
@@ -797,16 +803,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
 
       Map<String, MapOutputInfo> mapOutputInfoMap =
           new HashMap<String, MapOutputInfo>();
-      Channel ch = evt.getChannel();
-
+      Channel ch = ctx.channel();
       // In case of KeepAlive, ensure that timeout handler does not close connection until entire
       // response is written (i.e, response headers + mapOutput).
-      ChannelPipeline pipeline = ch.getPipeline();
+      ChannelPipeline pipeline = ch.pipeline();
       TimeoutHandler timeoutHandler = (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER);
       timeoutHandler.setEnabledTimeout(false);
 
       String user = userRsrc.get(jobId);
-
+      if (keepAliveParam || connectionKeepAliveEnabled){

Review comment:
       Why is this need now? How was connection kept alive before ?




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


[GitHub] [hive] abstractdog commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
abstractdog commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612323669



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
##########
@@ -339,27 +350,60 @@ private ShuffleHandler(Configuration conf) {
 
 
   public void start() throws Exception {
-    ServerBootstrap bootstrap = new ServerBootstrap(selector);
-    // Timer is shared across entire factory and must be released separately
-    timer = new HashedWheelTimer();
-    try {
-      pipelineFact = new HttpPipelineFactory(conf, timer);
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-    bootstrap.setPipelineFactory(pipelineFact);
-    bootstrap.setOption("backlog", NetUtil.SOMAXCONN);
+    ServerBootstrap bootstrap = new ServerBootstrap()
+        .channel(NioServerSocketChannel.class)
+        .group(bossGroup, workerGroup)
+        .localAddress(port)
+        .option(ChannelOption.SO_BACKLOG, NetUtil.SOMAXCONN)
+        .childOption(ChannelOption.SO_KEEPALIVE, true);
+    initPipeline(bootstrap, conf);
+
     port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
-    Channel ch = bootstrap.bind(new InetSocketAddress(port));
+    Channel ch = bootstrap.bind().sync().channel();
     accepted.add(ch);
-    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+    port = ((InetSocketAddress)ch.localAddress()).getPort();
     conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
-    pipelineFact.SHUFFLE.setPort(port);
+    SHUFFLE.setPort(port);
     if (dirWatcher != null) {
       dirWatcher.start();
     }
-    LOG.info("LlapShuffleHandler" + " listening on port " + port + " (SOMAXCONN: " + bootstrap.getOption("backlog")
-      + ")");
+    LOG.info("LlapShuffleHandler listening on port {} (SOMAXCONN: {})", port, NetUtil.SOMAXCONN);
+  }
+
+  private void initPipeline(ServerBootstrap bootstrap, Configuration conf) throws Exception {
+    SHUFFLE = getShuffle(conf);
+    // TODO Setup SSL Shuffle

Review comment:
       I think we don't support SSL shuffle for LLAP at the moment (+ the comment is quite old), e.g. Cloudera's data warehouse, ssl on shuffle is handled transparently by the environment
   I haven't touched this part in this patch, and not even sure what's the plan :) that's why I simply kept this as is




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


[GitHub] [hive] github-actions[bot] commented on pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #1778:
URL: https://github.com/apache/hive/pull/1778#issuecomment-792393727


   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed if no further activity occurs.
   Feel free to reach out on the dev@hive.apache.org list if the patch is in need of reviews.


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


[GitHub] [hive] abstractdog commented on a change in pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
abstractdog commented on a change in pull request #1778:
URL: https://github.com/apache/hive/pull/1778#discussion_r612343543



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java
##########
@@ -797,16 +803,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
 
       Map<String, MapOutputInfo> mapOutputInfoMap =
           new HashMap<String, MapOutputInfo>();
-      Channel ch = evt.getChannel();
-
+      Channel ch = ctx.channel();
       // In case of KeepAlive, ensure that timeout handler does not close connection until entire
       // response is written (i.e, response headers + mapOutput).
-      ChannelPipeline pipeline = ch.getPipeline();
+      ChannelPipeline pipeline = ch.pipeline();
       TimeoutHandler timeoutHandler = (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER);
       timeoutHandler.setEnabledTimeout(false);
 
       String user = userRsrc.get(jobId);
-
+      if (keepAliveParam || connectionKeepAliveEnabled){

Review comment:
       okay, in this case I'll have to include some unit tests here (which are part of tez codebase already) + create a simple repro to share with netty community




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


[GitHub] [hive] github-actions[bot] closed pull request #1778: HIVE-24524: LLAP ShuffleHandler: upgrade to netty4

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #1778:
URL: https://github.com/apache/hive/pull/1778


   


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