You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "MaxGekk (via GitHub)" <gi...@apache.org> on 2023/05/02 08:33:52 UTC

[GitHub] [spark] MaxGekk commented on a diff in pull request #40969: [SPARK-43286][SQL] Updates aes_encrypt CBC mode to generate random IVs

MaxGekk commented on code in PR #40969:
URL: https://github.com/apache/spark/pull/40969#discussion_r1182229918


##########
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java:
##########
@@ -26,27 +26,59 @@
 import javax.crypto.spec.SecretKeySpec;
 import java.nio.ByteBuffer;
 import java.security.GeneralSecurityException;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
 import java.security.SecureRandom;
-import java.util.Arrays;
+import java.security.spec.AlgorithmParameterSpec;
 
-import static java.nio.charset.StandardCharsets.US_ASCII;
 
 /**
  * An utility class for constructing expressions.
  */
 public class ExpressionImplUtils {
-  private static final SecureRandom secureRandom = new SecureRandom();
+  private static final ThreadLocal<SecureRandom> threadLocalSecureRandom =
+    new ThreadLocal<SecureRandom>() {
+      @Override
+      public SecureRandom initialValue() {
+        return new SecureRandom();
+      }
+    };
+
   private static final int GCM_IV_LEN = 12;
   private static final int GCM_TAG_LEN = 128;
-
   private static final int CBC_IV_LEN = 16;
-  private static final int CBC_SALT_LEN = 8;
-  /** OpenSSL's magic initial bytes. */
-  private static final String SALTED_STR = "Salted__";
-  private static final byte[] SALTED_MAGIC = SALTED_STR.getBytes(US_ASCII);
 
+  enum CipherMode {
+    ECB("ECB", 0, 0, "AES/ECB/PKCS5Padding", false),
+    CBC("CBC", CBC_IV_LEN, 0, "AES/CBC/PKCS5Padding", true),
+    GCM("GCM", GCM_IV_LEN, GCM_TAG_LEN, "AES/GCM/NoPadding", true);
+
+    private final String name;

Review Comment:
   Is the name used somewhere? If not, let's remove it.



##########
sql/core/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtilsSuite.scala:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.sql.catalyst.expressions
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.unsafe.types.UTF8String
+
+

Review Comment:
   nit: remove the empty line.



##########
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java:
##########
@@ -85,85 +117,68 @@ public static byte[] aesDecrypt(byte[] input, byte[] key, UTF8String mode, UTF8S
     return aesInternal(input, key, mode.toString(), padding.toString(), Cipher.DECRYPT_MODE);
   }
 
+  private static SecretKeySpec getSecretKeySpec(byte[] key) {
+    switch (key.length) {
+      case 16: case 24: case 32:
+        return new SecretKeySpec(key, 0, key.length, "AES");
+      default:
+        throw QueryExecutionErrors.invalidAesKeyLengthError(key.length);
+    }
+  }
+
+  private static byte[] generateIv(CipherMode mode) {
+    byte[] iv = new byte[mode.ivLength];
+    threadLocalSecureRandom.get().nextBytes(iv);
+    return iv;
+  }
+
+  private static AlgorithmParameterSpec getParamSpec(CipherMode mode, byte[] input, int offset) {
+    switch (mode) {
+      case CBC:
+        return new IvParameterSpec(input, offset, mode.ivLength);
+      case GCM:
+        return new GCMParameterSpec(mode.tagLength, input, offset, mode.ivLength);
+      default:
+        return null;
+    }
+  }
+
   private static byte[] aesInternal(
       byte[] input,
       byte[] key,
       String mode,
       String padding,
       int opmode) {
-    SecretKeySpec secretKey;
-
-    switch (key.length) {
-      case 16:
-      case 24:
-      case 32:
-        secretKey = new SecretKeySpec(key, 0, key.length, "AES");
-        break;
-      default:
-        throw QueryExecutionErrors.invalidAesKeyLengthError(key.length);
-      }
-
     try {
-      if (mode.equalsIgnoreCase("ECB") &&
-          (padding.equalsIgnoreCase("PKCS") || padding.equalsIgnoreCase("DEFAULT"))) {
-        Cipher cipher = Cipher.getInstance("AES/ECB/PKCS5Padding");
-        cipher.init(opmode, secretKey);
-        return cipher.doFinal(input, 0, input.length);
-      } else if (mode.equalsIgnoreCase("GCM") &&
-          (padding.equalsIgnoreCase("NONE") || padding.equalsIgnoreCase("DEFAULT"))) {
-        Cipher cipher = Cipher.getInstance("AES/GCM/NoPadding");
-        if (opmode == Cipher.ENCRYPT_MODE) {
-          byte[] iv = new byte[GCM_IV_LEN];
-          secureRandom.nextBytes(iv);
-          GCMParameterSpec parameterSpec = new GCMParameterSpec(GCM_TAG_LEN, iv);
-          cipher.init(Cipher.ENCRYPT_MODE, secretKey, parameterSpec);
-          byte[] encrypted = cipher.doFinal(input, 0, input.length);
+      SecretKeySpec secretKey = getSecretKeySpec(key);
+      CipherMode cipherMode = CipherMode.fromString(mode, padding);
+      Cipher cipher = Cipher.getInstance(cipherMode.transformation);
+      if (opmode == Cipher.ENCRYPT_MODE) {
+        // This IV will be 0-length for ECB
+        byte[] iv = generateIv(cipherMode);
+        if (cipherMode.usesSpec) {
+          AlgorithmParameterSpec algSpec = getParamSpec(cipherMode, iv, 0);
+          cipher.init(opmode, secretKey, algSpec);
+        } else {
+          cipher.init(opmode, secretKey);
+        }
+        byte[] encrypted = cipher.doFinal(input, 0, input.length);
+        if (iv.length > 0) {
           ByteBuffer byteBuffer = ByteBuffer.allocate(iv.length + encrypted.length);
           byteBuffer.put(iv);
           byteBuffer.put(encrypted);
           return byteBuffer.array();
         } else {
-          assert(opmode == Cipher.DECRYPT_MODE);
-          GCMParameterSpec parameterSpec = new GCMParameterSpec(GCM_TAG_LEN, input, 0, GCM_IV_LEN);
-          cipher.init(Cipher.DECRYPT_MODE, secretKey, parameterSpec);
-          return cipher.doFinal(input, GCM_IV_LEN, input.length - GCM_IV_LEN);
+          return encrypted;
         }
-      } else if (mode.equalsIgnoreCase("CBC") &&
-          (padding.equalsIgnoreCase("PKCS") || padding.equalsIgnoreCase("DEFAULT"))) {
-        Cipher cipher = Cipher.getInstance("AES/CBC/PKCS5Padding");
-        if (opmode == Cipher.ENCRYPT_MODE) {
-          byte[] salt = new byte[CBC_SALT_LEN];
-          secureRandom.nextBytes(salt);
-          final byte[] keyAndIv = getKeyAndIv(key, salt);
-          final byte[] keyValue = Arrays.copyOfRange(keyAndIv, 0, key.length);
-          final byte[] iv = Arrays.copyOfRange(keyAndIv, key.length, key.length + CBC_IV_LEN);
-          cipher.init(
-            Cipher.ENCRYPT_MODE,
-            new SecretKeySpec(keyValue, "AES"),
-            new IvParameterSpec(iv));
-          byte[] encrypted = cipher.doFinal(input, 0, input.length);
-          ByteBuffer byteBuffer = ByteBuffer.allocate(
-            SALTED_MAGIC.length + CBC_SALT_LEN + encrypted.length);
-          byteBuffer.put(SALTED_MAGIC);
-          byteBuffer.put(salt);
-          byteBuffer.put(encrypted);
-          return byteBuffer.array();
+      } else if (opmode == Cipher.DECRYPT_MODE) {

Review Comment:
   We pass `opmode` here in the file above, so, let's replace this `else if` by `else` and add an assert.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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