You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2021/03/03 16:10:32 UTC

[GitHub] [drill] rymarm opened a new pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

rymarm opened a new pull request #2185:
URL: https://github.com/apache/drill/pull/2185


   # [DRILL-7790](https://issues.apache.org/jira/browse/DRILL-7790): Build Drill with Netty version 4.1.50.Final
   
   `Netty` of version `4.0.48.Final` has vulnerabilities as CVE-2019-16869, CVE-2014-3488 and other. I want to update to the last available, stable version of `Netty` `4.1.59.Final`.
   
   `ChannelPromise` and `ChannelFuture` were replaced with `DefaultPromise` and `Future` according. It was done in response to changes in https://github.com/netty/netty/commit/1740f366eb728ea5a0a63d18e9042161673414cd . `ChannelPromise` and `ChannelFuture` are wrong used and netty's changes are predict it.
   
   Other one breaking `Netty` change is https://github.com/netty/netty/commit/39cc7a673939dec96258ff27f5b1874671838af0 . In Drill we have `ByteBuffAlocater` which doesn't support heap buffers. But in the netty's commit was changed internal behavior in `SslHandler`. Previously, regardless to chosen ssl engine, were using only `directBuffer()` or `buffer()`, which in our case both lid to the same - `directBuffer`. But now, behavior was changed and for JDK ssl engine is always used `heapBuffer()` which is not supported in Drill. So, I'm not sure, how to resolve this issue. In this PR I propose to use `directBuffer()` under `heapBuffer()`, but it is not the best solution. Maybe, someone from Drill community know a better solution? 
   
   ## Documentation
   No user visible changes
   
   ## Testing
   Unit tests
   


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



[GitHub] [drill] rymarm commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r591255395



##########
File path: exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/WebSessionResourcesTest.java
##########
@@ -65,15 +64,13 @@ public void operationComplete(Future<Void> future) throws Exception {
   @Test
   public void testChannelPromiseWithNullExecutor() throws Exception {
     try {
-      ChannelPromise closeFuture = new DefaultChannelPromise(null);
+      Promise closeFuture = new DefaultPromise(null);

Review comment:
       Fixed.




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



[GitHub] [drill] Agirish commented on pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
Agirish commented on pull request #2185:
URL: https://github.com/apache/drill/pull/2185#issuecomment-795969815


   @vvysotskyi thanks so much 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



[GitHub] [drill] rymarm commented on pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on pull request #2185:
URL: https://github.com/apache/drill/pull/2185#issuecomment-797355523


   @luocooong Hi. It wasn't tested in a real world in distributed mode. Only with a single drillbit.


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



[GitHub] [drill] vvysotskyi commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r591777310



##########
File path: exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/WebSessionResourcesTest.java
##########
@@ -85,14 +82,12 @@ public void testChannelPromiseWithNullExecutor() throws Exception {
   public void testChannelPromiseWithValidExecutor() throws Exception {
     try {
       EventExecutor mockExecutor = mock(EventExecutor.class);
-      ChannelPromise closeFuture = new DefaultChannelPromise(null, mockExecutor);
+      Promise closeFuture = new DefaultPromise(mockExecutor);

Review comment:
       Please add a generic type here and in other places.




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



[GitHub] [drill] rymarm commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r591253842



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebSessionResources.java
##########
@@ -40,10 +40,10 @@
 
   private final UserSession webUserSession;
 
-  private ChannelPromise closeFuture;
+  private Promise closeFuture;
 
   WebSessionResources(BufferAllocator allocator, SocketAddress remoteAddress,
-                      UserSession userSession, ChannelPromise closeFuture) {
+                      UserSession userSession, Promise closeFuture) {

Review comment:
       Fixed.

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebSessionResources.java
##########
@@ -58,7 +58,7 @@ public BufferAllocator getAllocator() {
     return allocator;
   }
 
-  public ChannelPromise getCloseFuture() {
+  public Promise getCloseFuture() {

Review comment:
       Fixed.




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



[GitHub] [drill] rymarm commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r591259363



##########
File path: exec/memory/base/src/main/java/org/apache/drill/exec/memory/DrillByteBufAllocator.java
##########
@@ -111,31 +118,68 @@ public boolean isDirectBufferPooled() {
 
   @Override
   public ByteBuf heapBuffer() {
-    throw fail();
+    return HEAP_ALLOCATOR.allocateHeap();
   }
 
   @Override
   public ByteBuf heapBuffer(int initialCapacity) {
-    throw fail();
+    return HEAP_ALLOCATOR.allocateHeap(initialCapacity);
   }
 
   @Override
   public ByteBuf heapBuffer(int initialCapacity, int maxCapacity) {
-    throw fail();
+    return HEAP_ALLOCATOR.allocateHeap(initialCapacity, maxCapacity);
   }
 
   @Override
   public CompositeByteBuf compositeHeapBuffer() {
-    throw fail();
+    return compositeHeapBuffer(DEFAULT_MAX_COMPOSITE_COMPONENTS);
   }
 
   @Override
   public CompositeByteBuf compositeHeapBuffer(int maxNumComponents) {
-    throw fail();
-  }
-
-  private RuntimeException fail() {
-    throw new UnsupportedOperationException("Allocator doesn't support heap-based memory.");
+    return new CompositeByteBuf(this, false, maxNumComponents);
+  }
+
+  /**
+   * This method was copied from AbstractByteBufAllocator. Netty 4.1.x moved this method from
+   * AbstractByteBuf to AbstractByteBufAllocator. However, as DrillByteBufAllocator doesn't extend
+   * AbstractByteBufAllocator, it doesn't get the implementation automatically and we have to copy
+   * the codes.
+   */
+  @Override
+  public int calculateNewCapacity(int minNewCapacity, int maxCapacity) {

Review comment:
       Yea, you are right. I were thinking about it, but for some reason decide to not do like that. But yes, it's a better solution that also let remove redundant code in `DrillByteBufAllocator` and in `PooledByteBufAllocatorL`.
   
   Done.

##########
File path: exec/memory/base/src/main/java/org/apache/drill/exec/memory/DrillByteBufAllocator.java
##########
@@ -111,31 +118,68 @@ public boolean isDirectBufferPooled() {
 
   @Override
   public ByteBuf heapBuffer() {
-    throw fail();
+    return HEAP_ALLOCATOR.allocateHeap();
   }
 
   @Override
   public ByteBuf heapBuffer(int initialCapacity) {
-    throw fail();
+    return HEAP_ALLOCATOR.allocateHeap(initialCapacity);
   }
 
   @Override
   public ByteBuf heapBuffer(int initialCapacity, int maxCapacity) {
-    throw fail();
+    return HEAP_ALLOCATOR.allocateHeap(initialCapacity, maxCapacity);
   }
 
   @Override
   public CompositeByteBuf compositeHeapBuffer() {
-    throw fail();
+    return compositeHeapBuffer(DEFAULT_MAX_COMPOSITE_COMPONENTS);
   }
 
   @Override
   public CompositeByteBuf compositeHeapBuffer(int maxNumComponents) {
-    throw fail();
-  }
-
-  private RuntimeException fail() {
-    throw new UnsupportedOperationException("Allocator doesn't support heap-based memory.");
+    return new CompositeByteBuf(this, false, maxNumComponents);
+  }
+
+  /**
+   * This method was copied from AbstractByteBufAllocator. Netty 4.1.x moved this method from
+   * AbstractByteBuf to AbstractByteBufAllocator. However, as DrillByteBufAllocator doesn't extend
+   * AbstractByteBufAllocator, it doesn't get the implementation automatically and we have to copy
+   * the codes.
+   */
+  @Override
+  public int calculateNewCapacity(int minNewCapacity, int maxCapacity) {

Review comment:
       Yeah, you are right. I were thinking about it, but for some reason decide to not do like that. But yes, it's a better solution that also let remove redundant code in `DrillByteBufAllocator` and in `PooledByteBufAllocatorL`.
   
   Done.




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



[GitHub] [drill] rymarm commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r592127374



##########
File path: exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/WebSessionResourcesTest.java
##########
@@ -85,14 +82,12 @@ public void testChannelPromiseWithNullExecutor() throws Exception {
   public void testChannelPromiseWithValidExecutor() throws Exception {
     try {
       EventExecutor mockExecutor = mock(EventExecutor.class);
-      ChannelPromise closeFuture = new DefaultChannelPromise(null, mockExecutor);
+      Promise closeFuture = new DefaultPromise(mockExecutor);

Review comment:
       Sorry, I've missed this admonition. Сorrected.




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



[GitHub] [drill] vdiravka commented on pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
vdiravka commented on pull request #2185:
URL: https://github.com/apache/drill/pull/2185#issuecomment-812444074


   > @luocooong Hi. It wasn't tested in a real world in distributed mode. Only with a single drillbit.
   @rymarm Possibly we may want regression test cluster to test all PR on the distributed Drill cluster
   


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



[GitHub] [drill] rymarm commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r591255051



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/work/prepare/PreparedStatementProvider.java
##########
@@ -244,8 +244,8 @@ public UserSession getSession() {
     }
 
     @Override
-    public ChannelFuture getChannelClosureFuture() {
-      return inner.getChannelClosureFuture();
+    public Future getClosureFuture() {

Review comment:
       Fixed.




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



[GitHub] [drill] rymarm commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r591253702



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebSessionResources.java
##########
@@ -40,10 +40,10 @@
 
   private final UserSession webUserSession;
 
-  private ChannelPromise closeFuture;
+  private Promise closeFuture;

Review comment:
       Fixed.




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



[GitHub] [drill] rymarm commented on pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on pull request #2185:
URL: https://github.com/apache/drill/pull/2185#issuecomment-796535919


   @vvysotskyi many-many thank you for review! I've made all changes, that you mentioned and squashed commits.


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



[GitHub] [drill] rymarm commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r591254131



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
##########
@@ -116,7 +115,7 @@
 
   private final ResponseSendListener responseListener = new ResponseSendListener();
   private final GenericFutureListener<Future<Void>> closeListener = future -> cancel();
-  private final ChannelFuture closeFuture;
+  private final Future closeFuture;

Review comment:
       Fixed.




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



[GitHub] [drill] rymarm commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r591252908



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/UserClientConnection.java
##########
@@ -60,10 +60,10 @@
   void sendData(RpcOutcomeListener<Ack> listener, QueryDataPackage data);
 
   /**
-   * Returns the {@link ChannelFuture} which will be notified when this
+   * Returns the {@link Future} which will be notified when this
    * channel is closed.  This method always returns the same future instance.
    */
-  ChannelFuture getChannelClosureFuture();
+  Future getClosureFuture();

Review comment:
       Thanks for admonition! Fixed.




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



[GitHub] [drill] vvysotskyi commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r590707689



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/UserClientConnection.java
##########
@@ -60,10 +60,10 @@
   void sendData(RpcOutcomeListener<Ack> listener, QueryDataPackage data);
 
   /**
-   * Returns the {@link ChannelFuture} which will be notified when this
+   * Returns the {@link Future} which will be notified when this
    * channel is closed.  This method always returns the same future instance.
    */
-  ChannelFuture getChannelClosureFuture();
+  Future getClosureFuture();

Review comment:
       `Future` is parametrized, please specify the `Void` type to avoid warnings. `ChannelFuture` extends `Future<Void>`.

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
##########
@@ -284,14 +282,9 @@ public void sendData(final RpcOutcomeListener<Ack> listener, final QueryDataPack
     }
 
     @Override
-    public ChannelFuture getChannelClosureFuture() {
+    public Future getClosureFuture() {

Review comment:
       And here.

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/work/prepare/PreparedStatementProvider.java
##########
@@ -244,8 +244,8 @@ public UserSession getSession() {
     }
 
     @Override
-    public ChannelFuture getChannelClosureFuture() {
-      return inner.getChannelClosureFuture();
+    public Future getClosureFuture() {

Review comment:
       And here (Void).

##########
File path: exec/memory/base/src/main/java/org/apache/drill/exec/memory/DrillByteBufAllocator.java
##########
@@ -111,31 +118,68 @@ public boolean isDirectBufferPooled() {
 
   @Override
   public ByteBuf heapBuffer() {
-    throw fail();
+    return HEAP_ALLOCATOR.allocateHeap();
   }
 
   @Override
   public ByteBuf heapBuffer(int initialCapacity) {
-    throw fail();
+    return HEAP_ALLOCATOR.allocateHeap(initialCapacity);
   }
 
   @Override
   public ByteBuf heapBuffer(int initialCapacity, int maxCapacity) {
-    throw fail();
+    return HEAP_ALLOCATOR.allocateHeap(initialCapacity, maxCapacity);
   }
 
   @Override
   public CompositeByteBuf compositeHeapBuffer() {
-    throw fail();
+    return compositeHeapBuffer(DEFAULT_MAX_COMPOSITE_COMPONENTS);
   }
 
   @Override
   public CompositeByteBuf compositeHeapBuffer(int maxNumComponents) {
-    throw fail();
-  }
-
-  private RuntimeException fail() {
-    throw new UnsupportedOperationException("Allocator doesn't support heap-based memory.");
+    return new CompositeByteBuf(this, false, maxNumComponents);
+  }
+
+  /**
+   * This method was copied from AbstractByteBufAllocator. Netty 4.1.x moved this method from
+   * AbstractByteBuf to AbstractByteBufAllocator. However, as DrillByteBufAllocator doesn't extend
+   * AbstractByteBufAllocator, it doesn't get the implementation automatically and we have to copy
+   * the codes.
+   */
+  @Override
+  public int calculateNewCapacity(int minNewCapacity, int maxCapacity) {

Review comment:
       Please remove this method copy and implementations above, and extend this class from `AbstractByteBufAllocator` (`newHeapBuffer` and `newDirectBuffer` implementations may be implemented as `HEAP_ALLOCATOR.allocateXXX`).

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebSessionResources.java
##########
@@ -58,7 +58,7 @@ public BufferAllocator getAllocator() {
     return allocator;
   }
 
-  public ChannelPromise getCloseFuture() {
+  public Promise getCloseFuture() {

Review comment:
       And here (Void).

##########
File path: exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/WebSessionResourcesTest.java
##########
@@ -65,15 +64,13 @@ public void operationComplete(Future<Void> future) throws Exception {
   @Test
   public void testChannelPromiseWithNullExecutor() throws Exception {
     try {
-      ChannelPromise closeFuture = new DefaultChannelPromise(null);
+      Promise closeFuture = new DefaultPromise(null);

Review comment:
       And here (Void).

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
##########
@@ -283,12 +283,12 @@ public WebUserConnection provide() {
         logger.trace("Failed to get the remote address of the http session request", ex);
       }
 
-      // Create a dummy close future which is needed by Foreman only. Foreman uses this future to add a close
+      // Create a close future which is needed by Foreman only. Foreman uses this future to add a close
       // listener to known about channel close event from underlying layer.
       //
       // The invocation of this close future is no-op as it will be triggered after query completion in unsecure case.
       // But we need this close future as it's expected by Foreman.
-      final ChannelPromise closeFuture = new DefaultChannelPromise(null, executor);
+      final Promise closeFuture = new DefaultPromise(executor);

Review comment:
       And here (Void).

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/BaseWebUserConnection.java
##########
@@ -40,7 +39,7 @@ public UserSession getSession() {
   }
 
   @Override
-  public ChannelFuture getChannelClosureFuture() {
+  public Future getClosureFuture() {

Review comment:
       And here.

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
##########
@@ -116,7 +115,7 @@
 
   private final ResponseSendListener responseListener = new ResponseSendListener();
   private final GenericFutureListener<Future<Void>> closeListener = future -> cancel();
-  private final ChannelFuture closeFuture;
+  private final Future closeFuture;

Review comment:
       And here (Void).

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
##########
@@ -221,11 +221,11 @@ public WebUserConnection provide() {
                 config.getLong(ExecConstants.HTTP_SESSION_MEMORY_RESERVATION),
                 config.getLong(ExecConstants.HTTP_SESSION_MEMORY_MAXIMUM));
 
-        // Create a dummy close future which is needed by Foreman only. Foreman uses this future to add a close
+        // Create a future which is needed by Foreman only. Foreman uses this future to add a close
         // listener to known about channel close event from underlying layer. We use this future to notify Foreman
         // listeners when the Web session (not connection) between Web Client and WebServer is closed. This will help
         // Foreman to cancel all the running queries for this Web Client.
-        final ChannelPromise closeFuture = new DefaultChannelPromise(null, executor);
+        final Promise closeFuture = new DefaultPromise(executor);

Review comment:
       And here (Void).

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebSessionResources.java
##########
@@ -40,10 +40,10 @@
 
   private final UserSession webUserSession;
 
-  private ChannelPromise closeFuture;
+  private Promise closeFuture;

Review comment:
       And here (Void).

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebSessionResources.java
##########
@@ -40,10 +40,10 @@
 
   private final UserSession webUserSession;
 
-  private ChannelPromise closeFuture;
+  private Promise closeFuture;
 
   WebSessionResources(BufferAllocator allocator, SocketAddress remoteAddress,
-                      UserSession userSession, ChannelPromise closeFuture) {
+                      UserSession userSession, Promise closeFuture) {

Review comment:
       And here (Void).




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



[GitHub] [drill] luocooong merged pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
luocooong merged pull request #2185:
URL: https://github.com/apache/drill/pull/2185


   


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



[GitHub] [drill] rymarm commented on a change in pull request #2185: DRILL-7790 : Build Drill with Netty version 4.1.59.Final

Posted by GitBox <gi...@apache.org>.
rymarm commented on a change in pull request #2185:
URL: https://github.com/apache/drill/pull/2185#discussion_r591253117



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
##########
@@ -284,14 +282,9 @@ public void sendData(final RpcOutcomeListener<Ack> listener, final QueryDataPack
     }
 
     @Override
-    public ChannelFuture getChannelClosureFuture() {
+    public Future getClosureFuture() {

Review comment:
       Fixed.

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/BaseWebUserConnection.java
##########
@@ -40,7 +39,7 @@ public UserSession getSession() {
   }
 
   @Override
-  public ChannelFuture getChannelClosureFuture() {
+  public Future getClosureFuture() {

Review comment:
       Fixed.

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
##########
@@ -221,11 +221,11 @@ public WebUserConnection provide() {
                 config.getLong(ExecConstants.HTTP_SESSION_MEMORY_RESERVATION),
                 config.getLong(ExecConstants.HTTP_SESSION_MEMORY_MAXIMUM));
 
-        // Create a dummy close future which is needed by Foreman only. Foreman uses this future to add a close
+        // Create a future which is needed by Foreman only. Foreman uses this future to add a close
         // listener to known about channel close event from underlying layer. We use this future to notify Foreman
         // listeners when the Web session (not connection) between Web Client and WebServer is closed. This will help
         // Foreman to cancel all the running queries for this Web Client.
-        final ChannelPromise closeFuture = new DefaultChannelPromise(null, executor);
+        final Promise closeFuture = new DefaultPromise(executor);

Review comment:
       Fixed.

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
##########
@@ -283,12 +283,12 @@ public WebUserConnection provide() {
         logger.trace("Failed to get the remote address of the http session request", ex);
       }
 
-      // Create a dummy close future which is needed by Foreman only. Foreman uses this future to add a close
+      // Create a close future which is needed by Foreman only. Foreman uses this future to add a close
       // listener to known about channel close event from underlying layer.
       //
       // The invocation of this close future is no-op as it will be triggered after query completion in unsecure case.
       // But we need this close future as it's expected by Foreman.
-      final ChannelPromise closeFuture = new DefaultChannelPromise(null, executor);
+      final Promise closeFuture = new DefaultPromise(executor);

Review comment:
       Fixed.




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