smaheshwar-pltr commented on code in PR #13066:
URL: https://github.com/apache/iceberg/pull/13066#discussion_r2124949952


##########
spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.sql;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.Ciphers;
+import org.apache.iceberg.encryption.UnitestKMS;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.spark.CatalogTestBase;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+
+public class TestTableEncryption extends CatalogTestBase {
+  private static Map<String, String> 
appendCatalogEncryptionProperties(Map<String, String> props) {
+    Map<String, String> newProps = Maps.newHashMap();
+    newProps.putAll(props);
+    newProps.put(CatalogProperties.ENCRYPTION_KMS_IMPL, 
UnitestKMS.class.getCanonicalName());
+    return newProps;
+  }
+
+  @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  protected static Object[][] parameters() {
+    return new Object[][] {
+      {
+        SparkCatalogConfig.HIVE.catalogName(),
+        SparkCatalogConfig.HIVE.implementation(),
+        appendCatalogEncryptionProperties(SparkCatalogConfig.HIVE.properties())
+      }
+    };
+  }
+
+  @BeforeEach
+  public void createTables() {
+    sql(
+        "CREATE TABLE %s (id bigint, data string, float float) USING iceberg "
+            + "TBLPROPERTIES ( "
+            + "'encryption.key-id'='%s')",
+        tableName, UnitestKMS.MASTER_KEY_NAME1);
+
+    sql("INSERT INTO %s VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', 
float('NaN'))", tableName);
+  }
+
+  @AfterEach
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @TestTemplate
+  public void testSelect() {
+    List<Object[]> expected =
+        ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", 
Float.NaN));
+
+    assertEquals("Should return all expected rows", expected, sql("SELECT * 
FROM %s", tableName));
+  }
+
+  @TestTemplate
+  public void testInsertAndDelete() {
+    sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0), (6, 'f', 
float('NaN'))", tableName);
+
+    List<Object[]> expected =
+        ImmutableList.of(
+            row(1L, "a", 1.0F),
+            row(2L, "b", 2.0F),
+            row(3L, "c", Float.NaN),
+            row(4L, "d", 4.0F),
+            row(5L, "e", 5.0F),
+            row(6L, "f", Float.NaN));
+
+    assertEquals(
+        "Should return all expected rows",
+        expected,
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+
+    sql("DELETE FROM %s WHERE id < 4", tableName);
+
+    expected = ImmutableList.of(row(4L, "d", 4.0F), row(5L, "e", 5.0F), 
row(6L, "f", Float.NaN));
+
+    assertEquals(
+        "Should return all expected rows",
+        expected,
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @TestTemplate
+  public void testKeyDelete() {
+    assertThatThrownBy(
+            () -> sql("ALTER TABLE %s UNSET TBLPROPERTIES 
(`encryption.key-id`)", tableName))
+        .hasMessageContaining("Cannot remove key in encrypted table");
+  }
+
+  @TestTemplate
+  public void testDirectDataFileRead() {
+    List<Object[]> dataFileTable =
+        sql("SELECT file_path FROM %s.%s", tableName, 
MetadataTableType.ALL_DATA_FILES);
+    List<String> dataFiles =
+        Streams.concat(dataFileTable.stream())
+            .map(row -> (String) row[0])
+            .collect(Collectors.toList());
+
+    if (dataFiles.isEmpty()) {
+      throw new RuntimeException("No data files found for table " + tableName);
+    }
+
+    Schema schema = new Schema(optional(0, "id", Types.IntegerType.get()));
+    for (String filePath : dataFiles) {
+      assertThatThrownBy(
+              () ->
+                  Parquet.read(localInput(filePath))
+                      .project(schema)
+                      .callInit()
+                      .build()
+                      .iterator()
+                      .next())
+          .isInstanceOf(ParquetCryptoRuntimeException.class)
+          .hasMessageContaining("Trying to read file with encrypted footer. No 
keys available");
+    }
+  }
+
+  @TestTemplate
+  public void testManifestEncryption() throws IOException {
+    List<Object[]> manifestFileTable =
+        sql("SELECT path FROM %s.%s", tableName, MetadataTableType.MANIFESTS);
+
+    List<String> manifestFiles =
+        Streams.concat(manifestFileTable.stream())
+            .map(row -> (String) row[0])
+            .collect(Collectors.toList());
+
+    if (manifestFiles.isEmpty()) {
+      throw new RuntimeException("No manifest files found for table " + 
tableName);
+    }
+
+    String metadataFolderPath = null;
+
+    // Check encryption of manifest files
+    for (String manifestFilePath : manifestFiles) {
+      checkMetadataFileEncryption(localInput(manifestFilePath));
+
+      if (metadataFolderPath == null) {
+        metadataFolderPath = new 
File(manifestFilePath).getParent().replaceFirst("file:", "");
+      }
+    }
+
+    if (metadataFolderPath == null) {
+      throw new RuntimeException("No metadata folder found for table " + 
tableName);
+    }
+
+    // Find manifest list and metadata files; check their encryption

Review Comment:
   @ggershinsky, curious what came of the metadata JSON encryption discussion. 
Do folks think that information contained within metadata JSONs are not 
sufficiently sensitive to be encrypted in storage?
   
   Also, I recall you mentioning that for REST catalogs, this discussion isn't 
significant because they can make either decision themselves. This makes sense 
to me - but I do see parts of the codebase (e.g. below) where clients seem to 
attempt to read directly from the metadata JSON in storage, which I guess 
wouldn't work if it's been silently encrypted by the REST catalog.
   
https://github.com/apache/iceberg/blob/931865ecaf40a827f9081dddb675bf1c95c05461/core/src/main/java/org/apache/iceberg/ManifestsTable.java#L84-L85



##########
spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.sql;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.Ciphers;
+import org.apache.iceberg.encryption.UnitestKMS;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.spark.CatalogTestBase;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+
+public class TestTableEncryption extends CatalogTestBase {
+  private static Map<String, String> 
appendCatalogEncryptionProperties(Map<String, String> props) {
+    Map<String, String> newProps = Maps.newHashMap();
+    newProps.putAll(props);
+    newProps.put(CatalogProperties.ENCRYPTION_KMS_IMPL, 
UnitestKMS.class.getCanonicalName());
+    return newProps;
+  }
+
+  @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  protected static Object[][] parameters() {
+    return new Object[][] {
+      {
+        SparkCatalogConfig.HIVE.catalogName(),
+        SparkCatalogConfig.HIVE.implementation(),
+        appendCatalogEncryptionProperties(SparkCatalogConfig.HIVE.properties())
+      }
+    };
+  }
+
+  @BeforeEach
+  public void createTables() {
+    sql(
+        "CREATE TABLE %s (id bigint, data string, float float) USING iceberg "
+            + "TBLPROPERTIES ( "
+            + "'encryption.key-id'='%s')",
+        tableName, UnitestKMS.MASTER_KEY_NAME1);
+
+    sql("INSERT INTO %s VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', 
float('NaN'))", tableName);
+  }
+
+  @AfterEach
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @TestTemplate
+  public void testSelect() {
+    List<Object[]> expected =
+        ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", 
Float.NaN));
+
+    assertEquals("Should return all expected rows", expected, sql("SELECT * 
FROM %s", tableName));

Review Comment:
   ```suggestion
       assertEquals(
           "Should return all expected rows",
           expected,
           sql("SELECT * FROM %s ORDER BY id", tableName));
   ```



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -97,17 +119,55 @@ protected String tableName() {
 
   @Override
   public FileIO io() {
-    return fileIO;
+    if (encryptionKeyId == null) {
+      return fileIO;
+    }
+
+    if (encryptingFileIO == null) {
+      encryptingFileIO = EncryptingFileIO.combine(fileIO, encryption());
+    }
+
+    return encryptingFileIO;
+  }
+
+  @Override
+  public EncryptionManager encryption() {
+    if (encryptionManager != null) {
+      return encryptionManager;
+    }
+
+    if (encryptionKeyId != null) {
+      if (keyManagementClient == null) {
+        throw new RuntimeException(
+            "Cant create encryption manager, because key management client is 
not set");
+      }
+
+      Map<String, String> tableProperties = Maps.newHashMap();
+      tableProperties.put(TableProperties.ENCRYPTION_TABLE_KEY, 
encryptionKeyId);
+      tableProperties.put(
+          TableProperties.ENCRYPTION_DEK_LENGTH, 
String.valueOf(encryptionDekLength));
+      encryptionManager =
+          EncryptionUtil.createEncryptionManager(
+              encryptedKeysFromMetadata, tableProperties, keyManagementClient);

Review Comment:
   I guess we need something like 
https://github.com/apache/iceberg/pull/13224/files#diff-f8f95b203a96a9ba193bcafdff1813f9f5be1da4d596bedc8db5178dd33d4900R44-R104
 for this but can update once #7770 merges 👍



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -124,13 +184,37 @@ protected void doRefresh() {
       throw new RuntimeException("Interrupted during refresh", e);
     }
 
+    if (encryptionKeyIdFromHMS != null) {
+      encryptionKeyId = encryptionKeyIdFromHMS; // todo gg
+      encryptionDekLength =
+          (dekLengthFromHMS != null)
+              ? Integer.parseInt(dekLengthFromHMS)
+              : TableProperties.ENCRYPTION_DEK_LENGTH_DEFAULT; // todo gg
+    }
+
     refreshFromMetadataLocation(metadataLocation, metadataRefreshMaxRetries);
+
+    if (encryptionKeyIdFromHMS != null) {
+      checkEncryptionProperties(encryptionKeyIdFromHMS, dekLengthFromHMS);
+      encryptedKeysFromMetadata = current().encryptionKeys();

Review Comment:
   Just assigning this field here isn't sufficient right? I think we need to 
add the loaded keys to the current encryption manager's state - perhaps via a 
new API, or maybe there are ideas like 
https://github.com/apache/iceberg/pull/13225/files#diff-8b93b61734b5ccae516e7dbeb26dd1b2329707b87935f32b545638594cb8e2e9R269-R271
 to force re-initializaton in `io` and `encryption` methods.



-- 
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-unsubscr...@iceberg.apache.org

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


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

Reply via email to