You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by sohami <gi...@git.apache.org> on 2018/03/02 01:35:08 UTC

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

GitHub user sohami opened a pull request:

    https://github.com/apache/drill/pull/1145

    DRILL-6187: Exception in RPC communication between DataClient/Control…

    …Client and respective servers when bit-to-bit security is on

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/sohami/drill DRILL-6187-2

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/1145.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1145
    
----
commit 4a7602b428ef4ef9fe358976713a78174bb82f57
Author: Sorabh Hamirwasia <sh...@...>
Date:   2018-03-01T23:08:10Z

    DRILL-6187: Exception in RPC communication between DataClient/ControlClient and respective servers when bit-to-bit security is on

----


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173529580
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BitRpcUtility.java ---
    @@ -0,0 +1,110 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.rpc;
    +
    +import com.google.common.collect.ImmutableList;
    +import com.google.protobuf.Internal.EnumLite;
    +import com.google.protobuf.MessageLite;
    +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
    +import org.apache.drill.exec.rpc.security.AuthenticatorFactory;
    +import org.apache.drill.exec.rpc.security.SaslProperties;
    +import org.apache.hadoop.security.UserGroupInformation;
    +
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Utility class providing common methods shared between {@link org.apache.drill.exec.rpc.data.DataClient} and
    + * {@link org.apache.drill.exec.rpc.control.ControlClient}
    + */
    +public final class BitRpcUtility {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitRpcUtility.class);
    +
    +  /**
    +   * Method to do validation on the handshake message received from server side. Only used by BitClients NOT UserClient.
    +   * Verify if rpc version of handshake message matches the supported RpcVersion and also validates the
    +   * security configuration between client and server
    +   * @param handshakeRpcVersion - rpc version received in handshake message
    +   * @param remoteAuthMechs - authentication mechanisms supported by server
    +   * @param rpcVersion - supported rpc version on client
    +   * @param connection - client connection
    +   * @param config - client connectin config
    +   * @param client - data client or control client
    +   * @return - Immutable list of authentication mechanisms supported by server or null
    +   * @throws RpcException - exception is thrown if rpc version or authentication configuration mismatch is found
    +   */
    +  public static List<String> validateHandshake(int handshakeRpcVersion, List<String> remoteAuthMechs, int rpcVersion,
    +                                               ClientConnection connection, BitConnectionConfig config,
    +                                               BasicClient client) throws RpcException {
    +
    +    if (handshakeRpcVersion != rpcVersion) {
    +      throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.",
    +        handshakeRpcVersion, rpcVersion));
    +    }
    +
    +    if (remoteAuthMechs.size() != 0) { // remote requires authentication
    +      client.setAuthComplete(false);
    +      return ImmutableList.copyOf(remoteAuthMechs);
    +    } else {
    +      if (config.getAuthMechanismToUse() != null) { // local requires authentication
    +        throw new RpcException(String.format("Remote Drillbit does not require auth, but auth is enabled in " +
    +          "local Drillbit configuration. [Details: connection: (%s) and LocalAuthMechanism: (%s). Please check " +
    +          "security configuration for bit-to-bit.", connection.getName(), config.getAuthMechanismToUse()));
    +      }
    +    }
    +    return null;
    +  }
    +
    +  /**
    +   * Creates various instances needed to start the SASL handshake. This is called from
    +   * {@link BasicClient#prepareSaslHandshake(RpcConnectionHandler, List)} only for
    +   * {@link org.apache.drill.exec.rpc.data.DataClient} and {@link org.apache.drill.exec.rpc.control.ControlClient}
    +   *
    +   * @param connectionHandler    - Connection handler used by client's to know about success/failure conditions.
    +   * @param serverAuthMechanisms - List of auth mechanisms configured on server side
    +   * @param connection - ClientConnection used for authentication
    +   * @param config - ClientConnection config
    +   * @param endpoint - Remote DrillbitEndpoint
    +   * @param client - Either of DataClient/ControlClient instance
    +   * @param saslRpcType - SASL_MESSAGE RpcType for Data and Control channel
    +   */
    +  public static <T extends EnumLite, CC extends ClientConnection, HS extends MessageLite, HR extends MessageLite>
    +  void prepareSaslHandshake(final RpcConnectionHandler<CC> connectionHandler, List<String> serverAuthMechanisms,
    +                            CC connection, BitConnectionConfig config, DrillbitEndpoint endpoint,
    +                            final BasicClient<T, CC, HS, HR> client, T saslRpcType) {
    +    try {
    +      final Map<String, String> saslProperties = SaslProperties.getSaslProperties(connection.isEncryptionEnabled(),
    +        connection.getMaxWrappedSize());
    +      final UserGroupInformation ugi = UserGroupInformation.getLoginUser();
    +      final AuthenticatorFactory factory = config.getAuthFactory(serverAuthMechanisms);
    +      client.startSaslHandshake(connectionHandler, config.getSaslClientProperties(endpoint, saslProperties),
    +        ugi, factory, saslRpcType);
    +    } catch (final IOException e) {
    +      logger.error("Failed while doing setup for starting sasl handshake for connection", connection.getName());
    +      final Exception ex = new RpcException(String.format("Failed to initiate authentication to %s",
    +        endpoint.getAddress()), e);
    +      connectionHandler.connectionFailed(RpcConnectionHandler.FailureType.AUTHENTICATION, ex);
    +    }
    +  }
    +
    +  // Suppress default constructor
    +  private BitRpcUtility() throws Exception {
    +    throw new UnsupportedOperationException("BitRpcUtility is a utility class and should not be instantiated.");
    +  }
    +}
    --- End diff --
    
    add LF.


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173234604
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java ---
    @@ -312,6 +312,11 @@ public synchronized void connect(String connect, Properties props) throws RpcExc
         if (connected) {
           return;
         }
    +
    +    if (props == null) {
    --- End diff --
    
    My recommendation is to change other 2 overloaded methods to pass `new Properties()` instead of `null` and making it explicit that `null` is not allowed (avoid passing `null` and checking for `null` at the same time).


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r172642452
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java ---
    @@ -371,17 +376,20 @@ protected void afterExecute(final Runnable r, final Throwable t) {
     
         while (triedEndpointIndex < connectTriesVal) {
           endpoint = endpoints.get(triedEndpointIndex);
    +
    +      // Set in both props and properties since props is passed to UserClient
    +      if (!properties.containsKey(DrillProperties.SERVICE_HOST)) {
    --- End diff --
    
    Use `putIfAbsent()` instead of `containsKey()` (avoid double get()).


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173041032
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java ---
    @@ -371,17 +376,20 @@ protected void afterExecute(final Runnable r, final Throwable t) {
     
         while (triedEndpointIndex < connectTriesVal) {
           endpoint = endpoints.get(triedEndpointIndex);
    +
    +      // Set in both props and properties since props is passed to UserClient
    +      if (!properties.containsKey(DrillProperties.SERVICE_HOST)) {
    --- End diff --
    
    `putIfAbsent` is Java 8 specific api. Today we had a discussion that until next release we don't want to bring dependency on Java 8 only api's.


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173236107
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java ---
    @@ -371,17 +376,20 @@ protected void afterExecute(final Runnable r, final Throwable t) {
     
         while (triedEndpointIndex < connectTriesVal) {
           endpoint = endpoints.get(triedEndpointIndex);
    +
    +      // Set in both props and properties since props is passed to UserClient
    +      if (!properties.containsKey(DrillProperties.SERVICE_HOST)) {
    --- End diff --
    
    - Use `put` and add `TODO` comment.
    
     - What is a reason not to use API available in the minimum supported version assuming that support for JDK 7 is dropped as part of https://issues.apache.org/jira/browse/DRILL-1491?


---

[GitHub] drill issue #1145: DRILL-6187: Exception in RPC communication between DataCl...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on the issue:

    https://github.com/apache/drill/pull/1145
  
    Made all the changes and squashed changes into 2 commits


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173041288
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java ---
    @@ -182,6 +196,66 @@ public boolean isActive() {
     
       protected abstract void validateHandshake(HR validateHandshake) throws RpcException;
     
    +  /**
    +   * Creates various instances needed to start the SASL handshake. This is called from
    +   * {@link BasicClient#validateHandshake(MessageLite)} if authentication is required from server side.
    +   * @param connectionListener
    +   * @throws RpcException
    +   */
    +  protected abstract void prepareSaslHandshake(final RpcConnectionHandler<CC> connectionListener) throws RpcException;
    --- End diff --
    
    removed from here and `startSaslHandshake`


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173237142
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java ---
    @@ -103,6 +96,23 @@ protected void handle(ControlConnection connection, int rpcType, ByteBuf pBody,
         connection.getCurrentHandler().handle(connection, rpcType, pBody, dBody, sender);
       }
     
    +  @Override
    +  protected void prepareSaslHandshake(final RpcConnectionHandler<ControlConnection> connectionListener)
    --- End diff --
    
    It is defined in `protocol`. Consider adding the dependency on the `protocol`jar.


---

[GitHub] drill issue #1145: DRILL-6187: Exception in RPC communication between DataCl...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on the issue:

    https://github.com/apache/drill/pull/1145
  
    @vrozov - Thanks for the feedback. Update the PR with latest changes. Please help to review.


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r172678483
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java ---
    @@ -103,6 +96,23 @@ protected void handle(ControlConnection connection, int rpcType, ByteBuf pBody,
         connection.getCurrentHandler().handle(connection, rpcType, pBody, dBody, sender);
       }
     
    +  @Override
    +  protected void prepareSaslHandshake(final RpcConnectionHandler<ControlConnection> connectionListener)
    --- End diff --
    
    The implementation seems to be common between Control and Data client, can it be unified here?


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173043030
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java ---
    @@ -312,6 +312,11 @@ public synchronized void connect(String connect, Properties props) throws RpcExc
         if (connected) {
           return;
         }
    +
    +    if (props == null) {
    --- End diff --
    
    Not totally sure what you mean here. Since there are 2 other overloaded methods which call's this method internally. They do pass null props and are used across multiple tests. 
    
    May be I can check for props in those method instead and create a instance of it if needed ? Then we can place NotNull tag on this connect method. That will not require to change any existing tests.


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r172718426
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java ---
    @@ -69,6 +74,11 @@
       private final IdlePingHandler pingHandler;
       private ConnectionMultiListener.SSLHandshakeListener sslHandshakeListener = null;
     
    +  // Authentication related parameters
    +  protected volatile List<String> serverAuthMechanisms = null;
    --- End diff --
    
    Is `volatile` necessary? Consider making all variables private and providing `setAuthRequired(List<String> authMechanisms)`.


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173529523
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BitRpcUtility.java ---
    @@ -0,0 +1,110 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.rpc;
    +
    +import com.google.common.collect.ImmutableList;
    +import com.google.protobuf.Internal.EnumLite;
    +import com.google.protobuf.MessageLite;
    +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
    +import org.apache.drill.exec.rpc.security.AuthenticatorFactory;
    +import org.apache.drill.exec.rpc.security.SaslProperties;
    +import org.apache.hadoop.security.UserGroupInformation;
    +
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Utility class providing common methods shared between {@link org.apache.drill.exec.rpc.data.DataClient} and
    + * {@link org.apache.drill.exec.rpc.control.ControlClient}
    + */
    +public final class BitRpcUtility {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitRpcUtility.class);
    +
    +  /**
    +   * Method to do validation on the handshake message received from server side. Only used by BitClients NOT UserClient.
    +   * Verify if rpc version of handshake message matches the supported RpcVersion and also validates the
    +   * security configuration between client and server
    +   * @param handshakeRpcVersion - rpc version received in handshake message
    +   * @param remoteAuthMechs - authentication mechanisms supported by server
    +   * @param rpcVersion - supported rpc version on client
    +   * @param connection - client connection
    +   * @param config - client connectin config
    +   * @param client - data client or control client
    +   * @return - Immutable list of authentication mechanisms supported by server or null
    +   * @throws RpcException - exception is thrown if rpc version or authentication configuration mismatch is found
    +   */
    +  public static List<String> validateHandshake(int handshakeRpcVersion, List<String> remoteAuthMechs, int rpcVersion,
    +                                               ClientConnection connection, BitConnectionConfig config,
    +                                               BasicClient client) throws RpcException {
    +
    +    if (handshakeRpcVersion != rpcVersion) {
    +      throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.",
    +        handshakeRpcVersion, rpcVersion));
    +    }
    +
    +    if (remoteAuthMechs.size() != 0) { // remote requires authentication
    +      client.setAuthComplete(false);
    +      return ImmutableList.copyOf(remoteAuthMechs);
    +    } else {
    +      if (config.getAuthMechanismToUse() != null) { // local requires authentication
    +        throw new RpcException(String.format("Remote Drillbit does not require auth, but auth is enabled in " +
    +          "local Drillbit configuration. [Details: connection: (%s) and LocalAuthMechanism: (%s). Please check " +
    +          "security configuration for bit-to-bit.", connection.getName(), config.getAuthMechanismToUse()));
    +      }
    +    }
    +    return null;
    +  }
    +
    +  /**
    +   * Creates various instances needed to start the SASL handshake. This is called from
    +   * {@link BasicClient#prepareSaslHandshake(RpcConnectionHandler, List)} only for
    +   * {@link org.apache.drill.exec.rpc.data.DataClient} and {@link org.apache.drill.exec.rpc.control.ControlClient}
    +   *
    +   * @param connectionHandler    - Connection handler used by client's to know about success/failure conditions.
    +   * @param serverAuthMechanisms - List of auth mechanisms configured on server side
    +   * @param connection - ClientConnection used for authentication
    +   * @param config - ClientConnection config
    +   * @param endpoint - Remote DrillbitEndpoint
    +   * @param client - Either of DataClient/ControlClient instance
    +   * @param saslRpcType - SASL_MESSAGE RpcType for Data and Control channel
    +   */
    +  public static <T extends EnumLite, CC extends ClientConnection, HS extends MessageLite, HR extends MessageLite>
    +  void prepareSaslHandshake(final RpcConnectionHandler<CC> connectionHandler, List<String> serverAuthMechanisms,
    +                            CC connection, BitConnectionConfig config, DrillbitEndpoint endpoint,
    +                            final BasicClient<T, CC, HS, HR> client, T saslRpcType) {
    +    try {
    +      final Map<String, String> saslProperties = SaslProperties.getSaslProperties(connection.isEncryptionEnabled(),
    +        connection.getMaxWrappedSize());
    +      final UserGroupInformation ugi = UserGroupInformation.getLoginUser();
    +      final AuthenticatorFactory factory = config.getAuthFactory(serverAuthMechanisms);
    +      client.startSaslHandshake(connectionHandler, config.getSaslClientProperties(endpoint, saslProperties),
    +        ugi, factory, saslRpcType);
    +    } catch (final IOException e) {
    +      logger.error("Failed while doing setup for starting sasl handshake for connection", connection.getName());
    +      final Exception ex = new RpcException(String.format("Failed to initiate authentication to %s",
    +        endpoint.getAddress()), e);
    +      connectionHandler.connectionFailed(RpcConnectionHandler.FailureType.AUTHENTICATION, ex);
    +    }
    +  }
    +
    +  // Suppress default constructor
    +  private BitRpcUtility() throws Exception {
    --- End diff --
    
    Exception is not required, just make the constructor private.


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173041172
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java ---
    @@ -103,6 +96,23 @@ protected void handle(ControlConnection connection, int rpcType, ByteBuf pBody,
         connection.getCurrentHandler().handle(connection, rpcType, pBody, dBody, sender);
       }
     
    +  @Override
    +  protected void prepareSaslHandshake(final RpcConnectionHandler<ControlConnection> connectionListener)
    --- End diff --
    
    `RpcType.SASL_MESSAGE` message accessed within `prepareSaslHandshake` implementation of DataClient/ControlClient is defined separately. Also each of these client except UserClient has access to ConnectionConfig which is not part of BasicClient too and is used in prepareSaslHandshake implementation. Hence I kept the implementations separate for both DataClient and ControlClient.


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r172678133
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java ---
    @@ -182,6 +196,66 @@ public boolean isActive() {
     
       protected abstract void validateHandshake(HR validateHandshake) throws RpcException;
     
    +  /**
    +   * Creates various instances needed to start the SASL handshake. This is called from
    +   * {@link BasicClient#validateHandshake(MessageLite)} if authentication is required from server side.
    +   * @param connectionListener
    +   * @throws RpcException
    +   */
    +  protected abstract void prepareSaslHandshake(final RpcConnectionHandler<CC> connectionListener) throws RpcException;
    --- End diff --
    
    None of the implementations seems to throw an RpcException.


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r172667685
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java ---
    @@ -312,6 +312,11 @@ public synchronized void connect(String connect, Properties props) throws RpcExc
         if (connected) {
           return;
         }
    +
    +    if (props == null) {
    --- End diff --
    
    Consider making `props` `@NotNull`.


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by vrozov <gi...@git.apache.org>.
Github user vrozov commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173375649
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java ---
    @@ -96,6 +96,8 @@
     
     import io.netty.channel.EventLoopGroup;
     
    +import javax.validation.constraints.NotNull;
    --- End diff --
    
    Please use java.annotation.Nonnull or org.jetbrains.annotations.NotNull (or do not annotate, it is inferred).


---

[GitHub] drill issue #1145: DRILL-6187: Exception in RPC communication between DataCl...

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on the issue:

    https://github.com/apache/drill/pull/1145
  
    +1


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/drill/pull/1145


---

[GitHub] drill pull request #1145: DRILL-6187: Exception in RPC communication between...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1145#discussion_r173041420
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java ---
    @@ -69,6 +74,11 @@
       private final IdlePingHandler pingHandler;
       private ConnectionMultiListener.SSLHandshakeListener sslHandshakeListener = null;
     
    +  // Authentication related parameters
    +  protected volatile List<String> serverAuthMechanisms = null;
    --- End diff --
    
    On second thought volatile doesn't seem necessary here since it will only be accessed by Netty's thread which is also fixed for a connection. 
    Made fields private.


---

[GitHub] drill issue #1145: DRILL-6187: Exception in RPC communication between DataCl...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on the issue:

    https://github.com/apache/drill/pull/1145
  
    @vrozov - Please help to review this PR.
    It address the concurrency issue during authentication of control/data client to server side. Rather than adding the connection into connection holder right after TCP connection is available, the listener for connection success is called after successful authentication (if needed).


---