[ https://issues.apache.org/jira/browse/HADOOP-15327?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17631245#comment-17631245 ]
ASF GitHub Bot commented on HADOOP-15327: ----------------------------------------- 9uapaw commented on code in PR #3259: URL: https://github.com/apache/hadoop/pull/3259#discussion_r1018347131 ########## hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java: ########## @@ -291,36 +302,86 @@ public void operationComplete(ChannelFuture future) throws Exception { } } + static class NettyChannelHelper { + static ChannelFuture writeToChannel(Channel ch, Object obj) { + LOG.debug("Writing {} to channel: {}", obj.getClass().getSimpleName(), ch.id()); + return ch.writeAndFlush(obj); + } + + static ChannelFuture writeToChannelAndClose(Channel ch, Object obj) { + return writeToChannel(ch, obj).addListener(ChannelFutureListener.CLOSE); + } + + static ChannelFuture writeToChannelAndAddLastHttpContent(Channel ch, HttpResponse obj) { + writeToChannel(ch, obj); + return writeLastHttpContentToChannel(ch); + } + + static ChannelFuture writeLastHttpContentToChannel(Channel ch) { + LOG.debug("Writing LastHttpContent, channel id: {}", ch.id()); + return ch.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); + } + + static ChannelFuture closeChannel(Channel ch) { + LOG.debug("Closing channel, channel id: {}", ch.id()); + return ch.close(); + } + + static void closeChannels(ChannelGroup channelGroup) { + channelGroup.close().awaitUninterruptibly(10, TimeUnit.SECONDS); + } + + public static ChannelFuture closeAsIdle(Channel channel, int timeout) { + LOG.debug("Closing channel as writer was idle for {} seconds", timeout); + return closeChannel(channel); + } + + public static void channelActive(Channel ch) { + LOG.debug("Executing channelActive, channel id: {}", ch.id()); + } + + public static void channelInactive(Channel channel) { + LOG.debug("Executing channelInactive, channel id: {}", channel.id()); + } Review Comment: We need an other patch anyway, so we might as well fix these to be more consistent. > Upgrade MR ShuffleHandler to use Netty4 > --------------------------------------- > > Key: HADOOP-15327 > URL: https://issues.apache.org/jira/browse/HADOOP-15327 > Project: Hadoop Common > Issue Type: Sub-task > Reporter: Xiaoyu Yao > Assignee: Szilard Nemeth > Priority: Major > Labels: pull-request-available > Attachments: HADOOP-15327.001.patch, HADOOP-15327.002.patch, > HADOOP-15327.003.patch, HADOOP-15327.004.patch, HADOOP-15327.005.patch, > HADOOP-15327.005.patch, > getMapOutputInfo_BlockingOperationException_awaitUninterruptibly.log, > hades-results-20221108.zip, testfailure-testMapFileAccess-emptyresponse.zip, > testfailure-testReduceFromPartialMem.zip > > Time Spent: 11.5h > Remaining Estimate: 0h > > This way, we can remove the dependencies on the netty3 (jboss.netty) -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org