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

    https://github.com/apache/spark/pull/15172#discussion_r87338617
  
    --- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
    @@ -0,0 +1,305 @@
    +/*
    + * 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.nio.ByteBuffer;
    +import java.nio.channels.ReadableByteChannel;
    +import java.nio.channels.WritableByteChannel;
    +import java.util.Properties;
    +import javax.crypto.spec.SecretKeySpec;
    +import javax.crypto.spec.IvParameterSpec;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.base.Throwables;
    +import io.netty.buffer.ByteBuf;
    +import io.netty.buffer.Unpooled;
    +import io.netty.channel.*;
    +import io.netty.util.AbstractReferenceCounted;
    +import org.apache.commons.crypto.cipher.CryptoCipherFactory;
    +import org.apache.commons.crypto.random.CryptoRandom;
    +import org.apache.commons.crypto.random.CryptoRandomFactory;
    +import org.apache.commons.crypto.stream.CryptoInputStream;
    +import org.apache.commons.crypto.stream.CryptoOutputStream;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.util.ByteArrayReadableChannel;
    +import org.apache.spark.network.util.ByteArrayWritableChannel;
    +import org.apache.spark.network.util.TransportConf;
    +
    +/**
    + * AES cipher for encryption and decryption.
    + */
    +public class AesCipher {
    +  private static final Logger logger = 
LoggerFactory.getLogger(AesCipher.class);
    +  public static final String ENCRYPTION_HANDLER_NAME = "AesEncryption";
    +  public static final String DECRYPTION_HANDLER_NAME = "AesDecryption";
    +  public static final int STREAM_BUFFER_SIZE = 1024 * 32;
    +  public static final String TRANSFORM = "AES/CTR/NoPadding";
    +
    +  private final SecretKeySpec inKeySpec;
    +  private final IvParameterSpec inIvSpec;
    +  private final SecretKeySpec outKeySpec;
    +  private final IvParameterSpec outIvSpec;
    +  private final Properties properties;
    +
    +  public AesCipher(
    +      Properties properties,
    +      byte[] inKey,
    +      byte[] outKey,
    +      byte[] inIv,
    +      byte[] outIv) throws IOException {
    +    this.properties = properties;
    +    inKeySpec = new SecretKeySpec(inKey, "AES");
    +    inIvSpec = new IvParameterSpec(inIv);
    +    outKeySpec = new SecretKeySpec(outKey, "AES");
    +    outIvSpec = new IvParameterSpec(outIv);
    +  }
    +
    +  public AesCipher(AesConfigMessage configMessage) throws IOException  {
    +    this(new Properties(), configMessage.inKey, configMessage.outKey,
    +      configMessage.inIv, configMessage.outIv);
    +  }
    +
    +  /**
    +   * Create AES crypto output stream
    +   * @param ch The underlying channel to write out.
    +   * @return Return output crypto stream for encryption.
    +   * @throws IOException
    +   */
    +  private CryptoOutputStream createOutputStream(WritableByteChannel ch) 
throws IOException {
    +    return new CryptoOutputStream(TRANSFORM, properties, ch, outKeySpec, 
outIvSpec);
    +  }
    +
    +  /**
    +   * Create AES crypto input stream
    +   * @param ch The underlying channel used to read data.
    +   * @return Return input crypto stream for decryption.
    +   * @throws IOException
    +   */
    +  private CryptoInputStream createInputStream(ReadableByteChannel ch) 
throws IOException {
    +    return new CryptoInputStream(TRANSFORM, properties, ch, inKeySpec, 
inIvSpec);
    +  }
    +
    +  /**
    +   * Add handlers to channel
    +   * @param ch the channel for adding handlers
    +   * @throws IOException
    +   */
    +  public void addToChannel(Channel ch) throws IOException {
    +    ch.pipeline()
    +      .addFirst(ENCRYPTION_HANDLER_NAME, new AesEncryptHandler(this))
    +      .addFirst(DECRYPTION_HANDLER_NAME, new AesDecryptHandler(this));
    +  }
    +
    +  /**
    +   * Create the configuration message
    +   * @param conf is the local transport configuration.
    +   * @return Config message for sending.
    +   */
    +  public static AesConfigMessage createConfigMessage(TransportConf conf) {
    +    int keySize = conf.aesCipherKeySize();
    +    Properties properties = CryptoStreamUtils.toCryptoConf(conf);
    +
    +    try {
    +      int paramLen = 
CryptoCipherFactory.getCryptoCipher(AesCipher.TRANSFORM, properties)
    +        .getBlockSize();
    +      byte[] inKey = new byte[keySize];
    +      byte[] outKey = new byte[keySize];
    +      byte[] inIv = new byte[paramLen];
    +      byte[] outIv = new byte[paramLen];
    +
    +      CryptoRandom random = 
CryptoRandomFactory.getCryptoRandom(properties);
    +      random.nextBytes(inKey);
    +      random.nextBytes(outKey);
    +      random.nextBytes(inIv);
    +      random.nextBytes(outIv);
    +
    +      return new AesConfigMessage(keySize, inKey, inIv, outKey, outIv);
    +    } catch (Exception e) {
    +      logger.error("AES config error", e);
    +      throw Throwables.propagate(e);
    +    }
    +  }
    +
    +  /**
    +   * CryptoStreamUtils is used to convert config from TransportConf to AES 
Crypto config.
    +   */
    +  private static class CryptoStreamUtils {
    +    public static Properties toCryptoConf(TransportConf conf) {
    +      Properties props = new Properties();
    +      if (conf.aesCipherClass() != null) {
    +        props.setProperty(CryptoCipherFactory.CLASSES_KEY, 
conf.aesCipherClass());
    +      }
    +      return props;
    +    }
    +  }
    +
    +  private static class AesEncryptHandler extends 
ChannelOutboundHandlerAdapter {
    +    private final ByteArrayWritableChannel byteChannel;
    +    private final CryptoOutputStream cos;
    +
    +    AesEncryptHandler(AesCipher cipher) throws IOException {
    +      byteChannel = new 
ByteArrayWritableChannel(AesCipher.STREAM_BUFFER_SIZE);
    +      cos = cipher.createOutputStream(byteChannel);
    +    }
    +
    +    @Override
    +    public void write(ChannelHandlerContext ctx, Object msg, 
ChannelPromise promise)
    +      throws Exception {
    +      ctx.write(new EncryptedMessage(cos, msg, byteChannel), promise);
    +    }
    +
    +    @Override
    +    public void close(ChannelHandlerContext ctx, ChannelPromise promise) 
throws Exception {
    +      try {
    +        cos.close();
    +      } finally {
    +        super.close(ctx, promise);
    +      }
    +    }
    +  }
    +
    +  private static class AesDecryptHandler extends 
ChannelInboundHandlerAdapter {
    +    private final CryptoInputStream cis;
    +    private final ByteArrayReadableChannel byteChannel;
    +
    +    AesDecryptHandler(AesCipher cipher) throws IOException {
    +      byteChannel = new ByteArrayReadableChannel();
    +      cis = cipher.createInputStream(byteChannel);
    +    }
    +
    +    @Override
    +    public void channelRead(ChannelHandlerContext ctx, Object data) throws 
Exception {
    +      ByteBuf in = (ByteBuf) data;
    +      byteChannel.feedData(in);
    --- End diff --
    
    Does `Apache Common Crypto` provide any API to decrypt `ByteBuffer`? If so, 
could you use it because the code will be much simpler (e.g., 
`ByteArrayReadableChannel` is totally not needed.)?


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