You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/12/07 06:47:51 UTC

[GitHub] [druid] xvrl opened a new pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

xvrl opened a new pull request #12032:
URL: https://github.com/apache/druid/pull/12032


   - HttpResponse no longer includes request content, this changes some
     HttpReponseHandler implementations to no longer assume content is
     generated as part of the handleResponse call, and instead expect all
     request content as part of the handleChunk
   
   - HttpResponseHandler changes uncovered a long-standing bug in
     AppendableByteArrayInputStream where calling exceptionCaught before
     appending any bytes would not throw the exception to the reader
   
   - Netty HTTP decoder now stores decoder errors in the message instead of
     triggering exceptionCaught, so we need to check the statuf of the
     decoder.
   
   - Netty Messages and underlying buffers are reference counted, so they
     need to be explicitly released after handling. As part of this
     ChannelBuffer became ByteBuf and are reference counted instead of
     garbage collected.
   
   - To support back-pressure, Channels need to disable AUTO_READ and call
     read() explicitly when needed instead of using setReadable(true/false)
   
   - ClientBootstrap no longer uses separate bosspool and workerpool sizes,
     instead relying on the workerpool.
   
   - Netty HTTP codec now supports resuming handling of HTTP messages after
     handling CONNECT, removing the need to manually add the codec back to
     the pipeline.
   
   - Remove deprecated Netty method calls and rename classes to align with
     Netty class name changes
   
   - Remove deprecated use of ExpectedException in favor of
     Assert.assertThrows for all tests that had to be touched.
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-988201547


   This pull request **introduces 3 alerts** when merging f2794fdeac57763a8967109ed731a34d1f0aa3f2 into 0b3f0bbbd81cc3ec33e1a4716ee33b149d1a2940 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-b1057b4fd74eddb2f1fb4d31b984351e678f4957)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-999002524


   This pull request **introduces 3 alerts** when merging be46d9f0aa08165a53992c491e76e3c3f60abc74 into f34575936091f7e6944e4458b91396e904620727 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-50b5ce67d9443bd00202943c65ab993b2cae71bd)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-987685656


   This pull request **introduces 1 alert** when merging 5c704e6e962e72691a3a7791439591fcff33e10d into 65cadbe42a23fc4dca0d1f703b755293f54f15c4 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-d2a81bab8aad7bf224e3ab8ebcc8138cd05bc48e)
   
   **new alerts:**
   
   * 1 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-1050253923


   This pull request **introduces 3 alerts** when merging 80feb252e0a5d2e169bcecf6fdeedf331548d588 into b86f2d4c2e935346d600e51b22403150ebd1501d - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-3e64cd8a90025c44cd90ce32a4596fefa9da0e19)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] FrankChen021 commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
FrankChen021 commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-1005475125


   @xvrl I see the router and the coordinator are using jetty as http client. Do you know why we need two http client libraries? Is there any special consideration?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-991962191


   This pull request **introduces 3 alerts** when merging 204aecff6d98bbd4254cf8505019bdc0ac673e8d into e53c3e80ca246c0bc9efd18d23858b791eb6e1a2 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-0151d9d749a62c1801b7dd1345719e39907f0562)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] xvrl commented on a change in pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #12032:
URL: https://github.com/apache/druid/pull/12032#discussion_r764193588



##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java
##########
@@ -102,14 +100,14 @@ public InputStream nextElement()
   @Override
   public ClientResponse<InputStream> handleChunk(
       ClientResponse<InputStream> clientResponse,
-      HttpChunk chunk,
+      HttpContent chunk,
       long chunkNum
   )
   {
-    final ChannelBuffer channelBuffer = chunk.getContent();
-    final int bytes = channelBuffer.readableBytes();
+    final ByteBuf byteBuf = chunk.content();
+    final int bytes = byteBuf.readableBytes();
     if (bytes > 0) {
-      try (ChannelBufferInputStream channelStream = new ChannelBufferInputStream(channelBuffer)) {
+      try (ByteBufInputStream channelStream = new ByteBufInputStream(byteBuf)) {

Review comment:
       @gianm I think this try-with-resources is incorrect, we don't want to be closing the stream at the end of the block, but instead let the downstream SequenceInputStream close the stream when done reading 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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #12032:
URL: https://github.com/apache/druid/pull/12032#discussion_r764228382



##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java
##########
@@ -102,14 +100,14 @@ public InputStream nextElement()
   @Override
   public ClientResponse<InputStream> handleChunk(
       ClientResponse<InputStream> clientResponse,
-      HttpChunk chunk,
+      HttpContent chunk,
       long chunkNum
   )
   {
-    final ChannelBuffer channelBuffer = chunk.getContent();
-    final int bytes = channelBuffer.readableBytes();
+    final ByteBuf byteBuf = chunk.content();
+    final int bytes = byteBuf.readableBytes();
     if (bytes > 0) {
-      try (ChannelBufferInputStream channelStream = new ChannelBufferInputStream(channelBuffer)) {
+      try (ByteBufInputStream channelStream = new ByteBufInputStream(byteBuf)) {

Review comment:
       It looks like ChannelBufferInputStream.close does nothing, so it didn't used to matter. But yeah, this code doesn't look right. It hands off the input stream and then closes it, which is poor form.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] xvrl commented on a change in pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #12032:
URL: https://github.com/apache/druid/pull/12032#discussion_r779179397



##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
##########
@@ -227,30 +221,27 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
                 }
 
                 assert currentChunkNum == 0;
-                possiblySuspendReads(response);
-
-                if (!httpResponse.isChunked()) {
-                  finishRequest();
-                }
-              } else if (msg instanceof HttpChunk) {
-                HttpChunk httpChunk = (HttpChunk) msg;
+                possiblyRead(response);
+              } else if (msg instanceof HttpContent) {
+                HttpContent httpChunk = (HttpContent) msg;
                 if (log.isDebugEnabled()) {
                   log.debug(
                       "[%s] Got chunk: %sB, last=%s",
                       requestDesc,
-                      httpChunk.getContent().readableBytes(),
-                      httpChunk.isLast()
+                      httpChunk.content().readableBytes(),
+                      httpChunk instanceof LastHttpContent
                   );
                 }
 
-                if (httpChunk.isLast()) {
+                response = handler.handleChunk(response, httpChunk, ++currentChunkNum);
+                if (response.isFinished() && !retVal.isDone()) {
+                  retVal.set((Final) response.getObj());
+                }
+
+                if (httpChunk instanceof LastHttpContent) {

Review comment:
       in the past we would only call handleChunk() on chunks that were not the last chunk. Not sure if that was a mistake or if netty 3 happened to always finish with an empty chunk. With netty 4 the last chunk can contain data, so we do need to call handleChunk of all HttpContent messages, irrespective of whether they implemenent LastHttpContent or not.
   isFinished() may also return true before we handle all the chunks (e.g. in the case of a streaming response) so we wouldn't want to call finishRequest until we handled all the chunks.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
clintropolis commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-1006132338


   it looks like a handful of integration tests perhaps have legitimate failures, though they all look similar-ish: exceptions that occur from some underlying network failure, so hopefully it isn't something very involved and just missing a catch or some form of exception handling somewhere
   
   examples:
   
   ```
   [ERROR] Tests run: 4, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 897.106 s <<< FAILURE! - in TestSuite
   [ERROR] testKafkaIndexDataWithStartStopSupervisor(org.apache.druid.tests.parallelized.ITKafkaIndexingServiceNonTransactionalParallelizedTest)  Time elapsed: 176.157 s
   [ERROR] testKafkaIndexDataWithKafkaReshardSplit(org.apache.druid.tests.parallelized.ITKafkaIndexingServiceNonTransactionalParallelizedTest)  Time elapsed: 1.738 s
   [ERROR] testKafkaTerminatedSupervisorAutoCleanup(org.apache.druid.tests.parallelized.ITKafkaIndexingServiceNonTransactionalParallelizedTest)  Time elapsed: 158.769 s
   [ERROR] testKafkaIndexDataWithWithAutoscaler(org.apache.druid.tests.parallelized.ITKafkaIndexingServiceNonTransactionalParallelizedTest)  Time elapsed: 602.337 s  <<< FAILURE!
   java.lang.RuntimeException: 
   org.apache.druid.java.util.common.ISE: Error while querying[https://127.0.0.1:8282/druid/v2?pretty] status[500 Internal Server Error] content[{
     "error" : "Unknown exception",
     "errorMessage" : "Channel disconnected",
     "errorClass" : "io.netty.channel.ChannelException",
     "host" : "druid-middlemanager:8103"
   }]
   ```
   
   and 
   
   ```
   [ERROR] testIndexData(org.apache.druid.tests.indexer.ITNestedQueryPushDownTest)  Time elapsed: 30.879 s  <<< FAILURE!
   java.lang.RuntimeException: java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.io.IOException: Connection reset by peer
   	at org.apache.druid.tests.indexer.ITNestedQueryPushDownTest.testIndexData(ITNestedQueryPushDownTest.java:96)
   Caused by: java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.io.IOException: Connection reset by peer
   	at org.apache.druid.tests.indexer.ITNestedQueryPushDownTest.testIndexData(ITNestedQueryPushDownTest.java:92)
   Caused by: java.util.concurrent.ExecutionException: java.io.IOException: Connection reset by peer
   	at org.apache.druid.tests.indexer.ITNestedQueryPushDownTest.testIndexData(ITNestedQueryPushDownTest.java:92)
   Caused by: java.io.IOException: Connection reset by peer
   ```
   
   i haven't had a chance to dig in yet, so not sure the exact issue


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-1005946920


   This pull request **introduces 3 alerts** when merging ac47ccaa5d98f89bb8a05b553930b44af4701a1b into b53e7f4d12f9c4922e7ef8f540b178d0dbc21410 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-abcdb7675f3e6268dde2b10769f1054abf5b816a)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-991409659


   This pull request **introduces 3 alerts** when merging 3e96349643cd3f52d05acbed7c240d4c8729226a into 761fe9f144d56e8c7c2f0a8e4838dc3f2c2d5d31 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-5cc2964b6ebb4a2e29e45f1f47b255967a01ab8d)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] xvrl commented on a change in pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #12032:
URL: https://github.com/apache/druid/pull/12032#discussion_r768187396



##########
File path: indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolderTest.java
##########
@@ -192,4 +200,72 @@ public void testSyncListener()
 
     updates.clear();
   }
+
+  @Test
+  public void testAssign() throws Exception

Review comment:
       these tests were added to satisfy the code coverage requirements for the diff.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] xvrl commented on a change in pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #12032:
URL: https://github.com/apache/druid/pull/12032#discussion_r778504064



##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
##########
@@ -355,30 +355,31 @@ public void channelDisconnected(ChannelHandlerContext context, ChannelStateEvent
           private void removeHandlers()
           {
             if (readTimeout > 0) {
-              channel.getPipeline().remove(READ_TIMEOUT_HANDLER_NAME);
+              channel.pipeline().remove(READ_TIMEOUT_HANDLER_NAME);
             }
-            channel.getPipeline().remove(LAST_HANDLER_NAME);
+            channel.pipeline().remove(LAST_HANDLER_NAME);
           }
         }
     );
 
-    channel.write(httpRequest).addListener(
-        new ChannelFutureListener()
-        {
-          @Override
-          public void operationComplete(ChannelFuture future)
-          {
-            if (!future.isSuccess()) {
-              channel.close();
-              channelResourceContainer.returnResource();
-              if (!retVal.isDone()) {
-                retVal.setException(
-                    new ChannelException(
-                        StringUtils.format("[%s] Failed to write request to channel", requestDesc),
-                        future.getCause()
-                    )
-                );
-              }
+    channel.write(httpRequest);
+    if (request.hasContent()) {
+      channel.write(new DefaultHttpContent(request.getContent()));

Review comment:
       it looks like there is `FullHttpRequest`, which implements both HttpRequest and LastHttpContent, and should be transparent to the HTTP encoder, so we could use that instead to avoid multiple messages.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] xvrl commented on a change in pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #12032:
URL: https://github.com/apache/druid/pull/12032#discussion_r768188198



##########
File path: core/src/test/java/org/apache/druid/java/util/http/client/FriendlyServersTest.java
##########
@@ -241,9 +201,42 @@ public void run()
               StatusResponseHandler.getInstance()
           ).get();
 
-      Assert.assertEquals(200, response.getStatus().getCode());
+      Assert.assertEquals(200, response.getStatus().code());
+      Assert.assertEquals("hello!", response.getContent());
+      Assert.assertEquals("accept-encoding: identity", foundAcceptEncoding.get());
+    }
+    finally {
+      exec.shutdownNow();
+      serverSocket.close();
+      lifecycle.stop();
+    }
+  }
+
+  @Test
+  public void testCompressionCodecHeader() throws Exception

Review comment:
       test added to satisfy code coverage checks

##########
File path: core/src/test/java/org/apache/druid/java/util/http/client/FriendlyServersTest.java
##########
@@ -252,6 +245,39 @@ public void run()
     }
   }
 
+  @Nonnull
+  private AtomicReference<String> acceptEncodingServer(ExecutorService exec, ServerSocket serverSocket)

Review comment:
       test added to satisfy code coverage checks




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cheddar commented on a change in pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
cheddar commented on a change in pull request #12032:
URL: https://github.com/apache/druid/pull/12032#discussion_r773669393



##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
##########
@@ -227,30 +221,27 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
                 }
 
                 assert currentChunkNum == 0;
-                possiblySuspendReads(response);
-
-                if (!httpResponse.isChunked()) {
-                  finishRequest();
-                }
-              } else if (msg instanceof HttpChunk) {
-                HttpChunk httpChunk = (HttpChunk) msg;
+                possiblyRead(response);
+              } else if (msg instanceof HttpContent) {
+                HttpContent httpChunk = (HttpContent) msg;
                 if (log.isDebugEnabled()) {
                   log.debug(
                       "[%s] Got chunk: %sB, last=%s",
                       requestDesc,
-                      httpChunk.getContent().readableBytes(),
-                      httpChunk.isLast()
+                      httpChunk.content().readableBytes(),
+                      httpChunk instanceof LastHttpContent
                   );
                 }
 
-                if (httpChunk.isLast()) {
+                response = handler.handleChunk(response, httpChunk, ++currentChunkNum);
+                if (response.isFinished() && !retVal.isDone()) {
+                  retVal.set((Final) response.getObj());
+                }
+
+                if (httpChunk instanceof LastHttpContent) {

Review comment:
       is a reason not to merge the logic from this `if` with the `if(response.isFinished()...)` above?

##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
##########
@@ -267,17 +258,23 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
 
               throw ex;
             }
+            finally {
+              ReferenceCountUtil.release(msg);
+            }
           }
 
-          private void possiblySuspendReads(ClientResponse<?> response)
+          private void possiblyRead(ClientResponse<?> response)
           {
-            if (!response.isContinueReading()) {
+            if (response.isContinueReading()) {
+              channel.read();
+            } else {
               synchronized (watermarkLock) {
                 suspendWatermark = Math.max(suspendWatermark, currentChunkNum);
                 if (suspendWatermark > resumeWatermark) {
-                  channel.setReadable(false);
                   backPressureStartTimeNs = System.nanoTime();
                   log.debug("[%s] Suspended reads from channel (chunkNum = %,d).", requestDesc, currentChunkNum);

Review comment:
       Nit: this should really be "delaying reads from channel" now.

##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java
##########
@@ -56,18 +56,15 @@
   @Override
   public ClientResponse<InputStream> handleResponse(HttpResponse response, TrafficCop trafficCop)
   {
-    try (ChannelBufferInputStream channelStream = new ChannelBufferInputStream(response.getContent())) {
-      queue.put(channelStream);
-    }
-    catch (IOException e) {
-      throw new RuntimeException(e);
+    try {
+      // add empty initial buffer since SequenceInputStream will peek the first element right away

Review comment:
       This comment is not telling me why it's bad that it's peeking the first element right away?  Should we fix that instead of priming the queue?

##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
##########
@@ -355,30 +355,31 @@ public void channelDisconnected(ChannelHandlerContext context, ChannelStateEvent
           private void removeHandlers()
           {
             if (readTimeout > 0) {
-              channel.getPipeline().remove(READ_TIMEOUT_HANDLER_NAME);
+              channel.pipeline().remove(READ_TIMEOUT_HANDLER_NAME);
             }
-            channel.getPipeline().remove(LAST_HANDLER_NAME);
+            channel.pipeline().remove(LAST_HANDLER_NAME);
           }
         }
     );
 
-    channel.write(httpRequest).addListener(
-        new ChannelFutureListener()
-        {
-          @Override
-          public void operationComplete(ChannelFuture future)
-          {
-            if (!future.isSuccess()) {
-              channel.close();
-              channelResourceContainer.returnResource();
-              if (!retVal.isDone()) {
-                retVal.setException(
-                    new ChannelException(
-                        StringUtils.format("[%s] Failed to write request to channel", requestDesc),
-                        future.getCause()
-                    )
-                );
-              }
+    channel.write(httpRequest);
+    if (request.hasContent()) {
+      channel.write(new DefaultHttpContent(request.getContent()));

Review comment:
       Is there really no way to add this to the `httpRequest` object such that it "just works"?  I haven't looked at the interfaces on Netty 4 to verify this, so it's perhaps a naive question.

##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
##########
@@ -192,18 +179,25 @@ public void stop()
           private long resumeWatermark = -1;
 
           @Override
-          public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+          public void channelRead(ChannelHandlerContext ctx, Object msg)
           {
             if (log.isDebugEnabled()) {

Review comment:
       This is a nit, but this can probably move inside the `try`.  If, for some reason, this threw an exception, we wouldn't release the `msg` resource.

##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/Request.java
##########
@@ -90,19 +76,38 @@ public boolean hasContent()
     return content != null;
   }
 
-  public ChannelBuffer getContent()
+  public ByteBuf getContent()
   {
-    return content;
+    // return a duplicate buffer since with increased reference count
+    // this ensures Netty does not free the underlying array after it gets handled,
+    // since we sometimes read the buffer after it has been dispatched to Netty
+    // (e.g. when alling withUrl or copy, which migh happen after Netty has handled it already)

Review comment:
       nit: missing `c` on `alling`.  Also missing `t` on `migh`




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] xvrl commented on a change in pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #12032:
URL: https://github.com/apache/druid/pull/12032#discussion_r779173654



##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/response/SequenceInputStreamResponseHandler.java
##########
@@ -56,18 +56,15 @@
   @Override
   public ClientResponse<InputStream> handleResponse(HttpResponse response, TrafficCop trafficCop)
   {
-    try (ChannelBufferInputStream channelStream = new ChannelBufferInputStream(response.getContent())) {
-      queue.put(channelStream);
-    }
-    catch (IOException e) {
-      throw new RuntimeException(e);
+    try {
+      // add empty initial buffer since SequenceInputStream will peek the first element right away

Review comment:
       I can make the comment more explicit, the SequenceInputStream constructor peeking will block on the empty queue, causing the entire method to block. Previously we were guaranteed at least one chunk in this method, but that's no longer the case now.
   
   To fix this, here are some options:
   1. write our own SequenceInputStream.
   2. change `HttpResponseHandler` to have an `InitialType` returned by handleReponse(), so that we can create the SequenceInputStream in handleResponse once we get the first chunk.
   
   I would probably opt for the first one, but either one seems a bit complex and my inclinations is to defer those improvements to avoid further increasing the scope of this PR




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #12032:
URL: https://github.com/apache/druid/pull/12032#discussion_r769206105



##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/HttpClientInit.java
##########
@@ -53,39 +48,13 @@ public static HttpClient createClient(HttpClientConfig config, Lifecycle lifecyc
     try {
       // We need to use the full constructor in order to set a ThreadNameDeterminer. The other parameters are taken
       // from the defaults in HashedWheelTimer's other constructors.

Review comment:
       these comments seem no longer relevant

##########
File path: core/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
##########
@@ -355,30 +355,31 @@ public void channelDisconnected(ChannelHandlerContext context, ChannelStateEvent
           private void removeHandlers()
           {
             if (readTimeout > 0) {
-              channel.getPipeline().remove(READ_TIMEOUT_HANDLER_NAME);
+              channel.pipeline().remove(READ_TIMEOUT_HANDLER_NAME);
             }
-            channel.getPipeline().remove(LAST_HANDLER_NAME);
+            channel.pipeline().remove(LAST_HANDLER_NAME);
           }
         }
     );
 
-    channel.write(httpRequest).addListener(
-        new ChannelFutureListener()
-        {
-          @Override
-          public void operationComplete(ChannelFuture future)
-          {
-            if (!future.isSuccess()) {
-              channel.close();
-              channelResourceContainer.returnResource();
-              if (!retVal.isDone()) {
-                retVal.setException(
-                    new ChannelException(
-                        StringUtils.format("[%s] Failed to write request to channel", requestDesc),
-                        future.getCause()
-                    )
-                );
-              }
+    channel.write(httpRequest);
+    if (request.hasContent()) {
+      channel.write(new DefaultHttpContent(request.getContent()));
+    }
+    channel.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT).addListener(

Review comment:
       the 'flush' change was the one that stuck out to me in https://netty.io/wiki/new-and-noteworthy-in-4.0.html, looks like we got them all afaict :+1:




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-991298534


   This pull request **introduces 3 alerts** when merging 69cb01b71d851a930d763b224619b03e5416c633 into 761fe9f144d56e8c7c2f0a8e4838dc3f2c2d5d31 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-e02b1e4cb54454e7b26d0e408f22921829b26242)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-994222065


   This pull request **introduces 3 alerts** when merging cc334bbf6ba9a6af2209be6554254b1889a40456 into 4ede3bbff69a9eb1e7f3108f3a46d180d783aba7 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-1402b772232f22de739a859381ed4115294749d6)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-1050405675


   This pull request **introduces 3 alerts** when merging d369faa40ec147b6c641f8607a00c3cf1ceecb91 into b86f2d4c2e935346d600e51b22403150ebd1501d - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-6df11adfe0cb7702d5bba1a081191e43f937d8bb)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-989318405


   This pull request **introduces 3 alerts** when merging d4264269de51988dd8d3ed7e5b81249fcccb5b7d into 6ac4e2dbb8d2e390f0f4a9c0130ce2fb223c297b - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-66a9db59e76dcb137d3d0d9b7027ee6b7b46ad84)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-993014684


   This pull request **introduces 3 alerts** when merging 5cec2c2e0c4f099a9dbaabbe75abb2483f684bd6 into e53c3e80ca246c0bc9efd18d23858b791eb6e1a2 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-e42b01e352b50bb1aa39273b8e656045aaa02223)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-998444034


   This pull request **introduces 3 alerts** when merging a9bce507501916791b8474f76a018b85c91e43f8 into f34575936091f7e6944e4458b91396e904620727 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-e6bd4d5b9409c1c560f7644a5530b68ae5072569)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-994294121


   This pull request **introduces 3 alerts** when merging 04a526430865ffd0bca8a7f0862f65cfcf3be237 into 4ede3bbff69a9eb1e7f3108f3a46d180d783aba7 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-efd830ffdb8d19cf4e4e9eef4d8c3aa04623514e)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis edited a comment on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
clintropolis edited a comment on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-1006132338


   it looks like a handful of integration tests perhaps have legitimate failures, though they all look similar-ish: exceptions that occur from some underlying network failure, so hopefully it isn't something very involved and just missing a catch or some form of exception handling somewhere
   
   examples:
   
   ```
   [ERROR] Tests run: 4, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 897.106 s <<< FAILURE! - in TestSuite
   [ERROR] testKafkaIndexDataWithStartStopSupervisor(org.apache.druid.tests.parallelized.ITKafkaIndexingServiceNonTransactionalParallelizedTest)  Time elapsed: 176.157 s
   [ERROR] testKafkaIndexDataWithKafkaReshardSplit(org.apache.druid.tests.parallelized.ITKafkaIndexingServiceNonTransactionalParallelizedTest)  Time elapsed: 1.738 s
   [ERROR] testKafkaTerminatedSupervisorAutoCleanup(org.apache.druid.tests.parallelized.ITKafkaIndexingServiceNonTransactionalParallelizedTest)  Time elapsed: 158.769 s
   [ERROR] testKafkaIndexDataWithWithAutoscaler(org.apache.druid.tests.parallelized.ITKafkaIndexingServiceNonTransactionalParallelizedTest)  Time elapsed: 602.337 s  <<< FAILURE!
   java.lang.RuntimeException: 
   org.apache.druid.java.util.common.ISE: Error while querying[https://127.0.0.1:8282/druid/v2?pretty] status[500 Internal Server Error] content[{
     "error" : "Unknown exception",
     "errorMessage" : "Channel disconnected",
     "errorClass" : "io.netty.channel.ChannelException",
     "host" : "druid-middlemanager:8103"
   }]
   ```
   
   ```
   [ERROR] Tests run: 3, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 532.75 s <<< FAILURE! - in TestSuite
   [ERROR] testKafkaIndexDataWithLosingCoordinator(org.apache.druid.tests.indexer.ITKafkaIndexingServiceNonTransactionalSerializedTest)  Time elapsed: 123.614 s
   [ERROR] testKafkaIndexDataWithLosingHistorical(org.apache.druid.tests.indexer.ITKafkaIndexingServiceNonTransactionalSerializedTest)  Time elapsed: 153.584 s  <<< FAILURE!
   java.lang.RuntimeException: 
   org.apache.druid.java.util.common.ISE: Error while querying[http://127.0.0.1:8082/druid/v2?pretty] status[500 Internal Server Error] content[{
     "error" : "Unknown exception",
     "errorMessage" : "Channel disconnected",
     "errorClass" : "io.netty.channel.ChannelException",
     "host" : "druid-middlemanager:8103"
   }]
   ```
   
   and 
   
   ```
   [ERROR] testIndexData(org.apache.druid.tests.indexer.ITNestedQueryPushDownTest)  Time elapsed: 30.879 s  <<< FAILURE!
   java.lang.RuntimeException: java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.io.IOException: Connection reset by peer
   	at org.apache.druid.tests.indexer.ITNestedQueryPushDownTest.testIndexData(ITNestedQueryPushDownTest.java:96)
   Caused by: java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.io.IOException: Connection reset by peer
   	at org.apache.druid.tests.indexer.ITNestedQueryPushDownTest.testIndexData(ITNestedQueryPushDownTest.java:92)
   Caused by: java.util.concurrent.ExecutionException: java.io.IOException: Connection reset by peer
   	at org.apache.druid.tests.indexer.ITNestedQueryPushDownTest.testIndexData(ITNestedQueryPushDownTest.java:92)
   Caused by: java.io.IOException: Connection reset by peer
   ```
   
   i haven't had a chance to dig in yet, so not sure the exact issue


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-991298534






-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on pull request #12032: Migrade Druid HttpClient from Netty 3 to 4

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #12032:
URL: https://github.com/apache/druid/pull/12032#issuecomment-988295089


   This pull request **introduces 3 alerts** when merging 1d38c3a0b06750b0400d74be55f797fe41a9c84a into 1d3c8c187b9c07ee1adc6a6669f756421ec9f324 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-f16634b69b41c76b7c91ac3c0f5a8331f83384c1)
   
   **new alerts:**
   
   * 3 for Potential input resource leak


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org