spark-reviews mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From vanzin <...@git.apache.org>
Subject [GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...
Date Mon, 24 Oct 2016 20:39:53 GMT
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/15172#discussion_r84773451
  
    --- Diff: common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
---
    @@ -0,0 +1,340 @@
    +/*
    + * 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.HashMap;
    +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";
    +
    +  private final SecretKeySpec inKeySpec;
    +  private final IvParameterSpec inIvSpec;
    +  private final SecretKeySpec outKeySpec;
    +  private final IvParameterSpec outIvSpec;
    +  private Properties properties;
    +
    +  private HashMap<ReadableByteChannel, CryptoInputStream> inputStreamMap;
    +  private HashMap<WritableByteChannel, CryptoOutputStream> outputStreamMap;
    +
    +  public static final int STREAM_BUFFER_SIZE = 1024 * 32;
    +  public static final String TRANSFORM = "AES/CTR/NoPadding";
    +
    +  public AesCipher(
    +      Properties properties,
    +      byte[] inKey,
    +      byte[] outKey,
    +      byte[] inIv,
    +      byte[] outIv) throws IOException {
    +    properties.setProperty(CryptoInputStream.STREAM_BUFFER_SIZE_KEY,
    +      String.valueOf(STREAM_BUFFER_SIZE));
    +    this.properties = properties;
    +
    +    inputStreamMap = new HashMap<>();
    +    outputStreamMap= new HashMap<>();
    +
    +    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
    +   */
    +  public CryptoOutputStream CreateOutputStream(WritableByteChannel ch) throws IOException
{
    +    if (!outputStreamMap.containsKey(ch)) {
    +      outputStreamMap.put(ch, new CryptoOutputStream(TRANSFORM, properties, ch, outKeySpec,
outIvSpec));
    +    }
    +
    +    return outputStreamMap.get(ch);
    +  }
    +
    +  /**
    +   * Create AES crypto input stream
    +   * @param ch The underlying channel used to read data.
    +   * @return Return input crypto stream for decryption.
    +   * @throws IOException
    +   */
    +  public CryptoInputStream CreateInputStream(ReadableByteChannel ch) throws IOException
{
    +    if (!inputStreamMap.containsKey(ch)) {
    +      inputStreamMap.put(ch, new CryptoInputStream(TRANSFORM, properties, ch, inKeySpec,
inIvSpec));
    +    }
    +
    +    return inputStreamMap.get(ch);
    +  }
    +
    +  /**
    +   * 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));
    +  }
    +
    +  /**
    +   * Generate a request config message which send to remote peer.
    +   * @param conf is the local transport configuration.
    +   * @return Config message for sending.
    +   */
    +  public static AesConfigMessage requestConfigMessage(TransportConf conf) {
    +    int keySize = conf.saslEncryptionAesCipherKeySizeBits();
    +    if (keySize % 8 != 0) {
    +      throw new IllegalArgumentException("The AES cipher key size in bits should be a
multiple " +
    +        "of byte");
    +    }
    +    return new AesConfigMessage(keySize/8, null, null, null, null);
    +  }
    +
    +  /**
    +   * Generate the configuration message according to request config message.
    +   * @param configMessage The request config message comes from remote.
    +   * @return Configuration message for sending.
    +   */
    +  public static AesConfigMessage responseConfigMessage(AesConfigMessage configMessage){
    +
    +    Properties properties = new Properties();
    +    int keyLen = configMessage.keySize;
    +
    +    try {
    +      int paramLen = CryptoCipherFactory.getCryptoCipher(AesCipher.TRANSFORM, properties)
    +        .getBlockSize();
    +      byte[] inKey = new byte[keyLen];
    +      byte[] outKey = new byte[keyLen];
    +      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);
    +
    +      configMessage.setParameters(keyLen, inKey, inIv, outKey, outIv);
    --- End diff --
    
    As with the previous comment, it's weird to modify input parameters. Instead, you should
be returning a new message with the response data.
    
    In fact, it feels like this method actually belongs in `SaslRpcHandler`, where it's called.
There's nothing in this method that is related to the `AesCipher` class. It also needs a better
name (e.g. `createResponseMessage`).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


Mime
View raw message