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

    https://github.com/apache/spark/pull/15172#discussion_r80304406
  
    --- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/SparkAesCipher.java
 ---
    @@ -0,0 +1,270 @@
    +/*
    + * 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.spark.network.sasl.aes;
    +
    +import java.io.IOException;
    +import java.security.InvalidAlgorithmParameterException;
    +import java.security.InvalidKeyException;
    +import java.security.NoSuchAlgorithmException;
    +import java.util.Arrays;
    +import java.util.Properties;
    +import javax.crypto.Cipher;
    +import javax.crypto.Mac;
    +import javax.crypto.SecretKey;
    +import javax.crypto.ShortBufferException;
    +import javax.crypto.spec.SecretKeySpec;
    +import javax.crypto.spec.IvParameterSpec;
    +import javax.security.sasl.SaslException;
    +
    +import org.apache.commons.crypto.cipher.CryptoCipher;
    +import org.apache.commons.crypto.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * AES cipher for encryption and decryption.
    + */
    +public class SparkAesCipher {
    +  private static final Logger logger = 
LoggerFactory.getLogger(SparkAesCipher.class);
    +  private static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
    +  public final static String supportedTransformation[] = {
    +    "AES/CBC/NoPadding", "AES/CTR/NoPadding"
    +  };
    +
    +  private final CryptoCipher encryptor;
    +  private final CryptoCipher decryptor;
    +
    +  private final Integrity integrity;
    +
    +  public SparkAesCipher(
    +      String cipherTransformation,
    +      Properties properties,
    +      byte[] inKey,
    +      byte[] outKey,
    +      byte[] inIv,
    +      byte[] outIv) throws IOException {
    +    if 
(!Arrays.asList(supportedTransformation).contains(cipherTransformation)) {
    +      logger.warn("AES cipher transformation is not supported: " + 
cipherTransformation);
    +      cipherTransformation = "AES/CTR/NoPadding";
    +      logger.warn("Use default AES/CTR/NoPadding");
    +    }
    +
    +    final SecretKeySpec inKeySpec = new SecretKeySpec(inKey, "AES");
    +    final IvParameterSpec inIvSpec = new IvParameterSpec(inIv);
    +    final SecretKeySpec outKeySpec = new SecretKeySpec(outKey, "AES");
    +    final IvParameterSpec outIvSpec = new IvParameterSpec(outIv);
    +
    +    // Encryptor
    +    encryptor = Utils.getCipherInstance(cipherTransformation, properties);
    +    try {
    +      logger.debug("Initialize encryptor");
    +      encryptor.init(Cipher.ENCRYPT_MODE, outKeySpec, outIvSpec);
    +    } catch (InvalidKeyException | InvalidAlgorithmParameterException e) {
    +      throw new IOException("Failed to initialize encryptor", e);
    +    }
    +
    +    // Decryptor
    +    decryptor = Utils.getCipherInstance(cipherTransformation, properties);
    +    try {
    +      logger.debug("Initialize decryptor");
    +      decryptor.init(Cipher.DECRYPT_MODE, inKeySpec, inIvSpec);
    +    } catch (InvalidKeyException | InvalidAlgorithmParameterException e) {
    +      throw new IOException("Failed to initialize decryptor", e);
    +    }
    +
    +    integrity = new Integrity(outKey, inKey);
    +  }
    +
    +  /**
    +   * Encrypts input data. The result composes of (msg, padding if needed, 
mac) and sequence num.
    +   * @param data the input byte array
    +   * @param offset the offset in input where the input starts
    +   * @param len the input length
    +   * @return the new encrypted byte array.
    +   * @throws SaslException if error happens
    +   */
    +  public byte[] wrap(byte[] data, int offset, int len) throws 
SaslException {
    +    byte[] mac = integrity.getHMAC(data, offset, len);
    +    integrity.incMySeqNum();
    +
    +    // Padding based on cipher
    +    byte[] padding;
    +    if ("AES/CBC/NoPadding".equals(encryptor.getAlgorithm())) {
    +      int bs = encryptor.getBlockSize();
    +      int pad = bs - (len + 10) % bs;
    +      padding = new byte[pad];
    +      for (int i = 0; i < pad; i ++) {
    +        padding[i] = (byte) pad;
    +      }
    +    } else {
    +      padding = EMPTY_BYTE_ARRAY;
    +    }
    +
    +    // Encrypt
    +    byte[] encrypted = new byte[len + 10 + padding.length + 4];
    +    try {
    +      int n = encryptor.update(data, offset, len, encrypted, 0);
    +      n += encryptor.update(padding, 0, padding.length, encrypted, n);
    +      encryptor.update(mac, 0, 10, encrypted, n);
    +    } catch (ShortBufferException sbe) {
    +      // This should not happen
    +      throw new SaslException("Error happens during encrypt data", sbe);
    +    }
    +
    +    // Append seqNum used for mac
    +    System.arraycopy(integrity.getSeqNum(), 0, encrypted, encrypted.length 
- 4, 4);
    +
    +    return encrypted;
    +  }
    +
    +  /**
    +   * Decrypt input data. The input composes of (msg, padding if needed, 
mac) and sequence num.
    +   * The result is msg.
    +   * @param data the input byte array
    +   * @param offset the offset in input where the input starts
    +   * @param len the input length
    +   * @return the new decrypted byte array.
    +   * @throws SaslException if error happens
    +   */
    +  public byte[] unwrap(byte[] data, int offset, int len) throws 
SaslException {
    +    // Get plaintext and seqNum
    +    byte[] decrypted = new byte[len - 4];
    +    byte[] peerSeqNum = new byte[4];
    +    try {
    +      decryptor.update(data, offset, len - 4, decrypted, 0);
    +    } catch (ShortBufferException sbe) {
    +      throw new SaslException("Error happens during decrypt data", sbe);
    +    }
    +    System.arraycopy(data, offset + decrypted.length, peerSeqNum, 0, 4);
    +
    +    // Get msg and mac
    +    byte[] msg = new byte[decrypted.length - 10];
    +    byte[] mac = new byte[10];
    +
    +    System.arraycopy(decrypted, msg.length, mac, 0, 10);
    --- End diff --
    
    There's a lot of allocation / copying going on here... using `ByteBuffer`, 
or even sticking to `byte[]` but with APIs that allow you to provide offset + 
count, would be better.


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