Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/15172#discussion_r83513580
  
    --- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java
 ---
    @@ -71,56 +75,78 @@
         this.secretKeyHolder = secretKeyHolder;
         this.saslServer = null;
         this.isComplete = false;
    +    this.isAuthenticated = false;
       }
     
       @Override
       public void receive(TransportClient client, ByteBuffer message, 
RpcResponseCallback callback) {
    +    boolean encrypt = conf.saslServerAlwaysEncrypt();
         if (isComplete) {
           // Authentication complete, delegate to base handler.
           delegate.receive(client, message, callback);
           return;
         }
    +    if (saslServer == null || !saslServer.isComplete()) {
    +      ByteBuf nettyBuf = Unpooled.wrappedBuffer(message);
    +      SaslMessage saslMessage;
    +      try {
    +        saslMessage = SaslMessage.decode(nettyBuf);
    +      } finally {
    +        nettyBuf.release();
    +      }
     
    -    ByteBuf nettyBuf = Unpooled.wrappedBuffer(message);
    -    SaslMessage saslMessage;
    -    try {
    -      saslMessage = SaslMessage.decode(nettyBuf);
    -    } finally {
    -      nettyBuf.release();
    -    }
    -
    -    if (saslServer == null) {
    -      // First message in the handshake, setup the necessary state.
    -      client.setClientId(saslMessage.appId);
    -      saslServer = new SparkSaslServer(saslMessage.appId, secretKeyHolder,
    -        conf.saslServerAlwaysEncrypt());
    -    }
    +      if (saslServer == null) {
    +        // First message in the handshake, setup the necessary state.
    +        client.setClientId(saslMessage.appId);
    +        saslServer = new SparkSaslServer(saslMessage.appId, 
secretKeyHolder, encrypt);
    +      }
     
    -    byte[] response;
    -    try {
    -      response = saslServer.response(JavaUtils.bufferToArray(
    -        saslMessage.body().nioByteBuffer()));
    -    } catch (IOException ioe) {
    -      throw new RuntimeException(ioe);
    +      byte[] response;
    +      try {
    +        response = saslServer.response(JavaUtils.bufferToArray(
    +          saslMessage.body().nioByteBuffer()));
    +      } catch (IOException ioe) {
    +        throw new RuntimeException(ioe);
    +      }
    +      callback.onSuccess(ByteBuffer.wrap(response));
         }
    -    callback.onSuccess(ByteBuffer.wrap(response));
     
         // Setup encryption after the SASL response is sent, otherwise the 
client can't parse the
         // response. It's ok to change the channel pipeline here since we are 
processing an incoming
         // message, so the pipeline is busy and no new incoming messages will 
be fed to it before this
         // method returns. This assumes that the code ensures, through other 
means, that no outbound
         // messages are being written to the channel while negotiation is 
still going on.
         if (saslServer.isComplete()) {
    -      logger.debug("SASL authentication successful for channel {}", 
client);
    -      isComplete = true;
           if 
(SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP)))
 {
    -        logger.debug("Enabling encryption for channel {}", client);
    -        SaslEncryption.addToChannel(channel, saslServer, 
conf.maxSaslEncryptedBlockSize());
    -        saslServer = null;
    +        try {
    +          if (conf.saslEncryptionAesEnabled()) {
    +            // Extra negotiation should happen after authentication, so 
return directly while
    +            // processing authenticate.
    +            if (!isAuthenticated) {
    +              logger.debug("SASL authentication successful for channel 
{}", client);
    +              isAuthenticated = true;
    +              return ;
    +            } else {
    +              Object result = saslServer.negotiate(message, callback, 
conf);
    +              if (result instanceof AesCipher) {
    +                logger.debug("Enabling AES cipher for Server channel {}", 
client);
    +                AesEncryption.addToChannel(channel, (AesCipher) result);
    +              }
    +            }
    +          } else {
    +            logger.info("Enabling encryption for channel {}", client);
    +            SaslEncryption.addToChannel(channel, saslServer, 
conf.maxSaslEncryptedBlockSize());
    +          }
    +          saslServer = null;
    +        } catch (SaslException e) {
    --- End diff --
    
    Why are you catching this exception and doing nothing with it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to