Giraph
  1. Giraph
  2. GIRAPH-437

Missing progress calls when stopping Netty server

    Details

    • Type: Improvement Improvement
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: None
    • Component/s: None
    • Labels:
      None

      Description

      At the end of a long running job I got an exception about not reporting progress. The last log line was: "stop: Halting netty server", so I suspect it's because awaitUninterruptibly() call there.

      1. GIRAPH-437.patch
        22 kB
        Maja Kabiljo

        Activity

        Maja Kabiljo created issue -
        Maja Kabiljo made changes -
        Field Original Value New Value
        Assignee Maja Kabiljo [ majakabiljo ]
        Hide
        Maja Kabiljo added a comment -

        Changing awaitUninterruptibly to periodical calls of await. Added a log line to the end of NettyServer.stop so if it happens again we can be sure where the problem is. I also refactored ProgressableUtils so new cases where we need to wait for something will be easier to write.

        Passes mvn verify.

        Show
        Maja Kabiljo added a comment - Changing awaitUninterruptibly to periodical calls of await. Added a log line to the end of NettyServer.stop so if it happens again we can be sure where the problem is. I also refactored ProgressableUtils so new cases where we need to wait for something will be easier to write. Passes mvn verify.
        Maja Kabiljo made changes -
        Attachment GIRAPH-437.patch [ 12555387 ]
        Maja Kabiljo made changes -
        Status Open [ 1 ] Patch Available [ 10002 ]
        Hide
        Avery Ching added a comment -

        This is useful. Can you please add a reviewboard as well?

        Show
        Avery Ching added a comment - This is useful. Can you please add a reviewboard as well?
        Hide
        Maja Kabiljo added a comment -
        Show
        Maja Kabiljo added a comment - Sure, here it is: https://reviews.apache.org/r/8286/
        Hide
        Eli Reisman added a comment -

        You know the problem you're monitoring here, the lockup on awaitUniteruptably() for NettyServer, might be caused by that call itself. This is how the NettyClient shuts down, perhaps this is the pattern we need to add to the NettyServer ChannelGroup shutdown? There is more info about this pattern and why to use it on the netty.io site. The snippet is part of the netty.io way to avoid using awaitUninterruptably as in NettyServer. Here's the NettyClient#stop() method for reference:

        public void stop() {
            // Close connections asynchronously, in a Netty-approved
            // way, without cleaning up thread pools until all channels
            // in addressChannelMap are closed (success or failure)
            int channelCount = 0;
            for (ChannelRotater channelRotater : addressChannelMap.values()) {
              channelCount += channelRotater.size();
            }
            final int done = channelCount;
            final AtomicInteger count = new AtomicInteger(0);
            for (ChannelRotater channelRotater : addressChannelMap.values()) {
              channelRotater.closeChannels(new ChannelFutureListener() {
                @Override
                public void operationComplete(ChannelFuture cf) {
                  context.progress();
                  if (count.incrementAndGet() == done) {
                    if (LOG.isInfoEnabled()) {
                      LOG.info("stop: reached wait threshold, " +
                          done + " connections closed, releasing " +
                          "NettyClient.bootstrap resources now.");
                    }
                    bossExecutorService.shutdownNow();
                    workerExecutorService.shutdownNow();
                    bootstrap.releaseExternalResources();
                  }
                }
              });
            }
          }
        

        I might be way off here, but when I implemented the original version of this pattern I only did it in one of the two files (Client not Server) so this could maybe be the reason for the hangup in your logs? ...Or not! Anyway, just a thought.

        Show
        Eli Reisman added a comment - You know the problem you're monitoring here, the lockup on awaitUniteruptably() for NettyServer, might be caused by that call itself. This is how the NettyClient shuts down, perhaps this is the pattern we need to add to the NettyServer ChannelGroup shutdown? There is more info about this pattern and why to use it on the netty.io site. The snippet is part of the netty.io way to avoid using awaitUninterruptably as in NettyServer. Here's the NettyClient#stop() method for reference: public void stop() { // Close connections asynchronously, in a Netty-approved // way, without cleaning up thread pools until all channels // in addressChannelMap are closed (success or failure) int channelCount = 0; for (ChannelRotater channelRotater : addressChannelMap.values()) { channelCount += channelRotater.size(); } final int done = channelCount; final AtomicInteger count = new AtomicInteger(0); for (ChannelRotater channelRotater : addressChannelMap.values()) { channelRotater.closeChannels( new ChannelFutureListener() { @Override public void operationComplete(ChannelFuture cf) { context.progress(); if (count.incrementAndGet() == done) { if (LOG.isInfoEnabled()) { LOG.info( "stop: reached wait threshold, " + done + " connections closed, releasing " + "NettyClient.bootstrap resources now." ); } bossExecutorService.shutdownNow(); workerExecutorService.shutdownNow(); bootstrap.releaseExternalResources(); } } }); } } I might be way off here, but when I implemented the original version of this pattern I only did it in one of the two files (Client not Server) so this could maybe be the reason for the hangup in your logs? ...Or not! Anyway, just a thought.
        Hide
        Hudson added a comment -

        Integrated in Giraph-trunk-Commit #300 (See https://builds.apache.org/job/Giraph-trunk-Commit/300/)
        GIRAPH-437: Missing progress calls when stopping Netty server (Revision 1415806)

        Result = SUCCESS
        maja : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1415806
        Files :

        • /giraph/trunk/CHANGELOG
        • /giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
        • /giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
        • /giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
        • /giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BspServiceMaster.java
        • /giraph/trunk/giraph/src/main/java/org/apache/giraph/utils/ProgressableUtils.java
        • /giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/ConnectionTest.java
        • /giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/RequestFailureTest.java
        • /giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/RequestTest.java
        • /giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
        Show
        Hudson added a comment - Integrated in Giraph-trunk-Commit #300 (See https://builds.apache.org/job/Giraph-trunk-Commit/300/ ) GIRAPH-437 : Missing progress calls when stopping Netty server (Revision 1415806) Result = SUCCESS maja : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1415806 Files : /giraph/trunk/CHANGELOG /giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java /giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/netty/NettyServer.java /giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java /giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BspServiceMaster.java /giraph/trunk/giraph/src/main/java/org/apache/giraph/utils/ProgressableUtils.java /giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/ConnectionTest.java /giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/RequestFailureTest.java /giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/RequestTest.java /giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
        Hide
        Maja Kabiljo added a comment -

        Eli, thank you for your comment. We did get the timeout here again.

        I've been looking through netty.io site, but it states that we can use ChannelGroup.close()
        http://static.netty.io/3.5/api/org/jboss/netty/channel/socket/nio/NioServerSocketChannelFactory.html
        The problem seems to be that we are not keeping track of all connected channels, so we don't try to close all of them.

        Please take a look at GIRAPH-441.

        One thing about NettyClient.stop() - we don't wait for all the connections to close and resources to be released before returning from the call. Is that intentional?

        Show
        Maja Kabiljo added a comment - Eli, thank you for your comment. We did get the timeout here again. I've been looking through netty.io site, but it states that we can use ChannelGroup.close() http://static.netty.io/3.5/api/org/jboss/netty/channel/socket/nio/NioServerSocketChannelFactory.html The problem seems to be that we are not keeping track of all connected channels, so we don't try to close all of them. Please take a look at GIRAPH-441 . One thing about NettyClient.stop() - we don't wait for all the connections to close and resources to be released before returning from the call. Is that intentional?
        Maja Kabiljo made changes -
        Status Patch Available [ 10002 ] Resolved [ 5 ]
        Resolution Fixed [ 1 ]

          People

          • Assignee:
            Maja Kabiljo
            Reporter:
            Maja Kabiljo
          • Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development