You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/08/16 06:28:46 UTC

[GitHub] [iceberg] ggershinsky opened a new pull request, #5544: Encryption integration and test

ggershinsky opened a new pull request, #5544:
URL: https://github.com/apache/iceberg/pull/5544

   Co-Authored-By: Jian Tang <ji...@apple.com>


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a diff in pull request #5544: Encryption integration and test

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#discussion_r1071094167


##########
core/src/main/java/org/apache/iceberg/encryption/DefaultEncryptionManagerFactory.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.encryption;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.PropertyUtil;
+
+public class DefaultEncryptionManagerFactory implements EncryptionManagerFactory {
+  public static final String ICEBERG_ENCRYPTION_TABLE_KEY = "iceberg.encryption.table.key.id";
+
+  private static class KmsClientSupplier {
+    private final boolean ignoreTableProperties;
+
+    KmsClientSupplier(boolean ignoreTableProperties) {
+      this.ignoreTableProperties = ignoreTableProperties;
+    }
+
+    KeyManagementClient create(Map<String, String> encryptionProperties) {
+      // load kms impl from catalog properties, if not present fall back to table properties (if
+      // allowed).
+      String kmsImpl = encryptionProperties.get(CatalogProperties.ENCRYPTION_KMS_CLIENT_IMPL);
+
+      Preconditions.checkArgument(
+          null != kmsImpl,
+          "KMS Client implementation class is not set (via "
+              + CatalogProperties.ENCRYPTION_KMS_CLIENT_IMPL
+              + " catalog property nor "
+              + TableProperties.ENCRYPTION_KMS_CLIENT_IMPL
+              + " table property). "
+              + "IgnoreTableProperties is: "
+              + ignoreTableProperties);
+
+      return KmsUtil.loadKmsClient(kmsImpl, encryptionProperties);
+    }
+  }
+
+  private KmsClientSupplier kmsClientSupplier;
+  private KeyManagementClient client;
+  private Map<String, String> catalogPropertyMap;
+  private boolean ignoreTableProperties;
+
+  @Override
+  public void initialize(Map<String, String> catalogProperties) {
+    this.catalogPropertyMap = catalogProperties;
+    ignoreTableProperties =
+        PropertyUtil.propertyAsBoolean(
+            catalogProperties,
+            CatalogProperties.ENCRYPTION_IGNORE_TABLE_PROPS,
+            CatalogProperties.ENCRYPTION_IGNORE_TABLE_PROPS_DEFAULT);
+    kmsClientSupplier = new KmsClientSupplier(ignoreTableProperties);
+  }
+
+  @Override
+  public EncryptionManager create(TableMetadata tableMetadata) {
+    if (null == tableMetadata) {
+      return PlaintextEncryptionManager.INSTANCE;
+    }
+
+    Map<String, String> tableProperties = tableMetadata.properties();
+    String fileFormat =
+        PropertyUtil.propertyAsString(tableProperties, DEFAULT_FILE_FORMAT, "parquet");
+    if (!fileFormat.equals("parquet")) {

Review Comment:
   `FileFormat.fromString` should be used to match the general usage pattern.
   In other cases we accept upper case `PARQUET` too, so I think we should accept it here too.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [PR] Encryption integration and test [iceberg]

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#issuecomment-1768297698

   @ggershinsky: Do we want to add Flink tests alongside the Spark tests as well?


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a diff in pull request #5544: Encryption integration and test

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#discussion_r1071095748


##########
core/src/main/java/org/apache/iceberg/encryption/DefaultEncryptionManagerFactory.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.encryption;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.PropertyUtil;
+
+public class DefaultEncryptionManagerFactory implements EncryptionManagerFactory {
+  public static final String ICEBERG_ENCRYPTION_TABLE_KEY = "iceberg.encryption.table.key.id";
+
+  private static class KmsClientSupplier {
+    private final boolean ignoreTableProperties;
+
+    KmsClientSupplier(boolean ignoreTableProperties) {
+      this.ignoreTableProperties = ignoreTableProperties;
+    }
+
+    KeyManagementClient create(Map<String, String> encryptionProperties) {
+      // load kms impl from catalog properties, if not present fall back to table properties (if
+      // allowed).
+      String kmsImpl = encryptionProperties.get(CatalogProperties.ENCRYPTION_KMS_CLIENT_IMPL);
+
+      Preconditions.checkArgument(
+          null != kmsImpl,
+          "KMS Client implementation class is not set (via "
+              + CatalogProperties.ENCRYPTION_KMS_CLIENT_IMPL
+              + " catalog property nor "
+              + TableProperties.ENCRYPTION_KMS_CLIENT_IMPL
+              + " table property). "
+              + "IgnoreTableProperties is: "
+              + ignoreTableProperties);
+
+      return KmsUtil.loadKmsClient(kmsImpl, encryptionProperties);
+    }
+  }
+
+  private KmsClientSupplier kmsClientSupplier;
+  private KeyManagementClient client;
+  private Map<String, String> catalogPropertyMap;
+  private boolean ignoreTableProperties;
+
+  @Override
+  public void initialize(Map<String, String> catalogProperties) {
+    this.catalogPropertyMap = catalogProperties;
+    ignoreTableProperties =
+        PropertyUtil.propertyAsBoolean(
+            catalogProperties,
+            CatalogProperties.ENCRYPTION_IGNORE_TABLE_PROPS,
+            CatalogProperties.ENCRYPTION_IGNORE_TABLE_PROPS_DEFAULT);
+    kmsClientSupplier = new KmsClientSupplier(ignoreTableProperties);
+  }
+
+  @Override
+  public EncryptionManager create(TableMetadata tableMetadata) {
+    if (null == tableMetadata) {
+      return PlaintextEncryptionManager.INSTANCE;
+    }
+
+    Map<String, String> tableProperties = tableMetadata.properties();
+    String fileFormat =
+        PropertyUtil.propertyAsString(tableProperties, DEFAULT_FILE_FORMAT, "parquet");
+    if (!fileFormat.equals("parquet")) {
+      throw new UnsupportedOperationException(
+          "Iceberg encryption currently supports only parquet format for data files");
+    }
+
+    final Map<String, String> encryptionProperties = Maps.newHashMap();
+    if (!ignoreTableProperties) {
+      encryptionProperties.putAll(tableProperties);
+    }
+
+    // Important: put catalog properties after table properties. Former overrides the latter.
+    encryptionProperties.putAll(catalogPropertyMap);
+
+    String tableKeyId = System.getProperty(ICEBERG_ENCRYPTION_TABLE_KEY);

Review Comment:
   This seems odd to me.
   Do we rely on `System` properties?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] renshangtao commented on pull request #5544: Encryption integration and test

Posted by GitBox <gi...@apache.org>.
renshangtao commented on PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#issuecomment-1311808301

   @ggershinsky Hello, excuse me
   I tested the code, and the file wasn't encrypted,test case execution failed.
   All the testxxxWithoutKeys() are failed,Is my configuration incorrect


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [PR] Encryption integration and test [iceberg]

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#discussion_r1577309737


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -137,17 +160,69 @@ protected String tableName() {
 
   @Override
   public FileIO io() {
-    return fileIO;
+    if (encryptingFileIO != null) {
+      return encryptingFileIO;
+    }
+
+    encryptingFileIO = EncryptingFileIO.combine(fileIO, encryption());

Review Comment:
   maybe no need to do this if `encryption()` is `PlaintextEncryptionManager.instance()` . Then we won't have to fix the tests with Hadoop config cast.



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -137,17 +160,69 @@ protected String tableName() {
 
   @Override
   public FileIO io() {
-    return fileIO;
+    if (encryptingFileIO != null) {
+      return encryptingFileIO;
+    }
+
+    encryptingFileIO = EncryptingFileIO.combine(fileIO, encryption());
+    return encryptingFileIO;
+  }
+
+  @Override
+  public EncryptionManager encryption() {
+    if (encryptionManager != null) {
+      return encryptionManager;
+    }
+
+    if (keyManagementClient == null) {

Review Comment:
   what if `tableKeyID` is not null? check and throw an exception



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #5544: Encryption integration and test

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#discussion_r1072119326


##########
core/src/main/java/org/apache/iceberg/encryption/DefaultEncryptionManagerFactory.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.encryption;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.PropertyUtil;
+
+public class DefaultEncryptionManagerFactory implements EncryptionManagerFactory {
+  public static final String ICEBERG_ENCRYPTION_TABLE_KEY = "iceberg.encryption.table.key.id";
+
+  private static class KmsClientSupplier {
+    private final boolean ignoreTableProperties;
+
+    KmsClientSupplier(boolean ignoreTableProperties) {
+      this.ignoreTableProperties = ignoreTableProperties;
+    }
+
+    KeyManagementClient create(Map<String, String> encryptionProperties) {
+      // load kms impl from catalog properties, if not present fall back to table properties (if
+      // allowed).
+      String kmsImpl = encryptionProperties.get(CatalogProperties.ENCRYPTION_KMS_CLIENT_IMPL);
+
+      Preconditions.checkArgument(
+          null != kmsImpl,
+          "KMS Client implementation class is not set (via "
+              + CatalogProperties.ENCRYPTION_KMS_CLIENT_IMPL
+              + " catalog property nor "
+              + TableProperties.ENCRYPTION_KMS_CLIENT_IMPL
+              + " table property). "
+              + "IgnoreTableProperties is: "
+              + ignoreTableProperties);
+
+      return KmsUtil.loadKmsClient(kmsImpl, encryptionProperties);
+    }
+  }
+
+  private KmsClientSupplier kmsClientSupplier;
+  private KeyManagementClient client;
+  private Map<String, String> catalogPropertyMap;
+  private boolean ignoreTableProperties;
+
+  @Override
+  public void initialize(Map<String, String> catalogProperties) {
+    this.catalogPropertyMap = catalogProperties;
+    ignoreTableProperties =
+        PropertyUtil.propertyAsBoolean(
+            catalogProperties,
+            CatalogProperties.ENCRYPTION_IGNORE_TABLE_PROPS,
+            CatalogProperties.ENCRYPTION_IGNORE_TABLE_PROPS_DEFAULT);
+    kmsClientSupplier = new KmsClientSupplier(ignoreTableProperties);
+  }
+
+  @Override
+  public EncryptionManager create(TableMetadata tableMetadata) {
+    if (null == tableMetadata) {
+      return PlaintextEncryptionManager.INSTANCE;
+    }
+
+    Map<String, String> tableProperties = tableMetadata.properties();
+    String fileFormat =
+        PropertyUtil.propertyAsString(tableProperties, DEFAULT_FILE_FORMAT, "parquet");
+    if (!fileFormat.equals("parquet")) {
+      throw new UnsupportedOperationException(
+          "Iceberg encryption currently supports only parquet format for data files");
+    }
+
+    final Map<String, String> encryptionProperties = Maps.newHashMap();
+    if (!ignoreTableProperties) {
+      encryptionProperties.putAll(tableProperties);
+    }
+
+    // Important: put catalog properties after table properties. Former overrides the latter.
+    encryptionProperties.putAll(catalogPropertyMap);
+
+    String tableKeyId = System.getProperty(ICEBERG_ENCRYPTION_TABLE_KEY);

Review Comment:
   This is basically a hack that allows to address a challenge of using different keys for different tables - in those cases where the table properties are not trusted. I'd say the purpose is somewhere between a "temporary solution" and a "discussion trigger" :). In the long run, we might have table properties safely kept in catalogs (eg the REST catalog), but we need a solution in a shorter term.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5544: Encryption integration and test

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#discussion_r1320863448


##########
core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java:
##########
@@ -116,11 +119,14 @@ public void initialize(String name, Map<String, String> properties) {
         fileIOImpl == null
             ? new HadoopFileIO(conf)
             : CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+    this.encryptionManagerFactory = new DefaultEncryptionManagerFactory();

Review Comment:
   Is the intent for this to be a `StandardEncryptionManagerFactory`?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5544: DON'T MERGE Encryption integration and test

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#discussion_r1119131376


##########
core/src/main/java/org/apache/iceberg/encryption/EncryptionManagerFactory.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.iceberg.encryption;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.TableMetadata;
+
+public interface EncryptionManagerFactory extends Closeable {
+
+  EncryptionManagerFactory NO_ENCRYPTION = tableMetadata -> PlaintextEncryptionManager.INSTANCE;
+
+  /**
+   * Initialize EncryptionManagerFactory from catalog properties.
+   *
+   * @param properties catalog properties
+   */
+  default void initialize(Map<String, String> properties) {}
+  /**

Review Comment:
   Missing whitespace.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5544: Encryption integration and test

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#discussion_r1320863582


##########
core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java:
##########
@@ -116,11 +119,14 @@ public void initialize(String name, Map<String, String> properties) {
         fileIOImpl == null
             ? new HadoopFileIO(conf)
             : CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+    this.encryptionManagerFactory = new DefaultEncryptionManagerFactory();
+    this.encryptionManagerFactory.initialize(properties);

Review Comment:
   This pattern is typically used for dynamically loaded implementations. Since this is part of the catalog, I think it makes sense to just pass the properties to the encryption manager constructor.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #5544: Encryption integration and test

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#discussion_r1072113069


##########
core/src/main/java/org/apache/iceberg/encryption/DefaultEncryptionManagerFactory.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.encryption;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.PropertyUtil;
+
+public class DefaultEncryptionManagerFactory implements EncryptionManagerFactory {
+  public static final String ICEBERG_ENCRYPTION_TABLE_KEY = "iceberg.encryption.table.key.id";
+
+  private static class KmsClientSupplier {
+    private final boolean ignoreTableProperties;
+
+    KmsClientSupplier(boolean ignoreTableProperties) {
+      this.ignoreTableProperties = ignoreTableProperties;
+    }
+
+    KeyManagementClient create(Map<String, String> encryptionProperties) {
+      // load kms impl from catalog properties, if not present fall back to table properties (if
+      // allowed).
+      String kmsImpl = encryptionProperties.get(CatalogProperties.ENCRYPTION_KMS_CLIENT_IMPL);
+
+      Preconditions.checkArgument(
+          null != kmsImpl,
+          "KMS Client implementation class is not set (via "
+              + CatalogProperties.ENCRYPTION_KMS_CLIENT_IMPL
+              + " catalog property nor "
+              + TableProperties.ENCRYPTION_KMS_CLIENT_IMPL
+              + " table property). "
+              + "IgnoreTableProperties is: "
+              + ignoreTableProperties);
+
+      return KmsUtil.loadKmsClient(kmsImpl, encryptionProperties);
+    }
+  }
+
+  private KmsClientSupplier kmsClientSupplier;
+  private KeyManagementClient client;
+  private Map<String, String> catalogPropertyMap;
+  private boolean ignoreTableProperties;
+
+  @Override
+  public void initialize(Map<String, String> catalogProperties) {
+    this.catalogPropertyMap = catalogProperties;
+    ignoreTableProperties =
+        PropertyUtil.propertyAsBoolean(
+            catalogProperties,
+            CatalogProperties.ENCRYPTION_IGNORE_TABLE_PROPS,
+            CatalogProperties.ENCRYPTION_IGNORE_TABLE_PROPS_DEFAULT);
+    kmsClientSupplier = new KmsClientSupplier(ignoreTableProperties);
+  }
+
+  @Override
+  public EncryptionManager create(TableMetadata tableMetadata) {
+    if (null == tableMetadata) {
+      return PlaintextEncryptionManager.INSTANCE;
+    }
+
+    Map<String, String> tableProperties = tableMetadata.properties();
+    String fileFormat =
+        PropertyUtil.propertyAsString(tableProperties, DEFAULT_FILE_FORMAT, "parquet");
+    if (!fileFormat.equals("parquet")) {

Review Comment:
   SGTM, thanks for the suggestion.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [PR] Encryption integration and test [iceberg]

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on PR #5544:
URL: https://github.com/apache/iceberg/pull/5544#issuecomment-1768384984

   The current PR sequence develops the encryption tools using Spark as a usecase. But once the toolset converges, we'll certainly need the Flink support and unitests.


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org