This is an automated email from the ASF dual-hosted git repository.

huaxingao pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/main by this push:
     new bec6793af8 Encryption clean up (#14579)
bec6793af8 is described below

commit bec6793af822cb791fa5975b41b8936631556c46
Author: ggershinsky <[email protected]>
AuthorDate: Wed Nov 19 19:38:04 2025 +0200

    Encryption clean up (#14579)
    
    * initial commit
    
    * more cleanup
    
    * update encr util
    
    * address review comments
---
 .../iceberg/encryption/EncryptingFileIO.java       | 20 +--------
 .../org/apache/iceberg/ManifestListWriter.java     | 10 ++---
 .../iceberg/deletes/EqualityDeleteWriter.java      |  4 +-
 .../iceberg/deletes/PositionDeleteWriter.java      |  4 +-
 .../apache/iceberg/encryption/AesGcmInputFile.java |  7 +++-
 .../iceberg/encryption/AesGcmOutputFile.java       |  2 +-
 .../apache/iceberg/encryption/EncryptionUtil.java  |  8 ++++
 .../encryption/NativeFileCryptoParameters.java     |  3 ++
 .../iceberg/encryption/NativelyEncryptedFile.java  |  3 ++
 .../encryption/StandardEncryptionManager.java      |  5 +++
 .../java/org/apache/iceberg/io/DataWriter.java     |  4 +-
 .../iceberg/avro/TestEncryptedAvroFileSplit.java   | 24 ++++++-----
 .../apache/iceberg/encryption/TestGcmStreams.java  | 48 ++++++++++++++--------
 .../apache/iceberg/hive/HiveTableOperations.java   |  9 ++--
 14 files changed, 91 insertions(+), 60 deletions(-)

diff --git 
a/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java 
b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java
index a4c708570d..2ab335a4d6 100644
--- a/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java
+++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java
@@ -165,7 +165,7 @@ public class EncryptingFileIO implements FileIO, 
Serializable {
   }
 
   private static EncryptionKeyMetadata toKeyMetadata(ByteBuffer buffer) {
-    return buffer != null ? new SimpleKeyMetadata(buffer) : 
EmptyKeyMetadata.get();
+    return buffer != null ? new SimpleKeyMetadata(buffer) : 
EncryptionKeyMetadata.empty();
   }
 
   private static class SimpleEncryptedInputFile implements EncryptedInputFile {
@@ -206,22 +206,4 @@ public class EncryptingFileIO implements FileIO, 
Serializable {
       return new SimpleKeyMetadata(metadataBuffer.duplicate());
     }
   }
-
-  private static class EmptyKeyMetadata implements EncryptionKeyMetadata {
-    private static final EmptyKeyMetadata INSTANCE = new EmptyKeyMetadata();
-
-    private static EmptyKeyMetadata get() {
-      return INSTANCE;
-    }
-
-    @Override
-    public ByteBuffer buffer() {
-      return null;
-    }
-
-    @Override
-    public EncryptionKeyMetadata copy() {
-      return this;
-    }
-  }
 }
diff --git a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java 
b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java
index b290ed4952..378bb9dffb 100644
--- a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java
+++ b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java
@@ -21,9 +21,9 @@ package org.apache.iceberg;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.Map;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
 import org.apache.iceberg.encryption.EncryptionManager;
 import org.apache.iceberg.encryption.NativeEncryptionKeyMetadata;
-import org.apache.iceberg.encryption.NativeEncryptionOutputFile;
 import org.apache.iceberg.encryption.StandardEncryptionManager;
 import org.apache.iceberg.exceptions.RuntimeIOException;
 import org.apache.iceberg.io.FileAppender;
@@ -42,9 +42,9 @@ abstract class ManifestListWriter implements 
FileAppender<ManifestFile> {
     if (encryptionManager instanceof StandardEncryptionManager) {
       // ability to encrypt the manifest list key is introduced for standard 
encryption.
       this.standardEncryptionManager = (StandardEncryptionManager) 
encryptionManager;
-      NativeEncryptionOutputFile encryptedFile = 
this.standardEncryptionManager.encrypt(file);
+      EncryptedOutputFile encryptedFile = 
this.standardEncryptionManager.encrypt(file);
       this.outputFile = encryptedFile.encryptingOutputFile();
-      this.manifestListKeyMetadata = encryptedFile.keyMetadata();
+      this.manifestListKeyMetadata = (NativeEncryptionKeyMetadata) 
encryptedFile.keyMetadata();
     } else {
       this.standardEncryptionManager = null;
       this.outputFile = file;
@@ -95,9 +95,9 @@ abstract class ManifestListWriter implements 
FileAppender<ManifestFile> {
 
   public ManifestListFile toManifestListFile() {
     if (manifestListKeyMetadata != null && 
manifestListKeyMetadata.encryptionKey() != null) {
-      manifestListKeyMetadata.copyWithLength(writer.length());
       String manifestListKeyID =
-          
standardEncryptionManager.addManifestListKeyMetadata(manifestListKeyMetadata);
+          standardEncryptionManager.addManifestListKeyMetadata(
+              manifestListKeyMetadata.copyWithLength(writer.length()));
       return new BaseManifestListFile(outputFile.location(), 
manifestListKeyID);
     } else {
       return new BaseManifestListFile(outputFile.location(), null);
diff --git 
a/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java 
b/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java
index 5decabd50d..06564ea3bf 100644
--- a/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java
+++ b/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java
@@ -27,6 +27,7 @@ import org.apache.iceberg.PartitionSpec;
 import org.apache.iceberg.SortOrder;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.encryption.EncryptionUtil;
 import org.apache.iceberg.io.DeleteWriteResult;
 import org.apache.iceberg.io.FileAppender;
 import org.apache.iceberg.io.FileWriter;
@@ -82,7 +83,8 @@ public class EqualityDeleteWriter<T> implements FileWriter<T, 
DeleteWriteResult>
               .withFormat(format)
               .withPath(location)
               .withPartition(partition)
-              .withEncryptionKeyMetadata(keyMetadata)
+              .withEncryptionKeyMetadata(
+                  EncryptionUtil.setFileLength(keyMetadata, appender.length()))
               .withFileSizeInBytes(appender.length())
               .withMetrics(appender.metrics())
               .withSplitOffsets(appender.splitOffsets())
diff --git 
a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java 
b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java
index c8193755f5..a8af5e9d0f 100644
--- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java
+++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java
@@ -32,6 +32,7 @@ import org.apache.iceberg.MetricsUtil;
 import org.apache.iceberg.PartitionSpec;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.encryption.EncryptionUtil;
 import org.apache.iceberg.io.DeleteWriteResult;
 import org.apache.iceberg.io.FileAppender;
 import org.apache.iceberg.io.FileWriter;
@@ -96,7 +97,8 @@ public class PositionDeleteWriter<T> implements 
FileWriter<PositionDelete<T>, De
               .withFormat(format)
               .withPath(location)
               .withPartition(partition)
-              .withEncryptionKeyMetadata(keyMetadata)
+              .withEncryptionKeyMetadata(
+                  EncryptionUtil.setFileLength(keyMetadata, appender.length()))
               .withSplitOffsets(appender.splitOffsets())
               .withFileSizeInBytes(appender.length())
               .withMetrics(metrics())
diff --git 
a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java 
b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java
index b03944859b..8678a6e336 100644
--- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java
+++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java
@@ -29,6 +29,11 @@ public class AesGcmInputFile implements InputFile {
   private Long encryptedLength;
   private Long plaintextLength;
 
+  /**
+   * @deprecated will be removed in 2.0.0 This API does not receive file 
length, and is therefore
+   *     not safe
+   */
+  @Deprecated
   public AesGcmInputFile(InputFile sourceFile, byte[] dataKey, byte[] 
fileAADPrefix) {
     this(sourceFile, dataKey, fileAADPrefix, null);
   }
@@ -43,7 +48,7 @@ public class AesGcmInputFile implements InputFile {
 
   private long encryptedLength() {
     if (encryptedLength == null) {
-      this.encryptedLength = sourceFile.getLength();
+      throw new IllegalArgumentException("File length is null");
     }
 
     return encryptedLength;
diff --git 
a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java 
b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java
index c3f5ee2d03..ec1433c1ed 100644
--- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java
+++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java
@@ -50,6 +50,6 @@ public class AesGcmOutputFile implements OutputFile {
 
   @Override
   public InputFile toInputFile() {
-    return new AesGcmInputFile(targetFile.toInputFile(), dataKey, 
fileAADPrefix);
+    throw new IllegalStateException("File length unknown, creating an 
AesGcmInputFile is not safe");
   }
 }
diff --git 
a/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java 
b/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java
index 74854b413e..fe228fae93 100644
--- a/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java
+++ b/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java
@@ -107,6 +107,14 @@ public class EncryptionUtil {
     return new BaseEncryptedOutputFile(encryptingOutputFile, 
EncryptionKeyMetadata.empty());
   }
 
+  public static ByteBuffer setFileLength(ByteBuffer keyMetadata, long 
fileLength) {
+    if (keyMetadata == null) {
+      return null;
+    }
+
+    return 
StandardKeyMetadata.parse(keyMetadata).copyWithLength(fileLength).buffer();
+  }
+
   /**
    * Decrypt the key metadata for a manifest list.
    *
diff --git 
a/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
 
b/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
index 2cade89dc7..143404edbc 100644
--- 
a/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
+++ 
b/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
@@ -25,7 +25,10 @@ import 
org.apache.iceberg.relocated.com.google.common.base.Preconditions;
  * Barebone encryption parameters, one object per content file. Carries the 
file encryption key
  * (later, will be extended with column keys and AAD prefix). Applicable only 
to formats with native
  * encryption support (Parquet and ORC).
+ *
+ * @deprecated will be removed in 2.0.0
  */
+@Deprecated
 public class NativeFileCryptoParameters {
   private final ByteBuffer fileKey;
   private final EncryptionAlgorithm fileEncryptionAlgorithm;
diff --git 
a/core/src/main/java/org/apache/iceberg/encryption/NativelyEncryptedFile.java 
b/core/src/main/java/org/apache/iceberg/encryption/NativelyEncryptedFile.java
index e55456591a..f436551d0c 100644
--- 
a/core/src/main/java/org/apache/iceberg/encryption/NativelyEncryptedFile.java
+++ 
b/core/src/main/java/org/apache/iceberg/encryption/NativelyEncryptedFile.java
@@ -22,7 +22,10 @@ package org.apache.iceberg.encryption;
  * This interface is applied to OutputFile and InputFile implementations, in 
order to enable
  * delivery of crypto parameters (such as encryption keys etc) from the 
Iceberg key management
  * module to the writers/readers of file formats that support encryption 
natively (Parquet and ORC).
+ *
+ * @deprecated will be removed in 2.0.0
  */
+@Deprecated
 public interface NativelyEncryptedFile {
   NativeFileCryptoParameters nativeCryptoParameters();
 
diff --git 
a/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java
 
b/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java
index 0523eb0d7f..1e6ac961d3 100644
--- 
a/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java
+++ 
b/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java
@@ -224,6 +224,11 @@ public class StandardEncryptionManager implements 
EncryptionManager {
           "Cannot find manifest list key metadata with id " + 
manifestListKeyID);
     }
 
+    if (encryptedKeyMetadata.encryptedById().equals(tableKeyId)) {
+      throw new IllegalArgumentException(
+          manifestListKeyID + " is a key encryption key, not manifest list key 
metadata");
+    }
+
     return 
transientState.unwrappedKeyCache.get(encryptedKeyMetadata.encryptedById());
   }
 
diff --git a/core/src/main/java/org/apache/iceberg/io/DataWriter.java 
b/core/src/main/java/org/apache/iceberg/io/DataWriter.java
index dfb372d3a8..3611d1f63b 100644
--- a/core/src/main/java/org/apache/iceberg/io/DataWriter.java
+++ b/core/src/main/java/org/apache/iceberg/io/DataWriter.java
@@ -27,6 +27,7 @@ import org.apache.iceberg.PartitionSpec;
 import org.apache.iceberg.SortOrder;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.encryption.EncryptionUtil;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 
 public class DataWriter<T> implements FileWriter<T, DataWriteResult> {
@@ -85,7 +86,8 @@ public class DataWriter<T> implements FileWriter<T, 
DataWriteResult> {
               .withFormat(format)
               .withPath(location)
               .withPartition(partition)
-              .withEncryptionKeyMetadata(keyMetadata)
+              .withEncryptionKeyMetadata(
+                  EncryptionUtil.setFileLength(keyMetadata, appender.length()))
               .withFileSizeInBytes(appender.length())
               .withMetrics(appender.metrics())
               .withSplitOffsets(appender.splitOffsets())
diff --git 
a/core/src/test/java/org/apache/iceberg/avro/TestEncryptedAvroFileSplit.java 
b/core/src/test/java/org/apache/iceberg/avro/TestEncryptedAvroFileSplit.java
index efb5de3e96..9aace99209 100644
--- a/core/src/test/java/org/apache/iceberg/avro/TestEncryptedAvroFileSplit.java
+++ b/core/src/test/java/org/apache/iceberg/avro/TestEncryptedAvroFileSplit.java
@@ -37,6 +37,7 @@ import org.apache.iceberg.encryption.EncryptedInputFile;
 import org.apache.iceberg.encryption.EncryptedOutputFile;
 import org.apache.iceberg.encryption.EncryptionManager;
 import org.apache.iceberg.encryption.EncryptionTestHelpers;
+import org.apache.iceberg.encryption.NativeEncryptionKeyMetadata;
 import org.apache.iceberg.io.FileAppender;
 import org.apache.iceberg.io.InputFile;
 import org.apache.iceberg.io.OutputFile;
@@ -72,24 +73,27 @@ public class TestEncryptedAvroFileSplit {
 
     EncryptedOutputFile eOut = ENCRYPTION_MANAGER.encrypt(out);
 
-    try (FileAppender<Object> writer =
+    FileAppender<Object> writer =
         Avro.write(eOut)
             .set(TableProperties.AVRO_COMPRESSION, "uncompressed")
             .createWriterFunc(DataWriter::create)
             .schema(SCHEMA)
             .overwrite()
-            .build()) {
+            .build();
 
-      Record record = GenericRecord.create(SCHEMA);
-      for (long i = 0; i < NUM_RECORDS; i += 1) {
-        Record next = record.copy(ImmutableMap.of("id", i, "data", 
UUID.randomUUID().toString()));
-        expected.add(next);
-        writer.add(next);
-      }
+    Record record = GenericRecord.create(SCHEMA);
+    for (long i = 0; i < NUM_RECORDS; i += 1) {
+      Record next = record.copy(ImmutableMap.of("id", i, "data", 
UUID.randomUUID().toString()));
+      expected.add(next);
+      writer.add(next);
     }
 
-    EncryptedInputFile encryptedIn =
-        EncryptedFiles.encryptedInput(out.toInputFile(), eOut.keyMetadata());
+    writer.close();
+
+    NativeEncryptionKeyMetadata kmWithLength =
+        ((NativeEncryptionKeyMetadata) 
eOut.keyMetadata()).copyWithLength(writer.length());
+
+    EncryptedInputFile encryptedIn = 
EncryptedFiles.encryptedInput(out.toInputFile(), kmWithLength);
 
     this.file = ENCRYPTION_MANAGER.decrypt(encryptedIn);
   }
diff --git 
a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java 
b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java
index 60ba15b200..066aed9070 100644
--- a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java
+++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java
@@ -56,7 +56,9 @@ public class TestGcmStreams {
     PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite();
     encryptedStream.close();
 
-    AesGcmInputFile decryptedFile = new 
AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
+    AesGcmInputFile decryptedFile =
+        new AesGcmInputFile(
+            Files.localInput(testFile), key, aadPrefix, 
encryptedStream.storedLength());
     assertThat(decryptedFile.getLength()).isEqualTo(0);
 
     try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
@@ -66,7 +68,9 @@ public class TestGcmStreams {
     // check that the AAD is still verified, even for an empty file
     byte[] badAAD = Arrays.copyOf(aadPrefix, aadPrefix.length);
     badAAD[1] -= 1; // modify the AAD slightly
-    AesGcmInputFile badAADFile = new 
AesGcmInputFile(Files.localInput(testFile), key, badAAD);
+    AesGcmInputFile badAADFile =
+        new AesGcmInputFile(
+            Files.localInput(testFile), key, badAAD, 
encryptedStream.storedLength());
     assertThat(badAADFile.getLength()).isEqualTo(0);
 
     try (SeekableInputStream decryptedStream = badAADFile.newStream()) {
@@ -91,12 +95,14 @@ public class TestGcmStreams {
 
     AesGcmOutputFile encryptedFile =
         new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix);
-    try (PositionOutputStream encryptedStream = 
encryptedFile.createOrOverwrite()) {
-      encryptedStream.write(content);
-    }
+    PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite();
+    encryptedStream.write(content);
+    encryptedStream.close();
 
     // verify the data can be read correctly with the right AAD
-    AesGcmInputFile decryptedFile = new 
AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
+    AesGcmInputFile decryptedFile =
+        new AesGcmInputFile(
+            Files.localInput(testFile), key, aadPrefix, 
encryptedStream.storedLength());
     assertThat(decryptedFile.getLength()).isEqualTo(content.length);
 
     try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
@@ -109,7 +115,9 @@ public class TestGcmStreams {
     // test with the wrong AAD
     byte[] badAAD = Arrays.copyOf(aadPrefix, aadPrefix.length);
     badAAD[1] -= 1; // modify the AAD slightly
-    AesGcmInputFile badAADFile = new 
AesGcmInputFile(Files.localInput(testFile), key, badAAD);
+    AesGcmInputFile badAADFile =
+        new AesGcmInputFile(
+            Files.localInput(testFile), key, badAAD, 
encryptedStream.storedLength());
     assertThat(badAADFile.getLength()).isEqualTo(content.length);
 
     try (SeekableInputStream decryptedStream = badAADFile.newStream()) {
@@ -151,12 +159,14 @@ public class TestGcmStreams {
 
     AesGcmOutputFile encryptedFile =
         new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix);
-    try (PositionOutputStream encryptedStream = 
encryptedFile.createOrOverwrite()) {
-      encryptedStream.write(content);
-    }
+    PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite();
+    encryptedStream.write(content);
+    encryptedStream.close();
 
     // verify the data can be read correctly with the right AAD
-    AesGcmInputFile decryptedFile = new 
AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
+    AesGcmInputFile decryptedFile =
+        new AesGcmInputFile(
+            Files.localInput(testFile), key, aadPrefix, 
encryptedStream.storedLength());
     assertThat(decryptedFile.getLength()).isEqualTo(content.length);
 
     try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
@@ -197,12 +207,14 @@ public class TestGcmStreams {
 
     AesGcmOutputFile encryptedFile =
         new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix);
-    try (PositionOutputStream encryptedStream = 
encryptedFile.createOrOverwrite()) {
-      encryptedStream.write(content);
-    }
+    PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite();
+    encryptedStream.write(content);
+    encryptedStream.close();
 
     // verify the data can be read correctly with the right AAD
-    AesGcmInputFile decryptedFile = new 
AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
+    AesGcmInputFile decryptedFile =
+        new AesGcmInputFile(
+            Files.localInput(testFile), key, aadPrefix, 
encryptedStream.storedLength());
     assertThat(decryptedFile.getLength()).isEqualTo(content.length);
 
     try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
@@ -279,7 +291,8 @@ public class TestGcmStreams {
             .isEqualTo(offset);
 
         AesGcmInputFile decryptedFile =
-            new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
+            new AesGcmInputFile(
+                Files.localInput(testFile), key, aadPrefix, 
encryptedStream.storedLength());
         SeekableInputStream decryptedStream = decryptedFile.newStream();
         assertThat(decryptedFile.getLength()).isEqualTo(testFileSize);
 
@@ -376,7 +389,8 @@ public class TestGcmStreams {
       assertThat(encryptedStream.getPos()).as("Final position in closed 
stream").isEqualTo(offset);
 
       AesGcmInputFile decryptedFile =
-          new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
+          new AesGcmInputFile(
+              Files.localInput(testFile), key, aadPrefix, 
encryptedStream.storedLength());
       SeekableInputStream decryptedStream = decryptedFile.newStream();
       assertThat(decryptedFile.getLength()).isEqualTo(testFileSize);
 
diff --git 
a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java 
b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
index 5e17243dc8..a4338ec07a 100644
--- 
a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
+++ 
b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
@@ -57,6 +57,7 @@ import org.apache.iceberg.io.LocationProvider;
 import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.PropertyUtil;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -558,11 +559,11 @@ public class HiveTableOperations extends 
BaseMetastoreTableOperations
     }
 
     if (tableKeyId != null && encryptionDekLength <= 0) {
-      String dekLength = 
tableProperties.get(TableProperties.ENCRYPTION_DEK_LENGTH);
       encryptionDekLength =
-          (dekLength == null)
-              ? TableProperties.ENCRYPTION_DEK_LENGTH_DEFAULT
-              : Integer.parseInt(dekLength);
+          PropertyUtil.propertyAsInt(
+              tableProperties,
+              TableProperties.ENCRYPTION_DEK_LENGTH,
+              TableProperties.ENCRYPTION_DEK_LENGTH_DEFAULT);
     }
   }
 

Reply via email to