[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-11 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/15172


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-10 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87446000
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java
 ---
@@ -80,46 +84,74 @@ public void receive(TransportClient client, ByteBuffer 
message, RpcResponseCallb
   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,
+  conf.saslServerAlwaysEncrypt());
+  }
 
-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)))
 {
+  if 
(!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP)))
 {
+logger.debug("SASL authentication successful for channel {}", 
client);
--- End diff --

No, the message is correct. This means auth was successful and encryption 
(`auth-conf`) isn't being used.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-10 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87389888
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for encryption negotiation.
+ */
+public class AesConfigMessage implements Encodable {
+  /** Serialization tag used to catch incorrect payloads. */
+  private static final byte TAG_BYTE = (byte) 0xEB;
+
+  public int keySize;
+  public byte[] inKey;
+  public byte[] outKey;
+  public byte[] inIv;
+  public byte[] outIv;
+
+  public AesConfigMessage(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
+this.keySize = keySize;
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  @Override
+  public int encodedLength() {
+return 1 + 4 + ((inKey != null && inIv != null && outKey != null && 
outIv != null) ?
+  Encoders.ByteArrays.encodedLength(inKey) + 
Encoders.ByteArrays.encodedLength(inKey) +
+  Encoders.ByteArrays.encodedLength(inIv) + 
Encoders.ByteArrays.encodedLength(outIv) : 0);
+  }
+
+  @Override
+  public void encode(ByteBuf buf) {
+buf.writeByte(TAG_BYTE);
+buf.writeInt(keySize);
+if (inKey != null && inIv != null && outKey != null && outIv != null) {
+  Encoders.ByteArrays.encode(buf, inKey);
+  Encoders.ByteArrays.encode(buf, inIv);
+  Encoders.ByteArrays.encode(buf, outKey);
+  Encoders.ByteArrays.encode(buf, outIv);
+}
+  }
+
+  /**
+   * Encode the config message.
+   * @return ByteBuffer which contains encoded config message.
+   */
+  public ByteBuffer encodeMessage(){
+ByteBuffer buf = ByteBuffer.allocate(encodedLength());
+
+ByteBuf wrappedBuf = Unpooled.wrappedBuffer(buf);
+wrappedBuf.clear();
+encode(wrappedBuf);
+
+return buf;
+  }
+
+  /**
+   * Decode the config message from buffer
+   * @param buffer the buffer contain encoded config message
+   * @return config message
+   */
+  public static AesConfigMessage decodeMessage(ByteBuffer buffer) {
+ByteBuf buf = Unpooled.wrappedBuffer(buffer);
+
+if (buf.readByte() != TAG_BYTE) {
+  throw new IllegalStateException("Expected AesConfigMessage, received 
something else"
++ " (maybe your client does not have AES enabled?)");
+}
+
+int keySize = buf.readInt();
+
+byte[] outKey = Encoders.ByteArrays.decode(buf);
--- End diff --

client use inKey to encrypt, server use inKey to decrypt, So exchange the 
inkey and outkey here.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-10 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87389461
  
--- 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-10 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87348637
  
--- 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87336716
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java
 ---
@@ -0,0 +1,73 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.util.LinkedList;
+
+import io.netty.buffer.ByteBuf;
+
+public class ByteArrayReadableChannel implements ReadableByteChannel {
+  private final LinkedList buffers = new LinkedList<>();
--- End diff --

Why not make`ByteArrayReadableChannel` be a wrapper for a single `ByteBuf`? 
Then the code will be much simpler. I don't see it's necessary to use 
`LinkedList`.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87337925
  
--- 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;
--- End diff --

This constructor is not used outside. Could you remove 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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87337577
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for encryption negotiation.
+ */
+public class AesConfigMessage implements Encodable {
+  /** Serialization tag used to catch incorrect payloads. */
+  private static final byte TAG_BYTE = (byte) 0xEB;
+
+  public int keySize;
+  public byte[] inKey;
+  public byte[] outKey;
+  public byte[] inIv;
+  public byte[] outIv;
+
+  public AesConfigMessage(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
+this.keySize = keySize;
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  @Override
+  public int encodedLength() {
+return 1 + 4 + ((inKey != null && inIv != null && outKey != null && 
outIv != null) ?
+  Encoders.ByteArrays.encodedLength(inKey) + 
Encoders.ByteArrays.encodedLength(inKey) +
--- End diff --

nit: Encoders.ByteArrays.encodedLength(inKey) + 
Encoders.ByteArrays.encodedLength(**outKey**) +


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87335532
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java
 ---
@@ -80,46 +84,74 @@ public void receive(TransportClient client, ByteBuffer 
message, RpcResponseCallb
   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,
+  conf.saslServerAlwaysEncrypt());
+  }
 
-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)))
 {
+  if 
(!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP)))
 {
+logger.debug("SASL authentication successful for channel {}", 
client);
--- End diff --

nit: this is a failure.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87337169
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for encryption negotiation.
+ */
+public class AesConfigMessage implements Encodable {
+  /** Serialization tag used to catch incorrect payloads. */
+  private static final byte TAG_BYTE = (byte) 0xEB;
+
+  public int keySize;
+  public byte[] inKey;
+  public byte[] outKey;
+  public byte[] inIv;
+  public byte[] outIv;
+
+  public AesConfigMessage(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
+this.keySize = keySize;
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  @Override
+  public int encodedLength() {
+return 1 + 4 + ((inKey != null && inIv != null && outKey != null && 
outIv != null) ?
+  Encoders.ByteArrays.encodedLength(inKey) + 
Encoders.ByteArrays.encodedLength(inKey) +
+  Encoders.ByteArrays.encodedLength(inIv) + 
Encoders.ByteArrays.encodedLength(outIv) : 0);
+  }
+
+  @Override
+  public void encode(ByteBuf buf) {
+buf.writeByte(TAG_BYTE);
+buf.writeInt(keySize);
+if (inKey != null && inIv != null && outKey != null && outIv != null) {
+  Encoders.ByteArrays.encode(buf, inKey);
+  Encoders.ByteArrays.encode(buf, inIv);
+  Encoders.ByteArrays.encode(buf, outKey);
+  Encoders.ByteArrays.encode(buf, outIv);
+}
+  }
+
+  /**
+   * Encode the config message.
+   * @return ByteBuffer which contains encoded config message.
+   */
+  public ByteBuffer encodeMessage(){
+ByteBuffer buf = ByteBuffer.allocate(encodedLength());
+
+ByteBuf wrappedBuf = Unpooled.wrappedBuffer(buf);
+wrappedBuf.clear();
+encode(wrappedBuf);
+
+return buf;
+  }
+
+  /**
+   * Decode the config message from buffer
+   * @param buffer the buffer contain encoded config message
+   * @return config message
+   */
+  public static AesConfigMessage decodeMessage(ByteBuffer buffer) {
+ByteBuf buf = Unpooled.wrappedBuffer(buffer);
+
+if (buf.readByte() != TAG_BYTE) {
+  throw new IllegalStateException("Expected AesConfigMessage, received 
something else"
++ " (maybe your client does not have AES enabled?)");
+}
+
+int keySize = buf.readInt();
+
+byte[] outKey = Encoders.ByteArrays.decode(buf);
--- End diff --

why the `decode` order is not same as the `encode` order?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread zsxwing
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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87337029
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for encryption negotiation.
+ */
+public class AesConfigMessage implements Encodable {
+  /** Serialization tag used to catch incorrect payloads. */
+  private static final byte TAG_BYTE = (byte) 0xEB;
+
+  public int keySize;
+  public byte[] inKey;
+  public byte[] outKey;
+  public byte[] inIv;
+  public byte[] outIv;
+
+  public AesConfigMessage(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
+this.keySize = keySize;
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  @Override
+  public int encodedLength() {
+return 1 + 4 + ((inKey != null && inIv != null && outKey != null && 
outIv != null) ?
--- End diff --

nit: since keySize is just `inKey.length`. You don't need to send it via 
the network.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87336929
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for encryption negotiation.
+ */
+public class AesConfigMessage implements Encodable {
+  /** Serialization tag used to catch incorrect payloads. */
+  private static final byte TAG_BYTE = (byte) 0xEB;
+
+  public int keySize;
+  public byte[] inKey;
+  public byte[] outKey;
+  public byte[] inIv;
+  public byte[] outIv;
+
+  public AesConfigMessage(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
+this.keySize = keySize;
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  @Override
+  public int encodedLength() {
+return 1 + 4 + ((inKey != null && inIv != null && outKey != null && 
outIv != null) ?
--- End diff --

nit: don't allow null values then the codes will be simpler.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87336379
  
--- 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87334433
  
--- 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87328090
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java
 ---
@@ -88,9 +90,26 @@ public void doBootstrap(TransportClient client, Channel 
channel) {
   throw new RuntimeException(
 new SaslException("Encryption requests by negotiated 
non-encrypted connection."));
 }
-SaslEncryption.addToChannel(channel, saslClient, 
conf.maxSaslEncryptedBlockSize());
+
+if (conf.aesEncryptionEnabled()) {
+  // Generate a request config message to send to server.
+  AesConfigMessage configMessage = 
AesCipher.createConfigMessage(conf);
+  ByteBuffer buf = configMessage.encodeMessage();
+
+  // Encrypted the config message.
+  ByteBuffer encrypted = ByteBuffer.wrap(
+saslClient.wrap(buf.array(), 0, buf.array().length));
--- End diff --

Really thanks:)


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87320958
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java
 ---
@@ -88,9 +90,20 @@ public void doBootstrap(TransportClient client, Channel 
channel) {
   throw new RuntimeException(
 new SaslException("Encryption requests by negotiated 
non-encrypted connection."));
 }
-SaslEncryption.addToChannel(channel, saslClient, 
conf.maxSaslEncryptedBlockSize());
+
+if (conf.aesEncryptionEnabled()) {
+  // Generate a request config message to send to server.
+  AesConfigMessage configMessage = 
AesCipher.createConfigMessage(conf);
+  ByteBuffer buf = configMessage.encodeMessage();
+  client.sendRpcSync(buf, conf.saslRTTimeoutMs());
+  AesCipher cipher = new AesCipher(configMessage);
+  logger.info("Enabling AES cipher for client channel {}", client);
+  cipher.addToChannel(channel);
--- End diff --

the dispose is called in the finally block.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87295828
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java
 ---
@@ -88,9 +90,20 @@ public void doBootstrap(TransportClient client, Channel 
channel) {
   throw new RuntimeException(
 new SaslException("Encryption requests by negotiated 
non-encrypted connection."));
 }
-SaslEncryption.addToChannel(channel, saslClient, 
conf.maxSaslEncryptedBlockSize());
+
+if (conf.aesEncryptionEnabled()) {
+  // Generate a request config message to send to server.
+  AesConfigMessage configMessage = 
AesCipher.createConfigMessage(conf);
+  ByteBuffer buf = configMessage.encodeMessage();
+  client.sendRpcSync(buf, conf.saslRTTimeoutMs());
+  AesCipher cipher = new AesCipher(configMessage);
+  logger.info("Enabling AES cipher for client channel {}", client);
+  cipher.addToChannel(channel);
--- End diff --

You need to dispose of the `saslClient` here.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r87295803
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java
 ---
@@ -88,9 +90,20 @@ public void doBootstrap(TransportClient client, Channel 
channel) {
   throw new RuntimeException(
 new SaslException("Encryption requests by negotiated 
non-encrypted connection."));
 }
-SaslEncryption.addToChannel(channel, saslClient, 
conf.maxSaslEncryptedBlockSize());
+
+if (conf.aesEncryptionEnabled()) {
+  // Generate a request config message to send to server.
+  AesConfigMessage configMessage = 
AesCipher.createConfigMessage(conf);
+  ByteBuffer buf = configMessage.encodeMessage();
+  client.sendRpcSync(buf, conf.saslRTTimeoutMs());
--- End diff --

Wait. Isn't this now sending the keys in plain text over the wire?

I commented before that using 3des to encrypt the keys is sub-optimal, but 
I didn't mean for you to remove the encryption altogether.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-07 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86913509
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java
 ---
@@ -0,0 +1,75 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.util.LinkedList;
+
+import io.netty.buffer.ByteBuf;
+
+public class ByteArrayReadableChannel implements ReadableByteChannel {
+  private final LinkedList buffers = new LinkedList<>();
+
+  public int readableBytes() {
+if (!buffers.isEmpty()) {
+  return buffers.getFirst().readableBytes();
+} else {
+  return 0;
+}
+  }
+
+  public ByteArrayReadableChannel() {
+  }
+
+  public void feedData(ByteBuf buf) {
+buffers.add(buf);
+  }
+
+  @Override
+  public int read(ByteBuffer dst) throws IOException {
+if (!buffers.isEmpty()) {
--- End diff --

It can be a while, just need a variable to track the total readable bytes. 
Actually, the while loop in channelRead do the same thing.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86849616
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java
 ---
@@ -0,0 +1,75 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.util.LinkedList;
+
+import io.netty.buffer.ByteBuf;
+
+public class ByteArrayReadableChannel implements ReadableByteChannel {
+  private final LinkedList buffers = new LinkedList<>();
+
+  public int readableBytes() {
+if (!buffers.isEmpty()) {
+  return buffers.getFirst().readableBytes();
+} else {
+  return 0;
+}
+  }
+
+  public ByteArrayReadableChannel() {
+  }
+
+  public void feedData(ByteBuf buf) {
+buffers.add(buf);
+  }
+
+  @Override
+  public int read(ByteBuffer dst) throws IOException {
+if (!buffers.isEmpty()) {
--- End diff --

This could be a while loop, right? So you fill the `dst` buffer with as 
much data as you can.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86850250
  
--- Diff: 
common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java
 ---
@@ -374,6 +375,69 @@ public void testDelegates() throws Exception {
 }
   }
 
+  @Test
+  public void testSaslEncryptionAes() throws Exception {
--- End diff --

nit: `testAesEncryption`


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86845462
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,298 @@
+/*
+ * 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 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
   

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86850097
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
 ---
@@ -175,4 +176,26 @@ public boolean saslServerAlwaysEncrypt() {
 return conf.getBoolean("spark.network.sasl.serverAlwaysEncrypt", 
false);
   }
 
+  /**
+   * The trigger for enabling AES encryption.
+   */
+  public boolean aesEncryptionEnabled() {
+return conf.getBoolean("spark.authenticate.encryption.aes.enabled", 
false);
+  }
+
+  /**
+   * The implementation class for crypto cipher
+   */
+  public String aesCipherClass() {
+return conf.get("spark.authenticate.encryption.aes.cipher.class",
+  CryptoCipherFactory.CipherProvider.OPENSSL.getClassName());
--- End diff --

This does not match the default value of the library. You should not use a 
default here; just use "null" as a trigger for "do not set a value", and let 
the library deal with the default.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86848275
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,298 @@
+/*
+ * 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 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
   

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86848702
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,298 @@
+/*
+ * 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 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
   

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86849329
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java
 ---
@@ -0,0 +1,75 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.util.LinkedList;
+
+import io.netty.buffer.ByteBuf;
+
+public class ByteArrayReadableChannel implements ReadableByteChannel {
+  private final LinkedList buffers = new LinkedList<>();
+
+  public int readableBytes() {
+if (!buffers.isEmpty()) {
+  return buffers.getFirst().readableBytes();
+} else {
+  return 0;
+}
+  }
+
+  public ByteArrayReadableChannel() {
--- End diff --

nit: no need to declare this if it's not doing anything.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86844899
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,298 @@
+/*
+ * 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 Properties properties;
--- End diff --

final?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-01 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86059917
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,298 @@
+/*
+ * 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 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
+   */
+  public 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
+   */
+  public CryptoInputStream createInputStream(ReadableByteChannel ch) 
throws IOException {
--- End diff --

Can be private?


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

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-01 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86060480
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,298 @@
+/*
+ * 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 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
+   */
+  public 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
+   */
+  public 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

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-01 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86060203
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,298 @@
+/*
+ * 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 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
+   */
+  public 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
+   */
+  public 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

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-01 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86059907
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,298 @@
+/*
+ * 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 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
+   */
+  public CryptoOutputStream createOutputStream(WritableByteChannel ch) 
throws IOException {
--- End diff --

This can be private, right?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-11-01 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r86060416
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,298 @@
+/*
+ * 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 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
+   */
+  public 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
+   */
+  public 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

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85875690
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java
 ---
@@ -80,46 +84,71 @@ public void receive(TransportClient client, ByteBuffer 
message, RpcResponseCallb
   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,
+  conf.saslServerAlwaysEncrypt());
+  }
 
-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)))
 {
+  if 
(!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP)))
 {
+logger.debug("SASL authentication successful for channel {}", 
client);
+complete(true);
+return ;
+  }
+
+  if (!conf.AesEncryptionEnabled()) {
 logger.debug("Enabling encryption for channel {}", client);
 SaslEncryption.addToChannel(channel, saslServer, 
conf.maxSaslEncryptedBlockSize());
-saslServer = null;
-  } else {
-saslServer.dispose();
-saslServer = null;
+complete(false);
+return;
+  }
+
+  // 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;
+  }
+
+  // Create AES cipher when it is authenticated
+  try {
+AesConfigMessage configMessage = 
AesConfigMessage.decodeMessage(message);
+AesCipher cipher = new AesCipher(configMessage);
+
+// Send response back to client to confirm that server accept 
config.
+callback.onSuccess(JavaUtils.stringToBytes(AesCipher.TRANSFORM));
--- End diff --

This should be call after AddtoChannel, otherwise it will be sent with 
encrypted data while at the moment client is not ready for decryption.


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

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85850906
  
--- Diff: 
common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java
 ---
@@ -374,6 +375,69 @@ public void testDelegates() throws Exception {
 }
   }
 
+  @Test
+  public void testSaslEncryptionAes() throws Exception {
+final AtomicReference response = new 
AtomicReference<>();
+final File file = File.createTempFile("sasltest", ".txt");
+SaslTestCtx ctx = null;
+try {
+  final TransportConf conf = new TransportConf("rpc", new 
SystemPropertyConfigProvider());
+  final TransportConf spyConf = spy(conf);
+  doReturn(true).when(spyConf).AesEncryptionEnabled();
+
+  StreamManager sm = mock(StreamManager.class);
+  when(sm.getChunk(anyLong(), anyInt())).thenAnswer(new 
Answer() {
+@Override
+public ManagedBuffer answer(InvocationOnMock invocation) {
+  return new FileSegmentManagedBuffer(spyConf, file, 0, 
file.length());
+}
+  });
+
+  RpcHandler rpcHandler = mock(RpcHandler.class);
+  when(rpcHandler.getStreamManager()).thenReturn(sm);
+
+  byte[] data = new byte[ 256 * 1024 * 1024];
--- End diff --

nit: no space after `[`


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85846395
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java
 ---
@@ -80,46 +84,71 @@ public void receive(TransportClient client, ByteBuffer 
message, RpcResponseCallb
   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,
+  conf.saslServerAlwaysEncrypt());
+  }
 
-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)))
 {
+  if 
(!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP)))
 {
+logger.debug("SASL authentication successful for channel {}", 
client);
+complete(true);
+return ;
--- End diff --

nit: no space before `;`


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85848802
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  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 {
+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
+   */
+  public 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
+   */
+  public 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85848940
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  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 {
+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
+   */
+  public 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
+   */
+  public 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85847613
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  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 {
+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
+   */
+  public 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
+   */
+  public CryptoInputStream CreateInputStream(ReadableByteChannel ch) 
throws IOException {
--- End diff --

nit: lower case.


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

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85847089
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java
 ---
@@ -80,46 +84,71 @@ public void receive(TransportClient client, ByteBuffer 
message, RpcResponseCallb
   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,
+  conf.saslServerAlwaysEncrypt());
+  }
 
-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)))
 {
+  if 
(!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP)))
 {
+logger.debug("SASL authentication successful for channel {}", 
client);
+complete(true);
+return ;
+  }
+
+  if (!conf.AesEncryptionEnabled()) {
 logger.debug("Enabling encryption for channel {}", client);
 SaslEncryption.addToChannel(channel, saslServer, 
conf.maxSaslEncryptedBlockSize());
-saslServer = null;
-  } else {
-saslServer.dispose();
-saslServer = null;
+complete(false);
+return;
+  }
+
+  // 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;
+  }
+
+  // Create AES cipher when it is authenticated
+  try {
+AesConfigMessage configMessage = 
AesConfigMessage.decodeMessage(message);
+AesCipher cipher = new AesCipher(configMessage);
+
+// Send response back to client to confirm that server accept 
config.
+callback.onSuccess(JavaUtils.stringToBytes(AesCipher.TRANSFORM));
--- End diff --

This should really be the last statement (after `complete(true)`).

It's a little weird to use the name of the transformation as the success 
message here... I'd say a reply is not even needed, but it's nice for the 
client to know that this particular message succeeded or not. So it's ok to use 
this, but I'd prefer if the client ignored the contents of the reply and 
instead just handled exceptions for the error case.


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

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85849596
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for encryption negotiation.
+ */
+public class AesConfigMessage implements Encodable {
+  /** Serialization tag used to catch incorrect payloads. */
+  private static final byte TAG_BYTE = (byte) 0xEB;
+
+  public int keySize;
+  public byte[] inKey;
+  public byte[] outKey;
+  public byte[] inIv;
+  public byte[] outIv;
+
+  public AesConfigMessage(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
+this.keySize = keySize;
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  @Override
+  public int encodedLength() {
+return 1 + 4 + ((inKey != null && inIv != null && outKey != null && 
outIv != null) ?
+  Encoders.ByteArrays.encodedLength(inKey) + 
Encoders.ByteArrays.encodedLength(inKey) +
+  Encoders.ByteArrays.encodedLength(inIv) + 
Encoders.ByteArrays.encodedLength(outIv) : 0);
+  }
+
+  @Override
+  public void encode(ByteBuf buf) {
+buf.writeByte(TAG_BYTE);
+buf.writeInt(keySize);
+if (inKey != null && inIv != null && outKey != null && outIv != null) {
+  Encoders.ByteArrays.encode(buf, inKey);
+  Encoders.ByteArrays.encode(buf, inIv);
+  Encoders.ByteArrays.encode(buf, outKey);
+  Encoders.ByteArrays.encode(buf, outIv);
+}
+  }
+
+  /**
+   * Encode the config message.
+   * @return ByteBuffer which contains encoded config message.
+   */
+  public ByteBuffer encodeMessage(){
+ByteBuffer buf = ByteBuffer.allocate(encodedLength());
+
+ByteBuf wrappedBuf = Unpooled.wrappedBuffer(buf);
+wrappedBuf.clear();
+encode(wrappedBuf);
+
+return buf;
+  }
+
+  /**
+   * Decode the config message from buffer
+   * @param buffer the buffer contain encoded config message
+   * @return config message
+   */
+  public static AesConfigMessage decodeMessage(ByteBuffer buffer) {
+ByteBuf buf = Unpooled.wrappedBuffer(buffer);
+
+if (buf.readByte() != TAG_BYTE) {
+  throw new IllegalStateException("Expected SaslMessage, received 
something else"
--- End diff --

s/SaslMessage/AesConfigMessage/


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85850400
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java
 ---
@@ -0,0 +1,71 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ByteArrayReadableChannel implements ReadableByteChannel {
+  private byte[] backArray;
+  private ByteBuf data;
+
+  public int length() {
+return data.readableBytes();
+  }
+
+  public void reset() {
+data.clear();
+  }
+
+  public ByteArrayReadableChannel(int size) {
+backArray = new byte[size];
+data = Unpooled.wrappedBuffer(backArray);
+data.clear();
+  }
+
+  public void feedData(ByteBuf buf) {
+int toFeed = Math.min(data.writableBytes(), buf.readableBytes());
+buf.readBytes(data, toFeed);
--- End diff --

Ok for now, but I wonder if it wouldn't be better to keep a list of buffers 
internally (like `TransportFrameDecoder`) instead of copying. Then instead of 
two copies (one here, one in `read`) you could have just one (in `read`).


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85849036
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  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 {
+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
+   */
+  public 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
+   */
+  public 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85849245
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  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 {
+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
+   */
+  public 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
+   */
+  public 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85849868
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java
 ---
@@ -0,0 +1,71 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
--- End diff --

nit: too many empty lines


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85848236
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  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 {
+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
+   */
+  public 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
+   */
+  public 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85847230
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  public static final int STREAM_BUFFER_SIZE = 1024 * 32;
--- End diff --

nit: move static fields before instance fields.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85849098
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  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 {
+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
+   */
+  public 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
+   */
+  public 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85847599
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  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 {
+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
+   */
+  public CryptoOutputStream CreateOutputStream(WritableByteChannel ch) 
throws IOException {
--- End diff --

nit: method names should start with lower-case letters.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85849759
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for encryption negotiation.
+ */
+public class AesConfigMessage implements Encodable {
+  /** Serialization tag used to catch incorrect payloads. */
+  private static final byte TAG_BYTE = (byte) 0xEB;
+
+  public int keySize;
+  public byte[] inKey;
+  public byte[] outKey;
+  public byte[] inIv;
+  public byte[] outIv;
+
+  public AesConfigMessage(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
+this.keySize = keySize;
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  @Override
+  public int encodedLength() {
+return 1 + 4 + ((inKey != null && inIv != null && outKey != null && 
outIv != null) ?
+  Encoders.ByteArrays.encodedLength(inKey) + 
Encoders.ByteArrays.encodedLength(inKey) +
+  Encoders.ByteArrays.encodedLength(inIv) + 
Encoders.ByteArrays.encodedLength(outIv) : 0);
+  }
+
+  @Override
+  public void encode(ByteBuf buf) {
+buf.writeByte(TAG_BYTE);
+buf.writeInt(keySize);
+if (inKey != null && inIv != null && outKey != null && outIv != null) {
+  Encoders.ByteArrays.encode(buf, inKey);
+  Encoders.ByteArrays.encode(buf, inIv);
+  Encoders.ByteArrays.encode(buf, outKey);
+  Encoders.ByteArrays.encode(buf, outIv);
+}
+  }
+
+  /**
+   * Encode the config message.
+   * @return ByteBuffer which contains encoded config message.
+   */
+  public ByteBuffer encodeMessage(){
+ByteBuffer buf = ByteBuffer.allocate(encodedLength());
+
+ByteBuf wrappedBuf = Unpooled.wrappedBuffer(buf);
+wrappedBuf.clear();
+encode(wrappedBuf);
+
+return buf;
+  }
+
+  /**
+   * Decode the config message from buffer
+   * @param buffer the buffer contain encoded config message
+   * @return config message
+   */
+  public static AesConfigMessage decodeMessage(ByteBuffer buffer) {
+ByteBuf buf = Unpooled.wrappedBuffer(buffer);
+
+if (buf.readByte() != TAG_BYTE) {
+  throw new IllegalStateException("Expected SaslMessage, received 
something else"
++ " (maybe your client does not have SASL enabled?)");
--- End diff --

If I understand correctly, the cause for this would be a server that does 
not support AES encryption, and a client that wants 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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85850499
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  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 {
+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
+   */
+  public 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
+   */
+  public 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85848147
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java
 ---
@@ -0,0 +1,311 @@
+/*
+ * 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";
+
+  private final SecretKeySpec inKeySpec;
+  private final IvParameterSpec inIvSpec;
+  private final SecretKeySpec outKeySpec;
+  private final IvParameterSpec outIvSpec;
+  private Properties properties;
+
+  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 {
+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
+   */
+  public 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
+   */
+  public 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 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85851023
  
--- Diff: docs/configuration.md ---
@@ -1435,6 +1435,21 @@ Apart from these, the following properties are also 
available, and may be useful
   
 
 
+  spark.authenticate.encryption.aes.enabled
+  false
+  
+Enable AES for over-the-wire encryption
+  
+
+
+  
spark.authenticate.encryption.aes.cipher.keySizeBits
+  16
+  
+The bytes of AES cipher key which is effective when AES cipher is 
enabled. AES
+works with 16, 24 and 32 bit keys.
--- End diff --

This is bytes, not bits, right?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r85845790
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java
 ---
@@ -88,9 +90,25 @@ public void doBootstrap(TransportClient client, Channel 
channel) {
   throw new RuntimeException(
 new SaslException("Encryption requests by negotiated 
non-encrypted connection."));
 }
-SaslEncryption.addToChannel(channel, saslClient, 
conf.maxSaslEncryptedBlockSize());
+
+if (conf.AesEncryptionEnabled()) {
--- End diff --

nit: method names should start with lower-case letters.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-25 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84947832
  
--- 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 inputStreamMap;
+  private HashMap 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 {

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84774034
  
--- 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 inputStreamMap;
+  private HashMap 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 {

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84776932
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java
 ---
@@ -80,47 +84,79 @@ public void receive(TransportClient client, ByteBuffer 
message, RpcResponseCallb
   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,
+  conf.saslServerAlwaysEncrypt());
+  }
 
-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)))
 {
+  if 
(!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP)))
 {
+logger.debug("SASL authentication successful for channel {}", 
client);
+saslServer.dispose();
+saslServer = null;
+isComplete = true;
+return ;
+  }
+
+  if (!conf.saslEncryptionAesEnabled()) {
 logger.debug("Enabling encryption for channel {}", client);
 SaslEncryption.addToChannel(channel, saslServer, 
conf.maxSaslEncryptedBlockSize());
-saslServer = null;
   } else {
-saslServer.dispose();
-saslServer = null;
+// 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 {
+  try {
+AesConfigMessage configMessage = 
AesConfigMessage.decodeMessage(message);
+configMessage = AesCipher.responseConfigMessage(configMessage);
+AesCipher cipher = new AesCipher(configMessage);
+
+ByteBuffer buf = configMessage.encodeMessage();
+
+// Encrypt the config message.
+ByteBuffer encrypted = ByteBuffer.wrap(
+  saslServer.wrap(buf.array(), 0, buf.array().length));
+callback.onSuccess(encrypted);
--- End diff --

At this point you're not using `saslServer` anymore, so you should dispose 
of it. I kinda prefer my previous suggestion where instead of `if..else` you 
have just `if`s and `return`s. I think it makes it a little easier to follow 
the code.

You can abstract the `saslServer` cleanup by adding a `complete(boolean 
dispose)` method that updates `isComplete` and `saslServer` and is called from 
this method 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84773772
  
--- 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 inputStreamMap;
+  private HashMap 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 {

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
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 inputStreamMap;
+  private HashMap 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 {

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84774791
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java
 ---
@@ -0,0 +1,124 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for SASL encryption negotiation.
+ */
+public class AesConfigMessage implements Encodable {
+  /** Serialization tag used to catch incorrect payloads. */
+  private static final byte TAG_BYTE = (byte) 0xEB;
+
+  public int keySize;
+  public byte[] inKey;
+  public byte[] outKey;
+  public byte[] inIv;
+  public byte[] outIv;
+
+  public AesConfigMessage(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
+this.keySize = keySize;
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  /**
+   * Set key and input vector for cipher option
+   * @param keySize the size of key in byte.
+   * @param inKey The decrypt key of one side.
+   * @param inIv The input vector of one side.
+   * @param outKey The decrypt key of another side.
+   * @param outIv The input vector of another side.
+   */
+  public void setParameters(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
+this.keySize = keySize;
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  @Override
+  public int encodedLength() {
+return 1 + 4 + ((inKey != null && inIv != null && outKey != null && 
outIv != null) ?
+  Encoders.ByteArrays.encodedLength(inKey) + 
Encoders.ByteArrays.encodedLength(inKey) +
+  Encoders.ByteArrays.encodedLength(inIv) + 
Encoders.ByteArrays.encodedLength(outIv) : 0);
+  }
+
+  @Override
+  public void encode(ByteBuf buf) {
+buf.writeByte(TAG_BYTE);
+buf.writeInt(keySize);
+if (inKey != null && inIv != null && outKey != null && outIv != null) {
+  Encoders.ByteArrays.encode(buf, inKey);
+  Encoders.ByteArrays.encode(buf, inIv);
+  Encoders.ByteArrays.encode(buf, outKey);
+  Encoders.ByteArrays.encode(buf, outIv);
+}
+  }
+
+  /**
+   * Encode the config message.
+   * @return ByteBuffer which contains encoded config message.
+   */
+  public ByteBuffer encodeMessage(){
+ByteBuffer buf = ByteBuffer.allocate(encodedLength());
+
+ByteBuf wrappedBuf = Unpooled.wrappedBuffer(buf);
+wrappedBuf.clear();
+encode(wrappedBuf);
+
+return buf;
+  }
+
+  /**
+   * Decode the config message from buffer
+   * @param buffer the buffer contain encoded config message
+   * @return config message
+   */
+  public static AesConfigMessage decodeMessage(ByteBuffer buffer) {
+ByteBuf buf = Unpooled.wrappedBuffer(buffer);
+
+if (buf.readByte() != TAG_BYTE) {
+  throw new IllegalStateException("Expected SaslMessage, received 
something else"
++ " (maybe your client does not have SASL enabled?)");
+}
+
+int keySize = buf.readInt();
+
+if (buf.readableBytes() > 0) {
--- End diff --

So... you need this check because the client is sending an empty 
`AesConfigMessage` to the server, which then creates the keys and sends a 
response back to the client. Instead, why not save the round trip and have the 
client generate the keys and send a single message to the server, without 
requiring a response? Then the overloading of `AesConfigMessage` as both 
request and response goes away, and things become less 

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84773832
  
--- 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 inputStreamMap;
+  private HashMap 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 {

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84772351
  
--- 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 inputStreamMap;
--- End diff --

Could you change the code so that these maps are not necessary?

You have a single `AesCipher` instance for each channel. So having a map 
here is already weird since this class won't ever have to deal with more than a 
single channel. You seem to be using it as a cache so you don't have to figure 
out how to propagate the input streams to the channel handlers; instead, just 
have the channel handlers hold a reference to the stream somehow.

(BTW I already asked for this in my previous review. In fact, you don't 
need this map at all currently for the output stream since you already did some 
of that work.)


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84777993
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java
 ---
@@ -0,0 +1,74 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ByteArrayReadableChannel implements ReadableByteChannel {
+  private final Logger logger = 
LoggerFactory.getLogger(ByteArrayReadableChannel.class);
+
+  private byte[] backArray;
+  private ByteBuf data;
+
+  public int length() {
+return data.readableBytes();
+  }
+
+  public void reset() {
+data.clear();
+  }
+
+  public ByteArrayReadableChannel(int size) {
+backArray = new byte[size];
+data = Unpooled.wrappedBuffer(backArray);
+data.clear();
+  }
+
+  public void feedData(ByteBuf buf) {
+int toFeed = Math.min(data.writableBytes(), buf.readableBytes());
+buf.readBytes(data, toFeed);
+logger.debug("ByteReadableChannel: get {} bytes", toFeed);
--- End diff --

nit: unless this is super useful I'd remove any logging calls from this 
code path.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84770543
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java
 ---
@@ -88,7 +90,35 @@ public void doBootstrap(TransportClient client, Channel 
channel) {
   throw new RuntimeException(
 new SaslException("Encryption requests by negotiated 
non-encrypted connection."));
 }
-SaslEncryption.addToChannel(channel, saslClient, 
conf.maxSaslEncryptedBlockSize());
+
+if (conf.saslEncryptionAesEnabled()) {
+  // Generate a request config message to send to server.
+  AesConfigMessage reqConfigMessage = 
AesCipher.requestConfigMessage(conf);
+  ByteBuffer buf = reqConfigMessage.encodeMessage();
+
+  ByteBuffer response = client.sendRpcSync(buf, 
conf.saslRTTimeoutMs());
+
+  // Decrypt the config message.
+  ByteBuffer decrypted = ByteBuffer.wrap(
+saslClient.unwrap(response.array(), 0, 
response.array().length));
+
+  AesConfigMessage configMessage = 
AesConfigMessage.decodeMessage(decrypted);
+
+  // Exchange the key and IV
+  configMessage.setParameters(
+configMessage.keySize,
+configMessage.outKey,
+configMessage.outIv,
+configMessage.inKey,
+configMessage.inIv
+  );
+
+  AesCipher cipher = new AesCipher(configMessage);
+  logger.info("Enabling AES cipher for client channel {}", client);
+  cipher.addToChannel(channel);
+} else {
+  SaslEncryption.addToChannel(channel, saslClient, 
conf.maxSaslEncryptedBlockSize());
+}
 saslClient = null;
 logger.debug("Channel {} configured for SASL encryption.", client);
--- End diff --

nit: remove `SASL` since it's no longer accurate now.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r8418
  
--- Diff: 
common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java
 ---
@@ -374,6 +378,71 @@ public void testDelegates() throws Exception {
 }
   }
 
+  @Test
+  public void testSaslEncryptionAes() throws Exception {
+System.setProperty(BLOCK_SIZE_CONF, "10k");
--- End diff --

There are exceptions that can be thrown before these properties are cleaned 
up. Can you try to avoid using system properties for this?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84773561
  
--- 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 inputStreamMap;
+  private HashMap 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 {

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84772631
  
--- 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 inputStreamMap;
+  private HashMap 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,
--- End diff --

It's generally bad practice to modify input parameters. Are you sure this 
is ok?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r8487
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java
 ---
@@ -0,0 +1,74 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ByteArrayReadableChannel implements ReadableByteChannel {
+  private final Logger logger = 
LoggerFactory.getLogger(ByteArrayReadableChannel.class);
--- End diff --

Same: loggers should be static.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84770349
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java
 ---
@@ -88,7 +90,35 @@ public void doBootstrap(TransportClient client, Channel 
channel) {
   throw new RuntimeException(
 new SaslException("Encryption requests by negotiated 
non-encrypted connection."));
 }
-SaslEncryption.addToChannel(channel, saslClient, 
conf.maxSaslEncryptedBlockSize());
+
+if (conf.saslEncryptionAesEnabled()) {
+  // Generate a request config message to send to server.
+  AesConfigMessage reqConfigMessage = 
AesCipher.requestConfigMessage(conf);
+  ByteBuffer buf = reqConfigMessage.encodeMessage();
+
+  ByteBuffer response = client.sendRpcSync(buf, 
conf.saslRTTimeoutMs());
+
+  // Decrypt the config message.
+  ByteBuffer decrypted = ByteBuffer.wrap(
+saslClient.unwrap(response.array(), 0, 
response.array().length));
+
+  AesConfigMessage configMessage = 
AesConfigMessage.decodeMessage(decrypted);
+
+  // Exchange the key and IV
+  configMessage.setParameters(
--- End diff --

Why is this call necessary? Shouldn't these fields have been populated by 
the `decodeMessage` call above?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84769927
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java
 ---
@@ -88,7 +90,35 @@ public void doBootstrap(TransportClient client, Channel 
channel) {
   throw new RuntimeException(
 new SaslException("Encryption requests by negotiated 
non-encrypted connection."));
 }
-SaslEncryption.addToChannel(channel, saslClient, 
conf.maxSaslEncryptedBlockSize());
+
+if (conf.saslEncryptionAesEnabled()) {
+  // Generate a request config message to send to server.
+  AesConfigMessage reqConfigMessage = 
AesCipher.requestConfigMessage(conf);
+  ByteBuffer buf = reqConfigMessage.encodeMessage();
+
+  ByteBuffer response = client.sendRpcSync(buf, 
conf.saslRTTimeoutMs());
+
+  // Decrypt the config message.
+  ByteBuffer decrypted = ByteBuffer.wrap(
+saslClient.unwrap(response.array(), 0, 
response.array().length));
--- End diff --

Use `JavaUtils.bufferToArray`.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84773959
  
--- 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 inputStreamMap;
+  private HashMap 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 {

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r84774148
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java
 ---
@@ -0,0 +1,124 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for SASL encryption negotiation.
+ */
+public class AesConfigMessage implements Encodable {
+  /** Serialization tag used to catch incorrect payloads. */
+  private static final byte TAG_BYTE = (byte) 0xEB;
+
+  public int keySize;
+  public byte[] inKey;
+  public byte[] outKey;
+  public byte[] inIv;
+  public byte[] outIv;
+
+  public AesConfigMessage(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
+this.keySize = keySize;
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  /**
+   * Set key and input vector for cipher option
+   * @param keySize the size of key in byte.
+   * @param inKey The decrypt key of one side.
+   * @param inIv The input vector of one side.
+   * @param outKey The decrypt key of another side.
+   * @param outIv The input vector of another side.
+   */
+  public void setParameters(int keySize, byte[] inKey, byte[] inIv, byte[] 
outKey, byte[] outIv) {
--- End diff --

See previous comments. It would be better if this method did not exist.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83516644
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipherOption.java
 ---
@@ -0,0 +1,94 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for SASL encryption negotiation.
+ */
+public class AesCipherOption implements Encodable {
--- End diff --

I'd name this `AesCipherConfigMessage` or something that makes it more 
explicit that it's a protocol message.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83514083
  
--- 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()) {
--- End diff --

I think this would be easier to follow like this:

```
if 
(!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP)))
 {
  // No encryption negotiated. Code around L145
  return;
}

if (!conf.saslEncryptionAesEnabled()) {
  // non-aes stuff
  return;
}

if (!isAuthenticated) {
  // code around current L125
}

// Here goes code to finish the AES key negotiation.
```

You'll need to set `isComplete = true` in the appropriate places too 
(instead of the single statement at the end currently).


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83516418
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesEncryption.java
 ---
@@ -0,0 +1,176 @@
+/*
+ * 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.channels.WritableByteChannel;
+
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.*;
+import io.netty.util.AbstractReferenceCounted;
+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;
+
+public class AesEncryption {
+  public static final String ENCRYPTION_HANDLER_NAME = "AesEncryption";
+  public static final String DECRYPTION_HANDLER_NAME = "AesDecryption";
+
+  public static void addToChannel(Channel ch, AesCipher cipher) {
+ch.pipeline().addFirst(ENCRYPTION_HANDLER_NAME, new 
AesEncryptHandler(cipher))
+  .addFirst(DECRYPTION_HANDLER_NAME, new AesDecryptHandler(cipher));
+  }
+
+  private static class AesEncryptHandler extends 
ChannelOutboundHandlerAdapter {
+private final AesCipher cipher;
+private ByteArrayWritableChannel byteChannel;
+
+AesEncryptHandler(AesCipher cipher) {
+  this.cipher = cipher;
+  byteChannel = new 
ByteArrayWritableChannel(AesCipher.STREAM_BUFFER_SIZE);
--- End diff --

Not sure why you need `ByteArrayWritableChannel`? `CryptoOutputStream` is 
already a `WritableByteChannel`, can't you write directly to it in 
`EncryptMessage`?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83514872
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
 ---
@@ -150,6 +161,62 @@ public synchronized void dispose() {
   }
 
   /**
+   * Negotiate with peer for extended options, such as using AES cipher.
+   * @param message is message receive from peer which may contains 
communication parameters.
+   * @param callback is rpc callback.
+   * @param conf contains transport configuration.
+   * @return Object which represent the result of negotiate.
+   */
+  public Object negotiate(ByteBuffer message, RpcResponseCallback 
callback, TransportConf conf)
+throws SaslException {
+AesCipher cipher;
+
+// Receive initial option from client
+AesCipherOption cipherOption = 
AesCipherOption.decode(Unpooled.wrappedBuffer(message));
+String transformation = AesCipher.TRANSFORM;
+Properties properties = new Properties();
+
+try {
+  // Generate key and iv
+  if (conf.saslEncryptionAesCipherKeySizeBits() % 8 != 0) {
+throw new IllegalArgumentException("The AES cipher key size in 
bits should be a multiple " +
+  "of byte");
+  }
+
+  int keyLen = conf.saslEncryptionAesCipherKeySizeBits() / 8;
+  int paramLen = 
CryptoCipherFactory.getCryptoCipher(transformation,properties).getBlockSize();
+  byte[] inKey = new byte[keyLen];
+  byte[] outKey = new byte[keyLen];
+  byte[] inIv = new byte[paramLen];
+  byte[] outIv = new byte[paramLen];
+
+  // Get the 'CryptoRandom' instance.
--- End diff --

nit: comment is unnecessary.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
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.

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83515559
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipherOption.java
 ---
@@ -0,0 +1,94 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.protocol.Encoders;
+
+/**
+ * The AES cipher options for SASL encryption negotiation.
+ */
+public class AesCipherOption implements Encodable {
+  /** Serialization tag used to catch incorrect payloads. */
+  private static final byte TAG_BYTE = (byte) 0xEB;
+
+  public byte[] inKey;
+  public byte[] outKey;
+  public byte[] inIv;
+  public byte[] outIv;
+
+  public AesCipherOption() {
+this(null, null, null, null);
+  }
+  public AesCipherOption(byte[] inKey, byte[] inIv, byte[] outKey, byte[] 
outIv) {
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  /**
+   * Set key and input vector for cipher option
+   * @param inKey The decrypt key of one side
+   * @param inIv The input vector of one side
+   * @param outKey The decrypt key of another side
+   * @param outIv The input vector of another side
+   */
+  public void setParameters(byte[] inKey, byte[] inIv, byte[] outKey, 
byte[] outIv) {
+this.inKey = inKey;
+this.inIv = inIv;
+this.outKey = outKey;
+this.outIv = outIv;
+  }
+
+  @Override
+  public int encodedLength() {
+return 1 + ((inKey != null && inIv != null && outKey != null && outIv 
!= null) ?
+  Encoders.ByteArrays.encodedLength(inKey) + 
Encoders.ByteArrays.encodedLength(inKey) +
+  Encoders.ByteArrays.encodedLength(inIv) + 
Encoders.ByteArrays.encodedLength(outIv) : 0);
+  }
+
+  @Override
+  public void encode(ByteBuf buf) {
+buf.writeByte(TAG_BYTE);
+if (inKey != null && inIv != null && outKey != null && outIv != null) {
+  Encoders.ByteArrays.encode(buf, inKey);
+  Encoders.ByteArrays.encode(buf, inIv);
+  Encoders.ByteArrays.encode(buf, outKey);
+  Encoders.ByteArrays.encode(buf, outIv);
+}
+  }
+
+  public static AesCipherOption decode(ByteBuf buf) {
+if (buf.readByte() != TAG_BYTE) {
+  throw new IllegalStateException("Expected SaslMessage, received 
something else"
++ " (maybe your client does not have SASL enabled?)");
+}
+
+if (buf.isReadable()) {
--- End diff --

Why do you need this check?

If you just let exceptions propagate, you wouldn't need the empty 
constructor, and you could remove all the null checks from this code.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83515287
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
 ---
@@ -150,6 +161,62 @@ public synchronized void dispose() {
   }
 
   /**
+   * Negotiate with peer for extended options, such as using AES cipher.
+   * @param message is message receive from peer which may contains 
communication parameters.
+   * @param callback is rpc callback.
+   * @param conf contains transport configuration.
+   * @return Object which represent the result of negotiate.
+   */
+  public Object negotiate(ByteBuffer message, RpcResponseCallback 
callback, TransportConf conf)
+throws SaslException {
+AesCipher cipher;
+
+// Receive initial option from client
+AesCipherOption cipherOption = 
AesCipherOption.decode(Unpooled.wrappedBuffer(message));
+String transformation = AesCipher.TRANSFORM;
+Properties properties = new Properties();
+
+try {
+  // Generate key and iv
+  if (conf.saslEncryptionAesCipherKeySizeBits() % 8 != 0) {
+throw new IllegalArgumentException("The AES cipher key size in 
bits should be a multiple " +
+  "of byte");
+  }
+
+  int keyLen = conf.saslEncryptionAesCipherKeySizeBits() / 8;
+  int paramLen = 
CryptoCipherFactory.getCryptoCipher(transformation,properties).getBlockSize();
+  byte[] inKey = new byte[keyLen];
+  byte[] outKey = new byte[keyLen];
+  byte[] inIv = new byte[paramLen];
+  byte[] outIv = new byte[paramLen];
+
+  // Get the 'CryptoRandom' instance.
+  CryptoRandom random = 
CryptoRandomFactory.getCryptoRandom(properties);
+  random.nextBytes(inKey);
+  random.nextBytes(outKey);
+  random.nextBytes(inIv);
+  random.nextBytes(outIv);
+
+  // Update cipher option for client. The key is encrypted.
+  cipherOption.setParameters(wrap(inKey, 0, inKey.length), inIv,
+wrap(outKey, 0, outKey.length), outIv);
+
+  // Enable AES on saslServer
+  cipher = new AesCipher(properties, inKey, outKey, inIv, outIv);
+
+  // Send cipher option to client
+  ByteBuf buf = Unpooled.buffer(cipherOption.encodedLength());
+  cipherOption.encode(buf);
+  callback.onSuccess(buf.nioBuffer());
+} catch (Exception e) {
+  logger.error("AES negotiation exception: ", e);
--- End diff --

nit: `...exception.` (the colon is not necessary and looks weird in the 
logged message)


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83517577
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java
 ---
@@ -0,0 +1,90 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+
+import io.netty.buffer.ByteBuf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ByteArrayReadableChannel implements ReadableByteChannel {
+  private final Logger logger = 
LoggerFactory.getLogger(ByteArrayReadableChannel.class);
+
+  private byte[] data;
+  private int offset;
+
+  public byte[] getData() {
+return data;
+  }
+
+  public int length() {
+return data.length;
+  }
+
+  public void reset() {
+offset = 0;
+data = null;
+  }
+
+  public ByteArrayReadableChannel() {
+
+  }
+
+  public void feedData(ByteBuf buf) {
+int length = buf.readableBytes();
+
+if (buf.hasArray()) {
+  data = buf.array();
+  buf.skipBytes(length);
+} else {
+  data = new byte[length];
--- End diff --

Do you need to do this using byte arrays? Why not use 
`ByteBuffer.put(ByteBuffer)`?

Also this code looks very suspicious. You're overwriting `data` but you 
don't reset `offset`. Which tells me that you're trying to handle short writes 
and doing it incorrectly.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83514258
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java
 ---
@@ -120,6 +130,38 @@ public synchronized void dispose() {
   }
 
   /**
+   * Negotiate extra encryption options for SASL
--- End diff --

"for AES"?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83513717
  
--- 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();
--- End diff --

nit: move this later closer to where it's used; avoids the method call when 
not necessary.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83514212
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java
 ---
@@ -32,9 +34,17 @@
 
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
--- End diff --

nit: remove this empty line


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83514397
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java
 ---
@@ -120,6 +130,38 @@ public synchronized void dispose() {
   }
 
   /**
+   * Negotiate extra encryption options for SASL
+   * @param client is transport client used to connect to peer.
+   * @param conf contain client transport configuration.
+   * @throws IOException
+   * @return The object represent the result of negotiate.
+   */
+  public Object negotiate(TransportClient client, TransportConf conf) 
throws IOException {
--- End diff --

Since this is now AES-specific, how about changing the name to 
`negotiateSessionKey` or something, and returning `AesCipher`? Then you 
wouldn't need the `instanceof` checks in the callers, which are kinda weird.

Another alternative would be to have this method return some new interface 
with a `addToChannel` method, and have different implementations for the 
SASL-based encryption and AES. But that feels like overkill right now.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83516052
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesEncryption.java
 ---
@@ -0,0 +1,176 @@
+/*
+ * 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.channels.WritableByteChannel;
+
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.*;
+import io.netty.util.AbstractReferenceCounted;
+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;
+
+public class AesEncryption {
+  public static final String ENCRYPTION_HANDLER_NAME = "AesEncryption";
+  public static final String DECRYPTION_HANDLER_NAME = "AesDecryption";
+
+  public static void addToChannel(Channel ch, AesCipher cipher) {
+ch.pipeline().addFirst(ENCRYPTION_HANDLER_NAME, new 
AesEncryptHandler(cipher))
+  .addFirst(DECRYPTION_HANDLER_NAME, new AesDecryptHandler(cipher));
+  }
+
+  private static class AesEncryptHandler extends 
ChannelOutboundHandlerAdapter {
+private final AesCipher cipher;
+private ByteArrayWritableChannel byteChannel;
+
+AesEncryptHandler(AesCipher cipher) {
+  this.cipher = cipher;
+  byteChannel = new 
ByteArrayWritableChannel(AesCipher.STREAM_BUFFER_SIZE);
+}
+
+@Override
+public void write(ChannelHandlerContext ctx, Object msg, 
ChannelPromise promise)
+  throws Exception {
+  ctx.write(new EncryptMessage(cipher, msg, byteChannel), promise);
+}
+  }
+
+  private static class AesDecryptHandler extends 
ChannelInboundHandlerAdapter {
+private final AesCipher cipher;
+private CryptoInputStream cis;
+private ByteArrayReadableChannel byteChannel;
+
+AesDecryptHandler(AesCipher cipher) {
+  this.cipher = cipher;
+  byteChannel = new ByteArrayReadableChannel();
+}
+
+@Override
+public void channelRead(ChannelHandlerContext ctx, Object data) throws 
Exception {
+  ByteBuf in = (ByteBuf) data;
+
+  try {
+while (in.isReadable()) {
+  byteChannel.feedData(in);
+  cis = cipher.CreateInputStream(byteChannel);
--- End diff --

Why isn't this done once in the constructor? Then you can explicitly close 
it in the `channelInactive` callback, and avoid the cache in `AesCipher` too.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83515036
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
 ---
@@ -150,6 +161,62 @@ public synchronized void dispose() {
   }
 
   /**
+   * Negotiate with peer for extended options, such as using AES cipher.
+   * @param message is message receive from peer which may contains 
communication parameters.
+   * @param callback is rpc callback.
+   * @param conf contains transport configuration.
+   * @return Object which represent the result of negotiate.
+   */
+  public Object negotiate(ByteBuffer message, RpcResponseCallback 
callback, TransportConf conf)
+throws SaslException {
+AesCipher cipher;
+
+// Receive initial option from client
+AesCipherOption cipherOption = 
AesCipherOption.decode(Unpooled.wrappedBuffer(message));
+String transformation = AesCipher.TRANSFORM;
+Properties properties = new Properties();
+
+try {
+  // Generate key and iv
+  if (conf.saslEncryptionAesCipherKeySizeBits() % 8 != 0) {
+throw new IllegalArgumentException("The AES cipher key size in 
bits should be a multiple " +
+  "of byte");
+  }
+
+  int keyLen = conf.saslEncryptionAesCipherKeySizeBits() / 8;
+  int paramLen = 
CryptoCipherFactory.getCryptoCipher(transformation,properties).getBlockSize();
+  byte[] inKey = new byte[keyLen];
+  byte[] outKey = new byte[keyLen];
+  byte[] inIv = new byte[paramLen];
+  byte[] outIv = new byte[paramLen];
+
+  // Get the 'CryptoRandom' instance.
+  CryptoRandom random = 
CryptoRandomFactory.getCryptoRandom(properties);
+  random.nextBytes(inKey);
+  random.nextBytes(outKey);
+  random.nextBytes(inIv);
+  random.nextBytes(outIv);
+
+  // Update cipher option for client. The key is encrypted.
+  cipherOption.setParameters(wrap(inKey, 0, inKey.length), inIv,
--- End diff --

There's a slight irony here in that you're using 3DES to encrypt the AES 
keys...


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83517143
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesEncryption.java
 ---
@@ -0,0 +1,176 @@
+/*
+ * 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.channels.WritableByteChannel;
+
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.*;
+import io.netty.util.AbstractReferenceCounted;
+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;
+
+public class AesEncryption {
--- End diff --

I'm not a big fan of this class. It seems to be complicating the code 
instead of helping.

Instead, if these static nested classes were instead child classes of 
`AesCipher`, you could simplify the code a lot.

First, instead of adding the `negotiate` method to the SASL-related 
classes, it could live in the AesCipher code. e.g., following up on my comment 
in `SaslRpcHandler`:

```
if (!isAuthenticated) {
  // code around current L125
}

// Here goes code to finish the AES key negotiation.
AesCipher cipher = new AesCipher(...);
rpcCallback.reply(cipher.createConfigMessage(...));
cipher.addToChannel(...);
```

(Didn't really look at API details but hope you get the idea.)

Second, if things like `AesEncryptHandler` and `EncryptMessage` were nested 
non-static classes, you could remove the stream caches you have, making their 
life cycle more explicitly managed (and removing some weird code in the 
process).


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83514678
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
 ---
@@ -150,6 +161,62 @@ public synchronized void dispose() {
   }
 
   /**
+   * Negotiate with peer for extended options, such as using AES cipher.
+   * @param message is message receive from peer which may contains 
communication parameters.
+   * @param callback is rpc callback.
+   * @param conf contains transport configuration.
+   * @return Object which represent the result of negotiate.
+   */
+  public Object negotiate(ByteBuffer message, RpcResponseCallback 
callback, TransportConf conf)
--- End diff --

Same comments here re: naming and return type.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r83513326
  
--- Diff: 
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java
 ---
@@ -88,7 +90,16 @@ public void doBootstrap(TransportClient client, Channel 
channel) {
   throw new RuntimeException(
 new SaslException("Encryption requests by negotiated 
non-encrypted connection."));
 }
-SaslEncryption.addToChannel(channel, saslClient, 
conf.maxSaslEncryptedBlockSize());
+
+if(conf.saslEncryptionAesEnabled()) {
--- End diff --

nit: space after `if`


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-10-03 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r81568360
  
--- 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 {
--- End diff --

I have some concern need your help to confirm. 

1. According to AES cipher API definition, it need to know the size of the 
encrypted data. I'm not sure whether AES decrypt incoming bytes without 
well-defined frames. Does receiver will receive exactly same message/ByteBuf 
from sender without decoding?  

2. It seems the new netty channel will look most same as SaslEncryption 
except the chunking you mentioned. So I think it would be better if we can 
reuse SaslEncryption here since chunking is also a optimization as it 
described.  Could you please help to advice?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-29 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r81183030
  
--- 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 {
--- End diff --

Lazy encryption is necessary, because the outbound pipeline is 
multi-threaded, and multiple threads might trigger the AES handler at the same 
time. So instead of encrypting in the handler method, you need to do like the 
SASL code and do it in a `FileRegion` implementation.

For AES, chunking should not be necessary. The SASL code chunks because it 
needs to know the size of each encrypted frame, to feed it to the `unwrap` 
method, otherwise decryption doesn't work. (Also to curb memory usage and avoid 
really large byte arrays.) AES should be able to decrypt the incoming bytes as 
they arrive, without the need for well-defined frames.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-28 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r81060550
  
--- 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 {
--- End diff --

I have removed the integrity check code it looks a bit simple now.

Also I am seeking for your advice to setup netty channel handler for AES 
path and found SaslEncryption implement channel handler with lazy encryption 
and also it breaks incomming msg to chunks for transfer.  So I'm concerning 
about whether we need to do same mechanism with Sasl encryption for netty 
channel handler, or just a simple implement that encrypt and decrypt the 
incoming msg. Could you please help to give some advice?


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-28 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r81030532
  
--- 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 {
--- End diff --

I understand your proposal and OK with the proposal, it should be more 
simpler.  I will update the patch accordingly.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-28 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r80976506
  
--- 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 {
--- End diff --

In fact, you don't even need to touch `SaslEncryption` at all. After you 
negotiate the keys, you can just install an AES-specific netty handler directly 
into the pipeline, instead of calling `SaslEncryption.addToChannel`.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-28 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r80972579
  
--- 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 {
--- End diff --

The key negotiation part is fine. That doesn't need to change.

But I'm strongly against your argument that because `SaslEncryption` exists 
you shouldn't change it. That's an internal Spark class, you have all the right 
to change it. If you change it to work with ByteBuffers, the AES path 
automatically becomes more efficient, while there's no gain or loss for the old 
SASL-based path. Yes it's more code churn, but for a good cause.

Similarly, all your arguments about needing to calculate an HMAC and keep a 
sequence number are based on the internal JVM implementation of DIGEST-MD5 
encryption. That's irrelevant to what you're trying to achieve here. Meaning 
that code is not necessary.

Basically, being blunt, what I'm saying is that I'm opposed to your current 
approach and I will -1 your patch the way it is.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-27 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r80846949
  
--- 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 {
--- End diff --

Thanks for elaboration. I agree with you to implement AES encryption not 
SASL encryption. And here are reasons why AES encryption implemented like this:

First of all, to enable AES encryption it has to negotiate a pair of key 
and input vector through SASL, that's the connection between AES and SASL 
Encryption. 

Secondly, the patch is not intended to implement a SASL encryption but to 
implement the AES encryption by utilizing the existing encryption code provided 
by SASL encryption (the SaslEncryption class,  which wraps the 
SaslEncryptionBackend as netty channelhandlers). So the patch do implement 
netty channel handler while function names are wrap/unwrap against write.

Lastly, I think the code cannot be simpler if implement separated channel 
handlers, since AES encryption needs key negotiation, Integrity check(seq 
number and hmac) as I mentioned before.







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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-27 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r80742189
  
--- 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 {
--- End diff --

You're missing my point.

The goal is not to implement "SASL encryption". The goal is to implement 
*encryption*.

The AES encryption being added does *NOT* need to follow any of the JVM's 
SASL APIs, because that's not the goal.

Basically, when using AES, you're using SASL strictly for authentication.
- open connection
- authenticate using SASL
- set up a netty channel handler that does encryption using AES

*There is no SASL required in the last step.* The only reason it's involved 
in the last step now is because for DIGEST-MD5, the JDK mixes authentication 
and encryption. That is *not* the case with AES.

You're writing suboptimal code just because you're trying to support a 
non-feature. If you ignore SASL once authentication is done, you can write much 
simpler and better code for the AES path.


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-27 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r80634963
  
--- 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];
+  

[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-26 Thread cjjnjust
Github user cjjnjust commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r80615974
  
--- 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 {
--- End diff --

Seq number and HMAC are necessary parts of SASL encryption,  Seq number can 
make same data with different offsets different, and HMAC is used for integrity 
check, see com.sun.security.sasl.digest.DigestMD5Base class from JDK as example.
 
JDK's SASL framework doesn't provide interface to transfer AES cipher, it 
creates cipher interanlly. For example DigestMD5 client create cipher 
internally (DigestMD5Base.java:1219). However, it provides SaslEncryptBackend 
interface for user to customize the SASL client/server, that's why the patch 
implements wrap/unwrap. 

Another way to make patch clean and clear is to add AES cipher support in 
JDK's Digest-MD5 mechanism or change JDK SASL framework to provide a interface 
support registering customized cipher, but that would be slow and depends on 
JDK release.  


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



[GitHub] spark pull request #15172: [SPARK-13331] AES support for over-the-wire encry...

2016-09-26 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15172#discussion_r80524986
  
--- 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 {
--- End diff --

See previous comment. You do not need to implement JDK interfaces here, 
especially because they don't really add any needed functionality.

They're needed for the SASL encryption backend because that's how you do it 
with those APIs. It doesn't need the AES encryption code needs to implement 
those APIs too.


---
If your 

  1   2   >