You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by do...@apache.org on 2022/07/27 04:45:00 UTC
[orc] branch main updated: ORC-1229: Move `KeyProviderImpl` to `org.apache.orc.impl`
This is an automated email from the ASF dual-hosted git repository.
dongjoon pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/orc.git
The following commit(s) were added to refs/heads/main by this push:
new b3e47bab0 ORC-1229: Move `KeyProviderImpl` to `org.apache.orc.impl`
b3e47bab0 is described below
commit b3e47bab0ccf924d2271402d741426f3ac7d8b8f
Author: William Hyun <wi...@apache.org>
AuthorDate: Tue Jul 26 21:44:49 2022 -0700
ORC-1229: Move `KeyProviderImpl` to `org.apache.orc.impl`
### What changes were proposed in this pull request?
This PR aims to move `KeyProviderImpl` to `org.apache.orc.impl`
### Why are the changes needed?
To clean up old Hadoop dependencies.
### How was this patch tested?
Pass the CIs.
Closes #1198 from williamhyun/keyprovider.
Authored-by: William Hyun <wi...@apache.org>
Signed-off-by: Dongjoon Hyun <do...@apache.org>
---
.../org/apache/orc/impl/HadoopShimsPre2_7.java | 149 ------------------
.../java/org/apache/orc/impl/KeyProviderImpl.java | 175 +++++++++++++++++++++
2 files changed, 175 insertions(+), 149 deletions(-)
diff --git a/java/shims/src/java/org/apache/orc/impl/HadoopShimsPre2_7.java b/java/shims/src/java/org/apache/orc/impl/HadoopShimsPre2_7.java
index 410be1eb8..a7542d30b 100644
--- a/java/shims/src/java/org/apache/orc/impl/HadoopShimsPre2_7.java
+++ b/java/shims/src/java/org/apache/orc/impl/HadoopShimsPre2_7.java
@@ -20,18 +20,14 @@ package org.apache.orc.impl;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
import org.apache.hadoop.crypto.key.KeyProviderFactory;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.orc.EncryptionAlgorithm;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import javax.crypto.spec.SecretKeySpec;
import java.io.IOException;
import java.io.OutputStream;
-import java.security.GeneralSecurityException;
-import java.security.Key;
import java.util.List;
import java.util.Random;
@@ -70,151 +66,6 @@ public class HadoopShimsPre2_7 implements HadoopShims {
return key.getKeyName() + "@" + key.getVersion();
}
- /**
- * Shim implementation for ORC's KeyProvider API that uses Hadoop's
- * KeyProvider API and implementations. Most users use a Hadoop or Ranger
- * KMS and thus should use this default implementation.
- *
- * The main two methods of ORC's KeyProvider are createLocalKey and
- * decryptLocalKey. These are very similar to Hadoop's
- * <pre>
- * EncryptedKeyVersion generateEncryptedKey(String keyVersionName);
- * KeyVersion decryptEncryptedKey(EncryptedKeyVersion encrypted)
- * </pre>
- * but there are some important differences.
- * <ul>
- * <li>Hadoop's generateEncryptedKey doesn't return the decrypted key, so it
- * would require two round trips (generateEncryptedKey and then
- * decryptEncryptedKey)to the KMS.</li>
- * <li>Hadoop's methods require storing both the IV and the encrypted key, so
- * for AES256, it is 48 random bytes.</li>
- * </ul>
- *
- * However, since the encryption in the KMS is using AES/CTR we know that the
- * flow is:
- *
- * <pre>
- * tmpKey = aes(masterKey, iv);
- * cypher = xor(tmpKey, plain);
- * </pre>
- *
- * which means that encryption and decryption are symmetric. Therefore, if we
- * use the KMS' decryptEncryptedKey, and feed in a random iv and the right
- * number of 0's as the encrypted key, we get the right length of a tmpKey.
- * Since it is symmetric, we can use it for both encryption and decryption
- * and we only need to store the random iv. Since the iv is 16 bytes, it is
- * only a third the size of the other solution, and only requires one trip to
- * the KMS.
- *
- * So the flow looks like:
- * <pre>
- * encryptedKey = securely random 16 or 32 bytes
- * iv = first 16 byte of encryptedKey
- * --- on KMS ---
- * tmpKey0 = aes(masterKey, iv)
- * tmpKey1 = aes(masterKey, iv+1)
- * decryptedKey0 = xor(tmpKey0, encryptedKey0)
- * decryptedKey1 = xor(tmpKey1, encryptedKey1)
- * </pre>
- *
- * In the long term, we should probably fix Hadoop's generateEncryptedKey
- * to either take the random key or pass it back.
- */
- static class KeyProviderImpl implements KeyProvider {
- private final org.apache.hadoop.crypto.key.KeyProvider provider;
- private final Random random;
-
- KeyProviderImpl(org.apache.hadoop.crypto.key.KeyProvider provider,
- Random random) {
- this.provider = provider;
- this.random = random;
- }
-
- @Override
- public List<String> getKeyNames() throws IOException {
- return provider.getKeys();
- }
-
- @Override
- public KeyMetadata getCurrentKeyVersion(String keyName) throws IOException {
- org.apache.hadoop.crypto.key.KeyProvider.Metadata meta =
- provider.getMetadata(keyName);
- return new KeyMetadata(keyName, meta.getVersions() - 1,
- findAlgorithm(meta));
- }
-
- /**
- * The Ranger/Hadoop KMS mangles the IV by bit flipping it in a misguided
- * attempt to improve security. By bit flipping it here, we undo the
- * silliness so that we get
- * @param input the input array to copy from
- * @param output the output array to write to
- */
- private static void unmangleIv(byte[] input, byte[] output) {
- for(int i=0; i < output.length && i < input.length; ++i) {
- output[i] = (byte) (0xff ^ input[i]);
- }
- }
-
- @Override
- public LocalKey createLocalKey(KeyMetadata key) throws IOException {
- EncryptionAlgorithm algorithm = key.getAlgorithm();
- byte[] encryptedKey = new byte[algorithm.keyLength()];
- random.nextBytes(encryptedKey);
- byte[] iv = new byte[algorithm.getIvLength()];
- unmangleIv(encryptedKey, iv);
- KeyProviderCryptoExtension.EncryptedKeyVersion param =
- KeyProviderCryptoExtension.EncryptedKeyVersion.createForDecryption(
- key.getKeyName(), buildKeyVersionName(key), iv, encryptedKey);
- try {
- KeyProviderCryptoExtension.KeyVersion decryptedKey;
- if (provider instanceof KeyProviderCryptoExtension) {
- decryptedKey = ((KeyProviderCryptoExtension) provider).decryptEncryptedKey(param);
- } else if (provider instanceof CryptoExtension) {
- decryptedKey = ((CryptoExtension) provider).decryptEncryptedKey(param);
- } else {
- throw new UnsupportedOperationException(
- provider.getClass().getCanonicalName() + " is not supported.");
- }
- return new LocalKey(algorithm, decryptedKey.getMaterial(),
- encryptedKey);
- } catch (GeneralSecurityException e) {
- throw new IOException("Can't create local encryption key for " + key, e);
- }
- }
-
- @Override
- public Key decryptLocalKey(KeyMetadata key,
- byte[] encryptedKey) throws IOException {
- EncryptionAlgorithm algorithm = key.getAlgorithm();
- byte[] iv = new byte[algorithm.getIvLength()];
- unmangleIv(encryptedKey, iv);
- KeyProviderCryptoExtension.EncryptedKeyVersion param =
- KeyProviderCryptoExtension.EncryptedKeyVersion.createForDecryption(
- key.getKeyName(), buildKeyVersionName(key), iv, encryptedKey);
- try {
- KeyProviderCryptoExtension.KeyVersion decryptedKey;
- if (provider instanceof KeyProviderCryptoExtension) {
- decryptedKey = ((KeyProviderCryptoExtension) provider).decryptEncryptedKey(param);
- } else if (provider instanceof CryptoExtension) {
- decryptedKey = ((CryptoExtension) provider).decryptEncryptedKey(param);
- } else {
- throw new UnsupportedOperationException(
- provider.getClass().getCanonicalName() + " is not supported.");
- }
- return new SecretKeySpec(decryptedKey.getMaterial(),
- algorithm.getAlgorithm());
- } catch (GeneralSecurityException e) {
- return null;
- }
- }
-
- @Override
- public HadoopShims.KeyProviderKind getKind() {
- return HadoopShims.KeyProviderKind.HADOOP;
- }
- }
-
static KeyProvider createKeyProvider(Configuration conf,
Random random) throws IOException {
List<org.apache.hadoop.crypto.key.KeyProvider> result =
diff --git a/java/shims/src/java/org/apache/orc/impl/KeyProviderImpl.java b/java/shims/src/java/org/apache/orc/impl/KeyProviderImpl.java
new file mode 100644
index 000000000..c1fda7972
--- /dev/null
+++ b/java/shims/src/java/org/apache/orc/impl/KeyProviderImpl.java
@@ -0,0 +1,175 @@
+/*
+ * 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.orc.impl;
+
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
+import org.apache.orc.EncryptionAlgorithm;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.security.Key;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Shim implementation for ORC's KeyProvider API that uses Hadoop's
+ * KeyProvider API and implementations. Most users use a Hadoop or Ranger
+ * KMS and thus should use this default implementation.
+ * <p>
+ * The main two methods of ORC's KeyProvider are createLocalKey and
+ * decryptLocalKey. These are very similar to Hadoop's
+ * <pre>
+ * EncryptedKeyVersion generateEncryptedKey(String keyVersionName);
+ * KeyVersion decryptEncryptedKey(EncryptedKeyVersion encrypted)
+ * </pre>
+ * but there are some important differences.
+ * <ul>
+ * <li>Hadoop's generateEncryptedKey doesn't return the decrypted key, so it
+ * would require two round trips (generateEncryptedKey and then
+ * decryptEncryptedKey)to the KMS.</li>
+ * <li>Hadoop's methods require storing both the IV and the encrypted key, so
+ * for AES256, it is 48 random bytes.</li>
+ * </ul>
+ * <p>
+ * However, since the encryption in the KMS is using AES/CTR we know that the
+ * flow is:
+ *
+ * <pre>
+ * tmpKey = aes(masterKey, iv);
+ * cypher = xor(tmpKey, plain);
+ * </pre>
+ * <p>
+ * which means that encryption and decryption are symmetric. Therefore, if we
+ * use the KMS' decryptEncryptedKey, and feed in a random iv and the right
+ * number of 0's as the encrypted key, we get the right length of a tmpKey.
+ * Since it is symmetric, we can use it for both encryption and decryption
+ * and we only need to store the random iv. Since the iv is 16 bytes, it is
+ * only a third the size of the other solution, and only requires one trip to
+ * the KMS.
+ * <p>
+ * So the flow looks like:
+ * <pre>
+ * encryptedKey = securely random 16 or 32 bytes
+ * iv = first 16 byte of encryptedKey
+ * --- on KMS ---
+ * tmpKey0 = aes(masterKey, iv)
+ * tmpKey1 = aes(masterKey, iv+1)
+ * decryptedKey0 = xor(tmpKey0, encryptedKey0)
+ * decryptedKey1 = xor(tmpKey1, encryptedKey1)
+ * </pre>
+ * <p>
+ * In the long term, we should probably fix Hadoop's generateEncryptedKey
+ * to either take the random key or pass it back.
+ */
+class KeyProviderImpl implements KeyProvider {
+ private final org.apache.hadoop.crypto.key.KeyProvider provider;
+ private final Random random;
+
+ KeyProviderImpl(org.apache.hadoop.crypto.key.KeyProvider provider,
+ Random random) {
+ this.provider = provider;
+ this.random = random;
+ }
+
+ @Override
+ public List<String> getKeyNames() throws IOException {
+ return provider.getKeys();
+ }
+
+ @Override
+ public HadoopShims.KeyMetadata getCurrentKeyVersion(String keyName) throws IOException {
+ org.apache.hadoop.crypto.key.KeyProvider.Metadata meta =
+ provider.getMetadata(keyName);
+ return new HadoopShims.KeyMetadata(keyName, meta.getVersions() - 1,
+ HadoopShimsPre2_7.findAlgorithm(meta));
+ }
+
+ /**
+ * The Ranger/Hadoop KMS mangles the IV by bit flipping it in a misguided
+ * attempt to improve security. By bit flipping it here, we undo the
+ * silliness so that we get
+ *
+ * @param input the input array to copy from
+ * @param output the output array to write to
+ */
+ private static void unmangleIv(byte[] input, byte[] output) {
+ for (int i = 0; i < output.length && i < input.length; ++i) {
+ output[i] = (byte) (0xff ^ input[i]);
+ }
+ }
+
+ @Override
+ public LocalKey createLocalKey(HadoopShims.KeyMetadata key) throws IOException {
+ EncryptionAlgorithm algorithm = key.getAlgorithm();
+ byte[] encryptedKey = new byte[algorithm.keyLength()];
+ random.nextBytes(encryptedKey);
+ byte[] iv = new byte[algorithm.getIvLength()];
+ unmangleIv(encryptedKey, iv);
+ EncryptedKeyVersion param = EncryptedKeyVersion.createForDecryption(
+ key.getKeyName(), HadoopShimsPre2_7.buildKeyVersionName(key), iv, encryptedKey);
+ try {
+ KeyProviderCryptoExtension.KeyVersion decryptedKey;
+ if (provider instanceof KeyProviderCryptoExtension) {
+ decryptedKey = ((KeyProviderCryptoExtension) provider).decryptEncryptedKey(param);
+ } else if (provider instanceof CryptoExtension) {
+ decryptedKey = ((CryptoExtension) provider).decryptEncryptedKey(param);
+ } else {
+ throw new UnsupportedOperationException(
+ provider.getClass().getCanonicalName() + " is not supported.");
+ }
+ return new LocalKey(algorithm, decryptedKey.getMaterial(),
+ encryptedKey);
+ } catch (GeneralSecurityException e) {
+ throw new IOException("Can't create local encryption key for " + key, e);
+ }
+ }
+
+ @Override
+ public Key decryptLocalKey(HadoopShims.KeyMetadata key,
+ byte[] encryptedKey) throws IOException {
+ EncryptionAlgorithm algorithm = key.getAlgorithm();
+ byte[] iv = new byte[algorithm.getIvLength()];
+ unmangleIv(encryptedKey, iv);
+ EncryptedKeyVersion param = EncryptedKeyVersion.createForDecryption(
+ key.getKeyName(), HadoopShimsPre2_7.buildKeyVersionName(key), iv, encryptedKey);
+ try {
+ KeyProviderCryptoExtension.KeyVersion decryptedKey;
+ if (provider instanceof KeyProviderCryptoExtension) {
+ decryptedKey = ((KeyProviderCryptoExtension) provider).decryptEncryptedKey(param);
+ } else if (provider instanceof CryptoExtension) {
+ decryptedKey = ((CryptoExtension) provider).decryptEncryptedKey(param);
+ } else {
+ throw new UnsupportedOperationException(
+ provider.getClass().getCanonicalName() + " is not supported.");
+ }
+ return new SecretKeySpec(decryptedKey.getMaterial(),
+ algorithm.getAlgorithm());
+ } catch (GeneralSecurityException e) {
+ return null;
+ }
+ }
+
+ @Override
+ public HadoopShims.KeyProviderKind getKind() {
+ return HadoopShims.KeyProviderKind.HADOOP;
+ }
+}