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 2021/05/15 02:18:31 UTC

[GitHub] [iceberg] jackye1995 opened a new pull request #2520: Core: add key_metadata in ManifestFile

jackye1995 opened a new pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520


   This PR adds key_metadata to the ManifestFile API, and updates all the read and write methods that directly references the constructor up to `ManifestFiles` to use `EncryptionManager`.
   
   To avoid touching too many files in different engines in a single PR, there will be subsequent PRs to update each engine and the metadata metadata tables to support encrypting manifest files.
   
   @yyanyy @rdblue @RussellSpitzer @ggershinsky @flyrain 
   
   


-- 
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.

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] jackye1995 closed pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 closed pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520


   


-- 
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.

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 change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r642695867



##########
File path: api/src/main/java/org/apache/iceberg/ManifestFile.java
##########
@@ -62,14 +62,16 @@
   Types.NestedField PARTITION_SUMMARIES = optional(507, "partitions",
       Types.ListType.ofRequired(508, PARTITION_SUMMARY_TYPE),
       "Summary for each partition");
-  // next ID to assign: 519
+  Types.NestedField KEY_METADATA = optional(519, "key_metadata", Types.BinaryType.get(),
+          "Encryption key metadata blob");

Review comment:
       Added this to the spec in #2654.




-- 
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.

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] flyrain commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r643569231



##########
File path: core/src/main/java/org/apache/iceberg/StaticTableOperations.java
##########
@@ -32,25 +34,36 @@
   private TableMetadata staticMetadata;
   private final String metadataFileLocation;
   private final FileIO io;
+  private final EncryptionManager encryption;
   private final LocationProvider locationProvider;
 
   /**
-   * Creates a StaticTableOperations tied to a specific static version of the TableMetadata
+   * @deprecated please use {@link #StaticTableOperations(String, FileIO, EncryptionManager)}

Review comment:
       Do we need to deprecate it? To provide a method with a plainText EncryptionManager as the default value looks harmless.




-- 
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.

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] jackye1995 commented on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-854435599


   @flyrain thanks for the review! I am marking this as a draft. I spent a few days to figure out the way to go forward with all the changes needed in `ManifestFiles` API and I think I have a plan. I will separate this PR to a few different ones, mainly:
   
   1. adding `keyMetadata` byte array to `ManifestFile` interface and implementation classes.
   2. add support in `EncryptionManager` to decrypt and encrypt by file type
   3. changes to use encryption manager to supply key metadata information down to manifest writer and reader. This will be separated to multiple PRs for core, and engine related changes.
   


-- 
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.

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 change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r624757534



##########
File path: api/src/main/java/org/apache/iceberg/ManifestFile.java
##########
@@ -179,6 +181,13 @@ default boolean hasDeletedFiles() {
    */
   List<PartitionFieldSummary> partitions();
 
+  /**
+   * Returns metadata about how this manifest file is encrypted, or null if the file is stored in plain text.
+   */
+  default ByteBuffer keyMetadata() {

Review comment:
       The intent here is that an encryption manager can decide what the key metadata holds. It could be an encrypted key or it could be a key reference. There are lots of possibilities and we did it this way to not constrain what the encryption manager can choose to do.




-- 
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.

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] RussellSpitzer commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r620795244



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -245,6 +251,11 @@ public Long deletedRowsCount() {
     return partitions == null ? null : Arrays.asList(partitions);
   }
 
+  @Override
+  public ByteBuffer keyMetadata() {
+    return keyMetadata != null ? ByteBuffer.wrap(keyMetadata) : null;

Review comment:
       I'd rather follow the pattern above 
   positive assertion ? positive result : otherwise
   ```
   keyMetadata == null ? null : ByteBuffer.wrap(keyMetadata)
   ```
   Just a personal thing but I always have a hard time with negated boolean checks




-- 
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.

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] flyrain commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r643564170



##########
File path: core/src/main/java/org/apache/iceberg/ManifestFiles.java
##########
@@ -97,20 +141,30 @@ private ManifestFiles() {
    * @param outputFile the destination file location
    * @return a manifest writer
    */
-  public static ManifestWriter<DataFile> write(PartitionSpec spec, OutputFile outputFile) {
+  public static ManifestWriter<DataFile> write(PartitionSpec spec, EncryptedOutputFile outputFile) {
     return write(1, spec, outputFile, null);
   }
 
   /**
-   * Create a new {@link ManifestWriter} for the given format version.
+   * @deprecated please use {@link #write(int, PartitionSpec, EncryptedOutputFile, Long)}
+   */
+  @Deprecated
+  public static ManifestWriter<DataFile> write(int formatVersion, PartitionSpec spec, OutputFile outputFile,
+                                               Long snapshotId) {
+    return write(formatVersion, spec, EncryptedFiles.encryptedOutput(outputFile,
+            EncryptionKeyMetadata.EMPTY), snapshotId);

Review comment:
       A nit: 4 spaces indentation rather than 8 spaces.




-- 
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.

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] jackye1995 commented on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-841538934


   @rdblue @flyrain @RussellSpitzer I have updated this PR with all the changes I expected to make in several PRs, it is a lot of files but I think it makes the discussion much easier to show them in a single place.
   
   The key thing we need to determine is: should we separate the code path for manifest read/write with and without encryption. My stand is that we should not separate it, because:
   1. data file read/write does not separate it
   2. managing things in a single  code path is much easier in long run
   
   With this assumption, the key principles I followed for the change is:
   1. for every method in `ManifestFiles`, create a new version that takes encryption manager, and deprecate the old methods
   2. change the callers of old methods all the way to the top, and deprecate any related public methods.
   3. use `EncryptionManagers.plaintext()` for all those deprecated methods so that we don't create a lot of redundant plain text manager.
   
   Regarding the concern brought up by @RussellSpitzer and @flyrain , my current take is that we have to pass `EncryptedOutputFile` all the way down to the place where the `encryptedOutputFile.keyMetadata()` is written by the engine to the actual manifest list. This does make the naming a bit awkward as @flyrain suggested, but I think this is the same strategy taken by the data file write path, where we can notice that the `FileAppenderFactory` also have those methods containing `EncryptedOutputFile` for exactly the same purpose.
   
   Regarding `EncryptionManagers` that @rdblue talked about removing, as I said in point 3 before, now I am treating it as the static factory to get a plain text manager singleton. I do expect to have a second usage of it, similar to `LocationProviders`, to add a method `EncryptionManagers.load(...)` to load a custom encryption manager and allow a `Catalog` to initialize a custom encryption manager for a `TableOperation`.


-- 
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.

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] flyrain commented on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-855039496


   @jackye1995 thanks for the update, looking forward to your new PRs. 


-- 
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.

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 change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r624757644



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -58,6 +60,7 @@
   private Long existingRowsCount = null;
   private Long deletedRowsCount = null;
   private PartitionFieldSummary[] partitions = null;
+  private byte[] keyMetadata = null;

Review comment:
       I think it's better to initialize than to let it be implicit. That way it is always clear to anyone reading the code what the initial value is, whether or not they know what the JVM would fill in.




-- 
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.

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] jackye1995 commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r621448945



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -349,6 +362,9 @@ public Object get(int i) {
         this.partitions = value == null ? null :
             ((List<PartitionFieldSummary>) value).toArray(new PartitionFieldSummary[0]);
         return;
+      case 14:
+        this.keyMetadata = ByteBuffers.toByteArray((ByteBuffer) value);

Review comment:
       `ByteBuffers.toByteArray` is null safe, that's why I did not add the check.




-- 
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.

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] jackye1995 commented on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-848118807


   > The main concern that I have is the ability to use the bulk decryption methods from the manager so planning a scan doesn't necessarily incur repeated RPCs to a key manager
   
   I suppose you are describing something similar to what is done for data files like here: 
   
   https://github.com/apache/iceberg/blob/master/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java#L74-L75
   
   I wonder if there are any real use cases where this approach actually saves RPC calls to a KMS. 
   
   Because in the envelope encryption schemes we discussed:
   1. if it is a single wrap system, then for each file we need to call KMS once to get a new data encryption key (DEK) based on the same key encryption key (KEK), so we always have N KMS calls for N files. (unless the KMS has a batch DEK generation API, which is not very common as far as I know)
   2. if it is a double wrap system, then the first call gets a master encryption key (MEK) to generate a KEK, and all DEKs are generated locally based on that cached KEK. So we always have 1 KMS call for N files.
   
   So the number of calls to KMS really depends on the algorithm, not that much on the encryption manager API, unless the KMS allows batch operation or it is a completely different encryption scheme. And the KMS would be embedded in the encryption manager implementation and perform necessary caching of keys behind the scene.
   
   With that being said, I completely agree that we should use batch operation whenever possible, I will try to find a best way to update the manifest read and write path to be able to leverage batch operations. I will provide an updated PR tonight, meanwhile please let me know what you think about the comment above.
   
   


-- 
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.

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] flyrain commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r643564071



##########
File path: core/src/main/java/org/apache/iceberg/ManifestFiles.java
##########
@@ -67,26 +90,47 @@ private ManifestFiles() {
    * @param io a FileIO
    * @return a manifest reader
    */
-  public static ManifestReader<DataFile> read(ManifestFile manifest, FileIO io) {
-    return read(manifest, io, null);
+  public static ManifestReader<DataFile> read(ManifestFile manifest, FileIO io, EncryptionManager encryption) {
+    return read(manifest, io, encryption, null);
+  }
+
+  /**
+   * @deprecated please use {@link #read(ManifestFile, FileIO, EncryptionManager, Map)}
+   */
+  @Deprecated
+  public static ManifestReader<DataFile> read(ManifestFile manifest, FileIO io, Map<Integer, PartitionSpec> specsById) {
+    return read(manifest, io, EncryptionManagers.plainText(), specsById);
   }
 
   /**
    * Returns a new {@link ManifestReader} for a {@link ManifestFile}.
    *
-   * @param manifest a {@link ManifestFile}
+   * @param manifest an encrypted {@link ManifestFile}
    * @param io a {@link FileIO}
+   * @param encryption a {@link EncryptionManager}
    * @param specsById a Map from spec ID to partition spec
    * @return a {@link ManifestReader}
    */
-  public static ManifestReader<DataFile> read(ManifestFile manifest, FileIO io, Map<Integer, PartitionSpec> specsById) {
+  public static ManifestReader<DataFile> read(ManifestFile manifest, FileIO io, EncryptionManager encryption,
+                                              Map<Integer, PartitionSpec> specsById) {
     Preconditions.checkArgument(manifest.content() == ManifestContent.DATA,
         "Cannot read a delete manifest with a ManifestReader: %s", manifest);
-    InputFile file = io.newInputFile(manifest.path());
+
+    EncryptedInputFile encryptedFile = EncryptedFiles.encryptedInput(
+            io.newInputFile(manifest.path()), manifest.keyMetadata());

Review comment:
       A nit: 4 spaces indentation rather than 8 spaces.




-- 
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.

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 pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-849386353


   Double wrapping will be an Iceberg mode, always under our control, an assured way to avoid per-file KMS calls.
   But certainly, if a KMS supports batch calls (some do; not all), this should be leveraged by Iceberg in either single or double wrapping mode. This might require an explicit addition in the Iceberg "kms client" plug-in interface (aka "key_provider"), because a typical per-key interface wouldn't know when the batch request is "full" and should be sent to the KMS server. 


-- 
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.

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] jackye1995 edited a comment on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 edited a comment on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-841538934


   @rdblue @flyrain @RussellSpitzer I have updated this PR with all the changes I expected to make in several PRs, it is a lot of files but I think it makes the discussion much easier to show them in a single place.
   
   The key thing we need to determine is: should we separate the code path for manifest read/write with and without encryption. My stand is that we should not separate it, because:
   1. data file read/write does not separate it
   2. managing things in a single  code path is much easier in long run
   
   With this assumption, the key principles I followed for the changes are:
   1. for every method in `ManifestFiles`, create a new version that takes encryption manager, and deprecate the old methods
   2. change the callers of old methods all the way to the top, and deprecate any related public methods.
   3. use `EncryptionManagers.plaintext()` for all those deprecated methods so that we don't create a lot of redundant plain text manager.
   
   Regarding the concern brought up by @RussellSpitzer and @flyrain , my current take is that we have to pass `EncryptedOutputFile` all the way down to the place where the `encryptedOutputFile.keyMetadata()` is written by the engine to the actual manifest list. This does make the naming a bit awkward as @flyrain suggested, but I think this is the same strategy taken by the data file write path, where we can notice that the `FileAppenderFactory` also have those methods containing `EncryptedOutputFile` for exactly the same purpose.
   
   Regarding `EncryptionManagers` that @rdblue talked about removing, as I said in point 3 before, now I am treating it as the static factory to get a plain text manager singleton and that is only used for deprecated methods. I do expect to have a second usage of the factory, similar to `LocationProviders`, to add a method `EncryptionManagers.load(...)` to load a custom encryption manager and allow a `Catalog` to initialize a custom encryption manager for a `TableOperations`.


-- 
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.

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] yyanyy commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r622668587



##########
File path: api/src/main/java/org/apache/iceberg/ManifestFile.java
##########
@@ -179,6 +181,13 @@ default boolean hasDeletedFiles() {
    */
   List<PartitionFieldSummary> partitions();
 
+  /**
+   * Returns metadata about how this manifest file is encrypted, or null if the file is stored in plain text.
+   */
+  default ByteBuffer keyMetadata() {

Review comment:
       I was wondering about this too, should we make this a struct with name like `encryptionContext` or something so that if we only plan to add new things in future (e.g. KEK id for double wrapping?), we can collect them in a single struct; and to workaround the problem of having to unwrap two layers to reach this buffer from `ManifestFile` we may return `EncryptionKeyMetadata` here, and potentially extend `EncryptionKeyMetadata` to have more fields when needed in the future. Or will this binary buffer free formed and could contain whatever information needed if the right encryption manager is used? 




-- 
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.

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] flyrain commented on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-852603603


   > Regarding the concern brought up by @RussellSpitzer and @flyrain , my current take is that we have to pass `EncryptedOutputFile` all the way down to the place where the `encryptedOutputFile.keyMetadata()` is written by the engine to the actual manifest list. This does make the naming a bit awkward as @flyrain suggested, but I think this is the same strategy taken by the data file write path, where we can notice that the `FileAppenderFactory` also have those methods containing `EncryptedOutputFile` for exactly the same purpose.
   
   I don't have a strong opinion on this. The logic looks good to me. But the more descriptive name provides benefits:
   1. Doesn't confuse code reader, reader won't confuse a plain text file with an encrypted file.
   2. Better naming usually means better abstraction, and more future-approved. However, sometimes, it is kind of over-engineered, but less likely, and need to be analyzed case by case.


-- 
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.

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] flyrain edited a comment on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain edited a comment on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-852603603


   > Regarding the concern brought up by @RussellSpitzer and @flyrain , my current take is that we have to pass `EncryptedOutputFile` all the way down to the place where the `encryptedOutputFile.keyMetadata()` is written by the engine to the actual manifest list. This does make the naming a bit awkward as @flyrain suggested, but I think this is the same strategy taken by the data file write path, where we can notice that the `FileAppenderFactory` also have those methods containing `EncryptedOutputFile` for exactly the same purpose.
   
   I don't have a strong opinion on this. The logic looks good to me. But the more descriptive name provides benefits:
   1. Doesn't confuse code reader, reader won't confuse a plain text file with an encrypted file.
   2. Better naming usually means better abstraction, and more future-proof. However, sometimes, it is kind of over-engineered, but less likely, and need to be analyzed case by case.


-- 
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.

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] RussellSpitzer commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r620796256



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -399,6 +415,7 @@ public String toString() {
         .add("deleted_data_files_count", deletedFilesCount)
         .add("deleted_rows_count", deletedRowsCount)
         .add("partitions", partitions)
+        .add("key_metadata", keyMetadata == null ? "null" : "(redacted)")

Review comment:
       Is this a place where we are leaking information? Sorry I'm still a crypto beginner, is it ok for us to reveal that a file is encrypted? I guess that's obvious if you can get the file ....?




-- 
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.

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] flyrain commented on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-828839921


   The change from OutputFile to EncryptedOutputFile looks weird to me as well. Logic is OK to me though. Just brainstorming ideas. 
   Option 1. We can rename EncryptedOutputFile something else, like GenericOutputFile, which doesn’t confuse people from name perspective. 
   Option 2. Have a new interface(e.g. GenericOutputFile) inherited by both EncryptedOutputFile and Output file. The interface can be empty. 


-- 
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.

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 pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-844585720


   @jackye1995, I'm not sure it makes sense to use the plain text manager as you're suggesting. I know it mimics the choice we made for data files, but data files are a bit cleaner because they're not using similar methods to open the files. The main concern that I have is the ability to use the bulk decryption methods from the manager so planning a scan doesn't necessarily incur repeated RPCs to a key manager. As long as that is possible, then I'm flexible on the exact API here.


-- 
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.

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] jackye1995 commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r621451754



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -399,6 +415,7 @@ public String toString() {
         .add("deleted_data_files_count", deletedFilesCount)
         .add("deleted_rows_count", deletedRowsCount)
         .add("partitions", partitions)
+        .add("key_metadata", keyMetadata == null ? "null" : "(redacted)")

Review comment:
       yeah that's a good catch, I am also not sure if this is considered fine, I just followed what the `BaseFile` did: https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/BaseFile.java#L454




-- 
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.

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 change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r624780509



##########
File path: core/src/main/java/org/apache/iceberg/encryption/EncryptionManagers.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+public class EncryptionManagers {

Review comment:
       Encryption managers are plugged in through tables, so I don't think it is a good idea to have a global default. That doesn't seem to make sense with how they are used.




-- 
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.

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] flyrain commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r622595167



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -58,6 +60,7 @@
   private Long existingRowsCount = null;
   private Long deletedRowsCount = null;
   private PartitionFieldSummary[] partitions = null;
+  private byte[] keyMetadata = null;

Review comment:
       A nit: We don't have to initialize a variable in Java, but since all others do the same thing. It should be fine to be consistent here. 




-- 
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.

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 pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-848311406


   The situation where we want to use the bulk API is whenever we are going to make a call to the KMS per file. If we can batch up the files then we can make a single call to get all of the keys at once. You may be right that the strategies that you want to build don't require it, but the plugin system is generic enough that we don't necessarily know that batching is unnecessary.


-- 
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.

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] flyrain commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r622606029



##########
File path: core/src/main/java/org/apache/iceberg/encryption/EncryptionManagers.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+public class EncryptionManagers {
+  private static final EncryptionManager DEFAULT_MANAGER = new PlaintextEncryptionManager();

Review comment:
       Maybe we call it PLAIN_TEXT_MANAGER? so that reader can get it right away instead of checking class `PlaintextEncryptionManager`. Besides, this is not introduced by this patch, but the name of `PlaintextEncryptionManager` itself is also confusing. If we cannot change it to a more descriptive name, we may add a comment in the class to indicate it doesn't do any encryption/decryption.




-- 
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.

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] RussellSpitzer commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r620801590



##########
File path: core/src/main/java/org/apache/iceberg/ManifestFiles.java
##########
@@ -80,9 +98,26 @@ private ManifestFiles() {
    * @return a {@link ManifestReader}
    */
   public static ManifestReader<DataFile> read(ManifestFile manifest, FileIO io, Map<Integer, PartitionSpec> specsById) {
+    return read(manifest, io, EncryptionManagers.defaultManager(), specsById);
+  }
+
+  /**
+   * Returns a new {@link ManifestReader} for an encrypted {@link ManifestFile}.
+   *
+   * @param manifest an encrypted {@link ManifestFile}
+   * @param io a {@link FileIO}
+   * @param encryption a {@link EncryptionManager}
+   * @param specsById a Map from spec ID to partition spec
+   * @return a {@link ManifestReader}
+   */
+  public static ManifestReader<DataFile> read(ManifestFile manifest, FileIO io, EncryptionManager encryption,
+                                              Map<Integer, PartitionSpec> specsById) {
     Preconditions.checkArgument(manifest.content() == ManifestContent.DATA,
-        "Cannot read a delete manifest with a ManifestReader: %s", manifest);
-    InputFile file = io.newInputFile(manifest.path());
+        "Cannot read a delete manifest with ManifestReader: %s", manifest);

Review comment:
       I'm not sure that the previous version of this error message is wrong since it's trying to create a ManifestReader to read a delete manifest ... I dunno, makes enough sense either way to me although I think I prefer the original




-- 
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.

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] flyrain commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r643569231



##########
File path: core/src/main/java/org/apache/iceberg/StaticTableOperations.java
##########
@@ -32,25 +34,36 @@
   private TableMetadata staticMetadata;
   private final String metadataFileLocation;
   private final FileIO io;
+  private final EncryptionManager encryption;
   private final LocationProvider locationProvider;
 
   /**
-   * Creates a StaticTableOperations tied to a specific static version of the TableMetadata
+   * @deprecated please use {@link #StaticTableOperations(String, FileIO, EncryptionManager)}

Review comment:
       Do we need to deprecate it? To provide a method with a plainText EncryptionManager looks harmless.




-- 
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.

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] jackye1995 commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r621453066



##########
File path: core/src/main/java/org/apache/iceberg/ManifestFiles.java
##########
@@ -80,9 +98,26 @@ private ManifestFiles() {
    * @return a {@link ManifestReader}
    */
   public static ManifestReader<DataFile> read(ManifestFile manifest, FileIO io, Map<Integer, PartitionSpec> specsById) {
+    return read(manifest, io, EncryptionManagers.defaultManager(), specsById);
+  }
+
+  /**
+   * Returns a new {@link ManifestReader} for an encrypted {@link ManifestFile}.
+   *
+   * @param manifest an encrypted {@link ManifestFile}
+   * @param io a {@link FileIO}
+   * @param encryption a {@link EncryptionManager}
+   * @param specsById a Map from spec ID to partition spec
+   * @return a {@link ManifestReader}
+   */
+  public static ManifestReader<DataFile> read(ManifestFile manifest, FileIO io, EncryptionManager encryption,
+                                              Map<Integer, PartitionSpec> specsById) {
     Preconditions.checkArgument(manifest.content() == ManifestContent.DATA,
-        "Cannot read a delete manifest with a ManifestReader: %s", manifest);
-    InputFile file = io.newInputFile(manifest.path());
+        "Cannot read a delete manifest with ManifestReader: %s", manifest);

Review comment:
       sorry that's definitely an unintentional change...




-- 
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.

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] jackye1995 commented on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-841430136


   Sorry I was a bit distracted and did not update this PR for a while.
   
   @rdblue thanks for the suggestion for refactoring, I actually thought about the approach you considered, and my major concern is that we are creating 2 different code paths for using or not using encryption. On the other hand, for data file encryption, Iceberg is currently having a single unified code path for always using encryption, with the default encryption manager to be the plaintext manager. I think it would be good to keep things consistent. 
   
   I completely agree that the use of a global default encryption manager class is not necessary, and I will remove that. What I hope to achieve is that the method `ManifestFiles.read(ManifestFile manifest, FileIO io, Map<Integer, PartitionSpec> specsById)` is completely not used and deprecated, and all the callers should use the method with encryption, and supply `table.encryption()` as the input to the encryption manager. I was trying to avoid too many single line changes in this PR, but my final intention became not clear, so I will update based on that and we can reevaluate the situation.
   
   For what @flyrain and @RussellSpitzer concern, I think I can make it better by directly passing in `EncryptedOutputFIle.encryptingOutputFile()` to minimize interface change, will notify you guys once I update.


-- 
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.

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] flyrain commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r625234029



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -58,6 +60,7 @@
   private Long existingRowsCount = null;
   private Long deletedRowsCount = null;
   private PartitionFieldSummary[] partitions = null;
+  private byte[] keyMetadata = null;

Review comment:
       Either way is OK to me as long as it is consistent here or across the project. To avoid getting back to this style issue again(we can work on more important things with that energy), I suggest to have this kind of style preference written down as a developer/reviewer guidance, so that at least we can refer to it. 




-- 
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.

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 change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r625023097



##########
File path: api/src/main/java/org/apache/iceberg/ManifestFile.java
##########
@@ -179,6 +181,13 @@ default boolean hasDeletedFiles() {
    */
   List<PartitionFieldSummary> partitions();
 
+  /**
+   * Returns metadata about how this manifest file is encrypted, or null if the file is stored in plain text.
+   */
+  default ByteBuffer keyMetadata() {

Review comment:
       +1 to a byte array, serialized by an encryption manager from its structs. Btw, besides encryption keys, we have the AAD prefixes. We can keep them inside the key metadata (because it is convenient and flexible) - or we can add a separate manifest field/column for them (because technically AADs are not used for key retrieval). In both cases, the decision can be made later, when we get to handle end-to-end table integrity.




-- 
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.

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] RussellSpitzer commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r621453224



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -349,6 +362,9 @@ public Object get(int i) {
         this.partitions = value == null ? null :
             ((List<PartitionFieldSummary>) value).toArray(new PartitionFieldSummary[0]);
         return;
+      case 14:
+        this.keyMetadata = ByteBuffers.toByteArray((ByteBuffer) value);

Review comment:
       Excellent!




-- 
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.

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 change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r624757760



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -399,6 +415,7 @@ public String toString() {
         .add("deleted_data_files_count", deletedFilesCount)
         .add("deleted_rows_count", deletedRowsCount)
         .add("partitions", partitions)
+        .add("key_metadata", keyMetadata == null ? "null" : "(redacted)")

Review comment:
       I think this is fine. The key metadata could be a 0-length buffer set an encryption manager if someone cares about this. I think it's reasonable to show that it is null when using the plain text manager.




-- 
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.

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] jackye1995 commented on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-854435599


   @flyrain thanks for the review! I am marking this as a draft. I spent a few days to figure out the way to go forward with all the changes needed in `ManifestFiles` API and I think I have a plan. I will separate this PR to a few different ones, mainly:
   
   1. adding `keyMetadata` byte array to `ManifestFile` interface and implementation classes.
   2. add support in `EncryptionManager` to decrypt and encrypt by file type
   3. changes to use encryption manager to supply key metadata information down to manifest writer and reader. This will be separated to multiple PRs for core, and engine related changes.
   


-- 
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.

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 change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r624777762



##########
File path: core/src/main/java/org/apache/iceberg/ManifestFiles.java
##########
@@ -80,9 +98,26 @@ private ManifestFiles() {
    * @return a {@link ManifestReader}
    */
   public static ManifestReader<DataFile> read(ManifestFile manifest, FileIO io, Map<Integer, PartitionSpec> specsById) {
+    return read(manifest, io, EncryptionManagers.defaultManager(), specsById);

Review comment:
       I'd like to have compatibility with the bulk-decryption and bulk-encryption methods in the encryption manager. For bulk decryption, the `decrypt` method is passed `Iterable<EncryptedInputFile>` and returns `Iterable<InputFile>`. Then those input files need to be used.
   
   I think that means we should refactor this a bit differently and create a `read(ManifestFile, InputFile, Map)` method that both this and the version with `EncryptionManager` call. Then this one could simply call `io.newInputFile(manifest.path())` and hand off to the `InputFile` reader. That avoids using the default manager where it isn't needed and allows us to use the bulk 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.

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] flyrain commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r622609911



##########
File path: api/src/main/java/org/apache/iceberg/ManifestFile.java
##########
@@ -179,6 +181,13 @@ default boolean hasDeletedFiles() {
    */
   List<PartitionFieldSummary> partitions();
 
+  /**
+   * Returns metadata about how this manifest file is encrypted, or null if the file is stored in plain text.
+   */
+  default ByteBuffer keyMetadata() {

Review comment:
       Does keyMetadata have substructure or is it a pure binary buffer? Looks like it will have substructures form the description. Are we going to to define it later or in this patch?




-- 
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.

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] flyrain commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r643564477



##########
File path: core/src/main/java/org/apache/iceberg/ManifestFiles.java
##########
@@ -121,34 +175,56 @@ private ManifestFiles() {
     throw new UnsupportedOperationException("Cannot write manifest for table version: " + formatVersion);
   }
 
+  /**
+   * @deprecated please use {@link #readDeleteManifest(ManifestFile, FileIO, EncryptionManager, Map)}
+   */
+  @Deprecated
+  public static ManifestReader<DeleteFile> readDeleteManifest(ManifestFile manifest, FileIO io,
+                                                              Map<Integer, PartitionSpec> specsById) {
+    return readDeleteManifest(manifest, io, EncryptionManagers.plainText(), specsById);
+  }
+
   /**
    * Returns a new {@link ManifestReader} for a {@link ManifestFile}.
    *
    * @param manifest a {@link ManifestFile}
    * @param io a {@link FileIO}
+   * @param encryption a {@link EncryptionManager}
    * @param specsById a Map from spec ID to partition spec
    * @return a {@link ManifestReader}
    */
   public static ManifestReader<DeleteFile> readDeleteManifest(ManifestFile manifest, FileIO io,
+                                                              EncryptionManager encryption,
                                                               Map<Integer, PartitionSpec> specsById) {
     Preconditions.checkArgument(manifest.content() == ManifestContent.DELETES,
         "Cannot read a data manifest with a DeleteManifestReader: %s", manifest);
-    InputFile file = io.newInputFile(manifest.path());
+    InputFile file = encryption.decrypt(EncryptedFiles.encryptedInput(
+        io.newInputFile(manifest.path()), manifest.keyMetadata()));
     InheritableMetadata inheritableMetadata = InheritableMetadataFactory.fromManifest(manifest);
     return new ManifestReader<>(file, specsById, inheritableMetadata, FileType.DELETE_FILES);
   }
 
+  /**
+   * @deprecated please use {@link #writeDeleteManifest(int, PartitionSpec, EncryptedOutputFile, Long)}
+   */
+  @Deprecated
+  public static ManifestWriter<DeleteFile> writeDeleteManifest(int formatVersion, PartitionSpec spec,
+                                                               OutputFile outputFile, Long snapshotId) {
+    return writeDeleteManifest(formatVersion, spec,
+            EncryptedFiles.encryptedOutput(outputFile, EncryptionKeyMetadata.EMPTY), snapshotId);

Review comment:
       A nit: 4 spaces indentation rather than 8 spaces.




-- 
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.

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] RussellSpitzer commented on a change in pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#discussion_r620795682



##########
File path: core/src/main/java/org/apache/iceberg/GenericManifestFile.java
##########
@@ -349,6 +362,9 @@ public Object get(int i) {
         this.partitions = value == null ? null :
             ((List<PartitionFieldSummary>) value).toArray(new PartitionFieldSummary[0]);
         return;
+      case 14:
+        this.keyMetadata = ByteBuffers.toByteArray((ByteBuffer) value);

Review comment:
       I think you'll need to do null guarding here as well,
   ```
   this.keyMetadata = value == null ? null : .....
   ```




-- 
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.

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] jackye1995 commented on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-827798130


   > The change from OutputFile to EncryptedOutputFile everywhere looks a little odd to me because I know sometimes our output will not be encrypted even though we use that class.
   
   @RussellSpitzer I think this pattern is also used currently by data files, where all files are encrypted before written and go through the OutputFileFactory to get the output file instances:
   
   https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java#L106-L118
   
   And we are not encrypting the file only because we are using the PlainTextEncryptionManager. The change here is to try achieving the same effect.


-- 
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.

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] jackye1995 commented on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-841585247


   restart test


-- 
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.

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] jackye1995 edited a comment on pull request #2520: Core: add key_metadata in ManifestFile

Posted by GitBox <gi...@apache.org>.
jackye1995 edited a comment on pull request #2520:
URL: https://github.com/apache/iceberg/pull/2520#issuecomment-841538934


   @rdblue @flyrain @RussellSpitzer I have updated this PR with all the changes I expected to make in several PRs, it is a lot of files but I think it makes the discussion much easier to show them in a single place.
   
   The key thing we need to determine is: should we separate the code path for manifest read/write with and without encryption. My stand is that we should not separate it, because:
   1. data file read/write does not separate it
   2. managing things in a single  code path is much easier in long run
   
   With this assumption, the key principles I followed for the changes are:
   1. for every method in `ManifestFiles`, create a new version that takes encryption manager, and deprecate the old methods
   2. change the callers of old methods all the way to the top, and deprecate any related public methods.
   3. use `EncryptionManagers.plaintext()` for all those deprecated methods so that we don't create a lot of redundant plain text manager.
   
   Regarding the concern brought up by @RussellSpitzer and @flyrain , my current take is that we have to pass `EncryptedOutputFile` all the way down to the place where the `encryptedOutputFile.keyMetadata()` is written by the engine to the actual manifest list. This does make the naming a bit awkward as @flyrain suggested, but I think this is the same strategy taken by the data file write path, where we can notice that the `FileAppenderFactory` also have those methods containing `EncryptedOutputFile` for exactly the same purpose.
   
   Regarding `EncryptionManagers` that @rdblue talked about removing, as I said in point 3 before, now I am treating it as the static factory to get a plain text manager singleton and that is only used for deprecated methods. I do expect to have a second usage of it, similar to `LocationProviders`, to add a method `EncryptionManagers.load(...)` to load a custom encryption manager and allow a `Catalog` to initialize a custom encryption manager for a `TableOperations`.


-- 
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.

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