Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix missing exception handling as part of `io.druid.java.util.http.client.netty.HttpClientPipelineFactory` #6090

Merged
merged 9 commits into from Aug 11, 2018

Conversation

Projects
None yet
2 participants
@asdf2014
Copy link
Member

asdf2014 commented Aug 1, 2018

Try to fix issues #6024

@jihoonson jihoonson added this to the 0.13.0 milestone Aug 1, 2018

@jihoonson
Copy link
Contributor

jihoonson left a comment

@asdf2014 thank you for the PR. I left some comments.

ctx.getChannel().close();
}
finally {
ctx.sendUpstream(e);

This comment has been minimized.

@jihoonson

jihoonson Aug 1, 2018

Contributor

What is this for?

This comment has been minimized.

@asdf2014

asdf2014 Aug 2, 2018

Author Member

Removed.

/**
*/
@ChannelHandler.Sharable
public class HttpClientHandler extends HttpContentDecompressor

This comment has been minimized.

@jihoonson

jihoonson Aug 1, 2018

Contributor

Why does this class extend HttpContentDecompressor? Probably you just want to extend SimpleChannelUpstreamHandler.

This comment has been minimized.

@asdf2014

asdf2014 Aug 2, 2018

Author Member

Done.

@@ -317,8 +317,8 @@ public void testHttpEchoServer() throws Throwable
e = e1.getCause();
}

Assert.assertTrue("IllegalArgumentException thrown by 'get'", e instanceof IllegalArgumentException);
Assert.assertTrue("Expected error message", e.getMessage().matches(".*invalid version format:.*"));
Assert.assertTrue("ChannelException thrown by 'get'", e instanceof ChannelException);

This comment has been minimized.

@jihoonson

jihoonson Aug 1, 2018

Contributor

Please use ExpectedException instead.

This comment has been minimized.

@asdf2014

asdf2014 Aug 2, 2018

Author Member

Done.

asdf2014 added some commits Aug 2, 2018

@jihoonson jihoonson added the HTTP label Aug 2, 2018

@jihoonson

This comment has been minimized.

Copy link
Contributor

jihoonson commented Aug 2, 2018

Hi @asdf2014, I've looked into this more and found something.

We already have a handler to handle the exception. The reason why the log in #6024 is printed is this handler is removed before the channel is closed. So, these changes would be enough.

@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 3, 2018

Hi, @jihoonson . Yep, that's right. 👍 Perhaps something in this PR is still useful, for example, use channel.isOpen() to check that the channel is open before call channel.close() method and use ExpectedException instead of Assert.assertTrue. Do you want me to close this, or change the title and keep those useful changes, or patch these changes to this PR? 😅

@jihoonson

This comment has been minimized.

Copy link
Contributor

jihoonson commented Aug 3, 2018

@asdf2014 yes, they are definitely useful. If you want, I think it's worth to keep those useful changes and change the title. Also, if you want, you can merge these changes to this PR as well.

@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 3, 2018

@jihoonson Okay, I keep those changes, and patch them to get the job done. Thanks for your comments.

@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 3, 2018

Hi, @jihoonson . After patching these changes, I found the same problem still exist in ChannelResourceFactory, you can run the JankyServersTest#testHttpsEchoServer test case to reproduce it. So, i added another anonymous class of SimpleChannelUpstreamHandler to fix it. PTAL.

@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 3, 2018

BTW, when i tried to use ExpectedException instead of Assert.assertTrue in JankyServersTest for other test cases, i realized the JankyServersTest#isChannelClosedException situation was hard to convert to ExpectedException way, so i created a sub-calss of TypeSafeMatcher to resolve the problem.

private static class CauseMatcher extends TypeSafeMatcher<Throwable>
{
  private final Class<? extends Throwable> expectedType;
  private final String expectedMessage;
  private final boolean isRegex;

  public CauseMatcher(Class<? extends Throwable> expectedType, String expectedMessage)
  {
    this.expectedType = expectedType;
    this.expectedMessage = expectedMessage;
    this.isRegex = false;
  }

  public CauseMatcher(Class<? extends Throwable> expectedType, String expectedMessage, boolean isRegex)
  {
    this.expectedType = expectedType;
    this.expectedMessage = expectedMessage;
    this.isRegex = isRegex;
  }

  @Override
  protected boolean matchesSafely(Throwable item)
  {
    if (item == null || item.getClass() == null || item.getMessage() == null) {
      return false;
    }
    if (!item.getClass().isAssignableFrom(expectedType)) {
      return false;
    }
    if (isRegex) {
      return Pattern.compile(expectedMessage).matcher(item.getMessage()).find();
    } else {
      return item.getMessage().contains(expectedMessage);
    }
  }

  @Override
  public void describeTo(Description description)
  {
    description.appendText("expects type is ")
        .appendValue(expectedType)
        .appendText(" and message is ")
        .appendValue(expectedMessage);
  }
}

Then, use the following code to express the JankyServersTest#isChannelClosedException logic.

expectedException.expectCause(
    anyOf(
        new CauseMatcher(ChannelException.class, "Faulty channel in resource pool"),
        new CauseMatcher(IOException.class, ".*Connection reset by peer.*", true)
    )
);

However, this change will add too many new lines of code. If just to solve this single situation would be a huge cost, but it can be used as a common util class for other test cases, then it might be worth adding. What do you think? @jihoonson

@jihoonson
Copy link
Contributor

jihoonson left a comment

@asdf2014 thanks for the quick update. I left some comments. Also, please move this and this to HttpClientPipelineFactory, so that we can put all channel pipeline initialization codes together.

Regarding CauseMatcher, it sounds good to me. Please raise a new PR if you want to apply it to all possible use cases.

}
finally {
removeHandlers();

This comment has been minimized.

@jihoonson

jihoonson Aug 3, 2018

Contributor

Hmm, this causes another wacky error like below:

2018-08-03T21:17:17,413 WARN [HttpClient-Netty-Worker-0] org.jboss.netty.channel.DefaultChannelPipeline - An exception was thrown by a user handler while handling an exception event ([id: 0x2a95abde, /127.0.0.1:58801 :> localhost/127.0.0.1:58798] EXCEPTION: java.lang.IllegalArgumentException: invalid version format: GET)
java.util.NoSuchElementException: last-handler

This is because the channel is already closed when we try to remove handlers. Let's just remove this line. It should be fine because the channel is already closed.

This comment has been minimized.

@asdf2014

asdf2014 Aug 5, 2018

Author Member

Removed.

@@ -109,6 +112,28 @@ public ChannelFuture generate(final String hostname)

final ChannelPipeline pipeline = connectFuture.getChannel().getPipeline();
pipeline.addFirst("ssl", sslHandler);
pipeline.addFirst("handler", new SimpleChannelUpstreamHandler()

This comment has been minimized.

@jihoonson

jihoonson Aug 3, 2018

Contributor

Thank you for catching this!

This comment has been minimized.

@jihoonson

jihoonson Aug 3, 2018

Contributor

I think this handler should be responsible for only connection errors. Every other errors should be handled by the handler in NettyHttpClient to properly notify callers on failures.

So, please rename the handler to be more intuitive like connectionErrorHandler. Also, this handler should be after sslHandler to catch all exceptions properly.

This comment has been minimized.

@asdf2014

asdf2014 Aug 5, 2018

Author Member

Done.

asdf2014 added some commits Aug 5, 2018

Rename the anonymous class of `SimpleChannelUpstreamHandler` as conne…
…ctionErrorHandler & use `addLast` instead of `addFirst`
@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 5, 2018

Hi, @jihoonson . Thanks for your comments.

Also, please move this and this to HttpClientPipelineFactory, so that we can put all channel pipeline initialization codes together.

Yes, this change will init those channel pipelines more efficiently. However, this SSL handler may be designed for HTTPs requests, so it might not be reasonable if we move it to HttpClientPipelineFactory.. What do you think?

Regarding CauseMatcher, it sounds good to me. Please raise a new PR if you want to apply it to all possible use cases.

Sure, I will create a new PR to improve them.

@jihoonson

This comment has been minimized.

Copy link
Contributor

jihoonson commented Aug 7, 2018

However, this SSL handler may be designed for HTTPs requests, so it might not be reasonable if we move it to HttpClientPipelineFactory.

Not sure what you mean. Would you tell me what's your concern? If you're concerned with its name indicates only Http, I think it's fine because it's easy to suppose it.

@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 7, 2018

Hi, @jihoonson . Yes, this is one of the things I concerned about. If I didn't think wrong, there are two more points.

  1. On the one hand, before adding SslHandler to pipeine, we should check if the URL protocol is HTTPs;

  2. On the other hand, the HttpClientPipelineFactory implements the getPipeline() method of ChannelPipelineFactory, which has a zero-lenght method argument, which means that those variables used to build SslHandler cannot be passed in by the getPipeline method. We can't even overload the getPipeline() method, only pass them through the constructor of ChannelPipelineFactory. Then, i sorted out all of variables used to build SslHandler, including String host, int port, ChannelFuture connectFuture, SSLContext sslContext, Timer timer, long sslHandshakeTimeout. But, in the place where HttpClientPipelineFactory is initialized, they cannot be in io.druid.java.util.http.client.HttpClientInit#createBootstrap method is obtained. In addition, the SslHandler instance itself still needs to be used by org.jboss.netty.channel.ChannelFutureListener#operationComplete, which means getPipeLine need to be able to return an instance of SslHandler, but it cannot be done, because its return value has been fixed to ChannelPipeline.

@jihoonson
Copy link
Contributor

jihoonson left a comment

In addition, the SslHandler instance itself still needs to be used by org.jboss.netty.channel.ChannelFutureListener#operationComplete, which means getPipeLine need to be able to return an instance of SslHandler, but it cannot be done, because its return value has been fixed to ChannelPipeline.

@asdf2014 good point! All other things needed to make SslHandler can be injected to HttpClientPipelineFactory, but this isn't feasible to move to it. Let's keep SslHandler as it is.

@@ -109,6 +112,28 @@ public ChannelFuture generate(final String hostname)

final ChannelPipeline pipeline = connectFuture.getChannel().getPipeline();
pipeline.addFirst("ssl", sslHandler);
pipeline.addLast("connectionErrorHandler", new SimpleChannelUpstreamHandler()

This comment has been minimized.

@jihoonson

jihoonson Aug 8, 2018

Contributor

addLast adds this handler at the last of the pipeline, but it should be right after sslHandler which is added at the first of the pipeline.

This comment has been minimized.

@asdf2014

asdf2014 Aug 8, 2018

Author Member

Hi, @jihoonson . What you mean is that we should use the following code to ensure that the order of those handlers is SSL handler + Error handler + Others handlers?

pipeline.addFirst("connectionErrorHandler", new SimpleChannelUpstreamHandler(){...})
pipeline.addFirst("ssl", sslHandler);

IMO, maybe the current order is fine, the Error handler should be placed at the end of the pipeline, so that the order of the handlers will be SSL handler + Others handlers + Error handler. What do you think?

This comment has been minimized.

@jihoonson

jihoonson Aug 8, 2018

Contributor

Sounds good. Thanks.

@@ -109,6 +112,28 @@ public ChannelFuture generate(final String hostname)

final ChannelPipeline pipeline = connectFuture.getChannel().getPipeline();
pipeline.addFirst("ssl", sslHandler);
pipeline.addLast("connectionErrorHandler", new SimpleChannelUpstreamHandler()
{
private final Logger LOGGER = new Logger(SimpleChannelUpstreamHandler.class);

This comment has been minimized.

@jihoonson

jihoonson Aug 8, 2018

Contributor

It looks that we can set exceptions for handshakeFuture rather than logging errors and this would be better because exceptions can be propagated to the callers properly.

This comment has been minimized.

@asdf2014

asdf2014 Aug 8, 2018

Author Member

Done.

This comment has been minimized.

@jihoonson

jihoonson Aug 8, 2018

Contributor

I'm seeing this code.

      pipeline.addLast("connectionErrorHandler", new SimpleChannelUpstreamHandler()
      {
        @Override
        public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
        {
          final Channel channel = ctx.getChannel();
          if (channel == null) {
            // For the case where this pipeline is not attached yet.
            return;
          }
          if (channel.isOpen()) {
            channel.close();
          }
        }
      });

Looks like the exception is not properly set for handshakeFuture. What I meant is like below:

      final ChannelFuture handshakeFuture = Channels.future(connectFuture.getChannel());

      pipeline.addLast("connectionErrorHandler", new SimpleChannelUpstreamHandler()
      {
        @Override
        public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
        {
          final Channel channel = ctx.getChannel();
          if (channel == null) {
            // For the case where this pipeline is not attached yet.
            handshakeFuture. setFailure(new ChannelException("Channel is null. The context name is ..."));
          }
          else {
            handshakeFuture.setFailure(e.getCause());
            if (channel.isOpen()) {
              channel.close();
            }
          }
        }
      });
catch (ExecutionException e1) {
e = e1.getCause();
}
expectedException.expect(ExecutionException.class);

This comment has been minimized.

@jihoonson

jihoonson Aug 8, 2018

Contributor

Would you please improve testHttpsEchoServer as well using expectedException?

This comment has been minimized.

@asdf2014

asdf2014 Aug 8, 2018

Author Member

Done.

@jihoonson

This comment has been minimized.

Copy link
Contributor

jihoonson commented Aug 9, 2018

@asdf2014 would you please check this comment: #6090 (comment)?

@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 10, 2018

Thanks. I will fix it. @jihoonson

@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 10, 2018

Hi, @jihoonson . It seems that the failure of travis build job is not related to me. Could you please help me to rebuild it? BTW, the No output has been received in the last 10m0s situation could be avoided by adding travis_wait 30 sleep 1800 option to the travis config file.

@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 10, 2018

@jihoonson Thanks a lot, I have seen that travis job has been restarted. However, it still fails, and maybe only rebuilding each failed jobs (job#414378582 / job#414378584 / job#414378587) may succesed. 😄

@jihoonson

This comment has been minimized.

Copy link
Contributor

jihoonson commented Aug 10, 2018

@asdf2014 thanks for the patience. I've just restarted them.

@jihoonson

This comment has been minimized.

Copy link
Contributor

jihoonson commented Aug 10, 2018

BTW, the No output has been received in the last 10m0s situation could be avoided by adding travis_wait 30 sleep 1800 option to the travis config file.

travis_wait is to avoid for the jobs to be killed when our tests are expected to produce no outputs for a while. The problem we're facing is that the Travis jobs were really stuck while preparing tests, so I think they should be killed so that we can figure out what happened.

@jihoonson
Copy link
Contributor

jihoonson left a comment

Thanks @asdf2014. LGTM.

@jihoonson jihoonson merged commit bd95b42 into apache:master Aug 11, 2018

1 check passed

continuous-integration/travis-ci/pr The Travis CI build passed
Details
@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 11, 2018

You are welcome.

@asdf2014

This comment has been minimized.

Copy link
Member Author

asdf2014 commented Aug 11, 2018

they should be killed so that we can figure out what happened

Yep, I agree with you.

@asdf2014 asdf2014 deleted the asdf2014:channel_handler branch Aug 11, 2018

jon-wei added a commit to implydata/druid that referenced this pull request Sep 20, 2018

Fix missing exception handling as part of `io.druid.java.util.http.cl…
…ient.netty.HttpClientPipelineFactory` (apache#6090)

* Fix missing exception handling as part of `io.druid.java.util.http.client.netty.HttpClientPipelineFactory`

* 1. Extends SimpleChannelUpstreamHandler; 2. Remove sendUpstream; 3. Using ExpectedException.

* Add more checks for channel

* Fix missing exception handler in NettyHttpClient and ChannelResourceFactory

* Rename the anonymous class of `SimpleChannelUpstreamHandler` as connectionErrorHandler & use `addLast` instead of `addFirst`

* Remove `removeHandlers()`

* Using expectedException.expect instead of Assert.assertNotNull in testHttpsEchoServer

* Using handshakeFuture.setFailure instead of logger

* Using handshakeFuture.setFailure instead of logger

jon-wei added a commit to implydata/druid that referenced this pull request Sep 20, 2018

Fix missing exception handling as part of `io.druid.java.util.http.cl…
…ient.netty.HttpClientPipelineFactory` (apache#6090)

* Fix missing exception handling as part of `io.druid.java.util.http.client.netty.HttpClientPipelineFactory`

* 1. Extends SimpleChannelUpstreamHandler; 2. Remove sendUpstream; 3. Using ExpectedException.

* Add more checks for channel

* Fix missing exception handler in NettyHttpClient and ChannelResourceFactory

* Rename the anonymous class of `SimpleChannelUpstreamHandler` as connectionErrorHandler & use `addLast` instead of `addFirst`

* Remove `removeHandlers()`

* Using expectedException.expect instead of Assert.assertNotNull in testHttpsEchoServer

* Using handshakeFuture.setFailure instead of logger

* Using handshakeFuture.setFailure instead of logger
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
You can’t perform that action at this time.