You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/07/29 07:50:25 UTC

[GitHub] [iceberg] zhongyujiang commented on a diff in pull request #3471: Core: Envelope encryption

zhongyujiang commented on code in PR #3471:
URL: https://github.com/apache/iceberg/pull/3471#discussion_r932928445


##########
core/src/main/java/org/apache/iceberg/encryption/EnvelopeEncryptionManager.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.nio.ByteBuffer;
+import java.security.SecureRandom;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Encryption manager which in conjunction with a KMS can encrypt {@link OutputFile} and decrypt
+ * {@link InputFile}. Envelope encryption uses a key wrapping strategy, where a Key Encryption Key (KEK) is used to
+ * wrap or unwrap a Data Encryption Key DEK which is used to encrypt the underlying files.
+ *
+ * When generating new DEKs for OutputFiles, this class will first attempt to have the KMS generate a new key. If the
+ * KMS does not support key generation a new key will be produced by pulling bytes from a {@link SecureRandom} on the
+ * JVM writing the file.
+ */
+public class EnvelopeEncryptionManager implements EncryptionManager {
+  private final EnvelopeConfiguration dataEncryptionConfig;
+  private final boolean nativeFormatEncryption;
+  private final KmsClient kmsClient;
+  private final int dataKeyLength;
+  private final boolean kmsGeneratedKeys;
+
+  private transient volatile SecureRandom workerRNG = null;
+
+  public static final String encryptionConfigMismatchMessagePrefix =
+      "Cannot create encryption manager - detected mismatch between client encryption properties and table " +
+          "encryption properties. Mismatch can be caused by changing encryption properties after table creation " +
+          "(currently not supported), or by malicious tampering with table properties kept in metadata.json in " +
+          "storage. Mismatch details: ";
+
+  /**
+   * @param nativeFormatEncryption true for native encryption inside file formats (Parquet or ORC), false for generic
+   *                               encryption of file streams
+   * @param dataEncryptionConfig configuration for encryption of data files. Future versions will add configuration for
+   *                             encryption of metadata files
+   * @param kmsClient Client of KMS used to wrap/unwrap keys in envelope encryption
+   * @param dataKeyLength If DEKs are not generated by KMS, they will be randomly generated by Iceberg using this length
+   */
+  public EnvelopeEncryptionManager(boolean nativeFormatEncryption,  EnvelopeConfiguration dataEncryptionConfig,
+      KmsClient kmsClient, int dataKeyLength) {
+    if (!nativeFormatEncryption) {
+      throw new UnsupportedOperationException("EnvelopeEncryptionManager currently only supports encryption " +
+          "provided by the underlying file format.");
+    }
+    this.nativeFormatEncryption = nativeFormatEncryption;
+    Preconditions.checkNotNull(dataEncryptionConfig,
+        "Cannot create EnvelopeEncryptionManager because data encryption config is not passed");
+    this.dataEncryptionConfig = dataEncryptionConfig;
+    Preconditions.checkNotNull(dataEncryptionConfig.kekId(),
+        "Cannot create EnvelopeEncryptionManager because table key encryption key ID is not specified");
+    Preconditions.checkNotNull(kmsClient,
+        "Cannot create EnvelopeEncryptionManager because KmsClient is null");
+    this.kmsClient = kmsClient;
+    this.dataKeyLength = dataKeyLength;
+    this.kmsGeneratedKeys = kmsClient.supportsKeyGeneration();
+  }
+
+  @Override
+  public EncryptedOutputFile encrypt(OutputFile rawOutput) {
+    EnvelopeMetadata metadata = generateEnvelopeMetadata(dataEncryptionConfig);
+
+    if (nativeFormatEncryption) {
+      NativeFileCryptoParameters nativeEncryptParams = NativeFileCryptoParameters.create(metadata.dek())
+          .encryptionAlgorithm(metadata.algorithm())
+          .build();
+
+      if (!(rawOutput instanceof NativelyEncryptedFile)) {
+        throw new RuntimeException("Can't natively encrypt " + rawOutput.location() + " because the class " +
+            rawOutput.getClass() + " doesn't implement NativelyEncryptedFile interface");
+      }
+
+      ((NativelyEncryptedFile) rawOutput).setNativeCryptoParameters(nativeEncryptParams);
+
+      return new BaseEncryptedOutputFile(rawOutput, metadata);
+    } else {
+      throw new UnsupportedOperationException("EnvelopeEncryptionManager currently supports only file formats which " +
+          "have native encryption implementations");
+    }
+  }
+
+  @Override
+  public InputFile decrypt(EncryptedInputFile encrypted) {
+    if (encrypted.keyMetadata().buffer() == null) { // unencrypted file
+      return encrypted.encryptedInputFile();
+    }
+    EnvelopeMetadata metadata = EnvelopeMetadataParser.fromJson(encrypted.keyMetadata().buffer());
+    ByteBuffer fileDek = kmsClient.unwrapKey(metadata.wrappedDek(), metadata.kekId());

Review Comment:
   I think `unwrap` should only be called when it is sure that decryption is needed to avoid unnecessary RPC calls.



##########
core/src/main/java/org/apache/iceberg/encryption/EnvelopeEncryptionManager.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.nio.ByteBuffer;
+import java.security.SecureRandom;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Encryption manager which in conjunction with a KMS can encrypt {@link OutputFile} and decrypt
+ * {@link InputFile}. Envelope encryption uses a key wrapping strategy, where a Key Encryption Key (KEK) is used to
+ * wrap or unwrap a Data Encryption Key DEK which is used to encrypt the underlying files.
+ *
+ * When generating new DEKs for OutputFiles, this class will first attempt to have the KMS generate a new key. If the
+ * KMS does not support key generation a new key will be produced by pulling bytes from a {@link SecureRandom} on the
+ * JVM writing the file.
+ */
+public class EnvelopeEncryptionManager implements EncryptionManager {
+  private final EnvelopeConfiguration dataEncryptionConfig;
+  private final boolean nativeFormatEncryption;
+  private final KmsClient kmsClient;
+  private final int dataKeyLength;
+  private final boolean kmsGeneratedKeys;
+
+  private transient volatile SecureRandom workerRNG = null;
+
+  public static final String encryptionConfigMismatchMessagePrefix =
+      "Cannot create encryption manager - detected mismatch between client encryption properties and table " +
+          "encryption properties. Mismatch can be caused by changing encryption properties after table creation " +
+          "(currently not supported), or by malicious tampering with table properties kept in metadata.json in " +
+          "storage. Mismatch details: ";
+
+  /**
+   * @param nativeFormatEncryption true for native encryption inside file formats (Parquet or ORC), false for generic
+   *                               encryption of file streams
+   * @param dataEncryptionConfig configuration for encryption of data files. Future versions will add configuration for
+   *                             encryption of metadata files
+   * @param kmsClient Client of KMS used to wrap/unwrap keys in envelope encryption
+   * @param dataKeyLength If DEKs are not generated by KMS, they will be randomly generated by Iceberg using this length
+   */
+  public EnvelopeEncryptionManager(boolean nativeFormatEncryption,  EnvelopeConfiguration dataEncryptionConfig,
+      KmsClient kmsClient, int dataKeyLength) {
+    if (!nativeFormatEncryption) {
+      throw new UnsupportedOperationException("EnvelopeEncryptionManager currently only supports encryption " +
+          "provided by the underlying file format.");
+    }
+    this.nativeFormatEncryption = nativeFormatEncryption;
+    Preconditions.checkNotNull(dataEncryptionConfig,
+        "Cannot create EnvelopeEncryptionManager because data encryption config is not passed");
+    this.dataEncryptionConfig = dataEncryptionConfig;
+    Preconditions.checkNotNull(dataEncryptionConfig.kekId(),
+        "Cannot create EnvelopeEncryptionManager because table key encryption key ID is not specified");
+    Preconditions.checkNotNull(kmsClient,
+        "Cannot create EnvelopeEncryptionManager because KmsClient is null");
+    this.kmsClient = kmsClient;
+    this.dataKeyLength = dataKeyLength;
+    this.kmsGeneratedKeys = kmsClient.supportsKeyGeneration();
+  }
+
+  @Override
+  public EncryptedOutputFile encrypt(OutputFile rawOutput) {
+    EnvelopeMetadata metadata = generateEnvelopeMetadata(dataEncryptionConfig);

Review Comment:
   Same here, should generate keys only when confirming that encryption is required.



##########
core/src/main/java/org/apache/iceberg/encryption/EnvelopeMetadataParser.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class EnvelopeMetadataParser {
+
+  private EnvelopeMetadataParser() {
+  }
+
+  private static final String KEK_ID = "kek-id";
+  private static final String WRAPPED_DEK = "wrapped-dek";
+  private static final String ALGORITHM = "algorithm";
+
+  public static void toJson(EnvelopeMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeStringField(KEK_ID, metadata.kekId());
+    generator.writeStringField(WRAPPED_DEK, metadata.wrappedDek());
+    JsonUtil.writeStringIf(null != metadata.algorithm(), ALGORITHM, metadata.algorithm().name(), generator);
+
+    generator.writeEndObject();
+  }
+
+  public static ByteBuffer toJson(EnvelopeMetadata metadata) {
+    try {
+      ByteArrayOutputStream stream = new ByteArrayOutputStream();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(stream);
+      toJson(metadata, generator);
+      generator.flush();
+      return ByteBuffer.wrap(stream.toByteArray());
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  public static EnvelopeMetadata fromJson(JsonNode json) throws IOException {
+    Preconditions.checkArgument(json.isObject(), "Cannot parse envelope metadata from non-object: %s", json);

Review Comment:
   nit: For the reason mentioned above by shangxinli, this check should also be replaced with an if block to avoiding string concat, it's called per file too.



##########
core/src/main/java/org/apache/iceberg/encryption/EnvelopeEncryptionManager.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.nio.ByteBuffer;
+import java.security.SecureRandom;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Encryption manager which in conjunction with a KMS can encrypt {@link OutputFile} and decrypt
+ * {@link InputFile}. Envelope encryption uses a key wrapping strategy, where a Key Encryption Key (KEK) is used to
+ * wrap or unwrap a Data Encryption Key DEK which is used to encrypt the underlying files.
+ *
+ * When generating new DEKs for OutputFiles, this class will first attempt to have the KMS generate a new key. If the
+ * KMS does not support key generation a new key will be produced by pulling bytes from a {@link SecureRandom} on the
+ * JVM writing the file.
+ */
+public class EnvelopeEncryptionManager implements EncryptionManager {
+  private final EnvelopeConfiguration dataEncryptionConfig;
+  private final boolean nativeFormatEncryption;
+  private final KmsClient kmsClient;
+  private final int dataKeyLength;
+  private final boolean kmsGeneratedKeys;
+
+  private transient volatile SecureRandom workerRNG = null;
+
+  public static final String encryptionConfigMismatchMessagePrefix =
+      "Cannot create encryption manager - detected mismatch between client encryption properties and table " +
+          "encryption properties. Mismatch can be caused by changing encryption properties after table creation " +
+          "(currently not supported), or by malicious tampering with table properties kept in metadata.json in " +
+          "storage. Mismatch details: ";
+
+  /**
+   * @param nativeFormatEncryption true for native encryption inside file formats (Parquet or ORC), false for generic
+   *                               encryption of file streams
+   * @param dataEncryptionConfig configuration for encryption of data files. Future versions will add configuration for
+   *                             encryption of metadata files
+   * @param kmsClient Client of KMS used to wrap/unwrap keys in envelope encryption
+   * @param dataKeyLength If DEKs are not generated by KMS, they will be randomly generated by Iceberg using this length
+   */
+  public EnvelopeEncryptionManager(boolean nativeFormatEncryption,  EnvelopeConfiguration dataEncryptionConfig,
+      KmsClient kmsClient, int dataKeyLength) {
+    if (!nativeFormatEncryption) {
+      throw new UnsupportedOperationException("EnvelopeEncryptionManager currently only supports encryption " +
+          "provided by the underlying file format.");
+    }
+    this.nativeFormatEncryption = nativeFormatEncryption;
+    Preconditions.checkNotNull(dataEncryptionConfig,
+        "Cannot create EnvelopeEncryptionManager because data encryption config is not passed");
+    this.dataEncryptionConfig = dataEncryptionConfig;
+    Preconditions.checkNotNull(dataEncryptionConfig.kekId(),
+        "Cannot create EnvelopeEncryptionManager because table key encryption key ID is not specified");
+    Preconditions.checkNotNull(kmsClient,
+        "Cannot create EnvelopeEncryptionManager because KmsClient is null");
+    this.kmsClient = kmsClient;
+    this.dataKeyLength = dataKeyLength;
+    this.kmsGeneratedKeys = kmsClient.supportsKeyGeneration();
+  }
+
+  @Override
+  public EncryptedOutputFile encrypt(OutputFile rawOutput) {
+    EnvelopeMetadata metadata = generateEnvelopeMetadata(dataEncryptionConfig);
+
+    if (nativeFormatEncryption) {
+      NativeFileCryptoParameters nativeEncryptParams = NativeFileCryptoParameters.create(metadata.dek())
+          .encryptionAlgorithm(metadata.algorithm())
+          .build();
+
+      if (!(rawOutput instanceof NativelyEncryptedFile)) {
+        throw new RuntimeException("Can't natively encrypt " + rawOutput.location() + " because the class " +
+            rawOutput.getClass() + " doesn't implement NativelyEncryptedFile interface");
+      }
+
+      ((NativelyEncryptedFile) rawOutput).setNativeCryptoParameters(nativeEncryptParams);
+
+      return new BaseEncryptedOutputFile(rawOutput, metadata);
+    } else {
+      throw new UnsupportedOperationException("EnvelopeEncryptionManager currently supports only file formats which " +
+          "have native encryption implementations");
+    }
+  }
+
+  @Override
+  public InputFile decrypt(EncryptedInputFile encrypted) {
+    if (encrypted.keyMetadata().buffer() == null) { // unencrypted file
+      return encrypted.encryptedInputFile();
+    }
+    EnvelopeMetadata metadata = EnvelopeMetadataParser.fromJson(encrypted.keyMetadata().buffer());
+    ByteBuffer fileDek = kmsClient.unwrapKey(metadata.wrappedDek(), metadata.kekId());
+
+    if (nativeFormatEncryption) {
+      // Pushdown to data formats with native decryption support (such as Parquet)
+      NativeFileCryptoParameters nativeDecryptParams = NativeFileCryptoParameters.create(fileDek).build();
+      InputFile rawInput = encrypted.encryptedInputFile();
+
+      if (!(rawInput instanceof NativelyEncryptedFile)) {
+        throw new RuntimeException("Can't natively decrypt " + rawInput.location() + " because the class " +
+            rawInput.getClass() + " doesn't implement NativelyEncryptedFile interface");
+      }
+
+      ((NativelyEncryptedFile) rawInput).setNativeCryptoParameters(nativeDecryptParams);
+
+      return rawInput;
+    } else {
+      throw new UnsupportedOperationException("EnvelopeEncryptionManager currently supports only file formats which " +
+          "have native encryption implementations");
+    }
+  }
+
+  private EnvelopeMetadata generateEnvelopeMetadata(EnvelopeConfiguration config) {
+    String tableKekID = config.kekId();
+    byte[] fileDek;
+    String wrappedFileDEK;
+
+    if (kmsGeneratedKeys) {
+      KmsClient.KeyGenerationResult generatedDek = kmsClient.generateKey(tableKekID);
+      fileDek = generatedDek.key().array();
+      wrappedFileDEK = generatedDek.wrappedKey();

Review Comment:
   nit: `fileDek` but `wrappedFileDEK `, different style.



##########
core/src/main/java/org/apache/iceberg/encryption/EnvelopeEncryptionManager.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.nio.ByteBuffer;
+import java.security.SecureRandom;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Encryption manager which in conjunction with a KMS can encrypt {@link OutputFile} and decrypt
+ * {@link InputFile}. Envelope encryption uses a key wrapping strategy, where a Key Encryption Key (KEK) is used to
+ * wrap or unwrap a Data Encryption Key DEK which is used to encrypt the underlying files.
+ *
+ * When generating new DEKs for OutputFiles, this class will first attempt to have the KMS generate a new key. If the
+ * KMS does not support key generation a new key will be produced by pulling bytes from a {@link SecureRandom} on the
+ * JVM writing the file.
+ */
+public class EnvelopeEncryptionManager implements EncryptionManager {
+  private final EnvelopeConfiguration dataEncryptionConfig;
+  private final boolean nativeFormatEncryption;
+  private final KmsClient kmsClient;
+  private final int dataKeyLength;
+  private final boolean kmsGeneratedKeys;
+
+  private transient volatile SecureRandom workerRNG = null;
+
+  public static final String encryptionConfigMismatchMessagePrefix =
+      "Cannot create encryption manager - detected mismatch between client encryption properties and table " +
+          "encryption properties. Mismatch can be caused by changing encryption properties after table creation " +
+          "(currently not supported), or by malicious tampering with table properties kept in metadata.json in " +
+          "storage. Mismatch details: ";
+
+  /**
+   * @param nativeFormatEncryption true for native encryption inside file formats (Parquet or ORC), false for generic
+   *                               encryption of file streams
+   * @param dataEncryptionConfig configuration for encryption of data files. Future versions will add configuration for
+   *                             encryption of metadata files
+   * @param kmsClient Client of KMS used to wrap/unwrap keys in envelope encryption
+   * @param dataKeyLength If DEKs are not generated by KMS, they will be randomly generated by Iceberg using this length
+   */
+  public EnvelopeEncryptionManager(boolean nativeFormatEncryption,  EnvelopeConfiguration dataEncryptionConfig,

Review Comment:
   nit: an extra space after `nativeFormatEncryption`.



##########
core/src/main/java/org/apache/iceberg/encryption/EnvelopeMetadataParser.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class EnvelopeMetadataParser {
+
+  private EnvelopeMetadataParser() {
+  }
+
+  private static final String KEK_ID = "kek-id";
+  private static final String WRAPPED_DEK = "wrapped-dek";
+  private static final String ALGORITHM = "algorithm";
+
+  public static void toJson(EnvelopeMetadata metadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeStringField(KEK_ID, metadata.kekId());
+    generator.writeStringField(WRAPPED_DEK, metadata.wrappedDek());
+    JsonUtil.writeStringIf(null != metadata.algorithm(), ALGORITHM, metadata.algorithm().name(), generator);
+
+    generator.writeEndObject();
+  }
+
+  public static ByteBuffer toJson(EnvelopeMetadata metadata) {
+    try {
+      ByteArrayOutputStream stream = new ByteArrayOutputStream();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(stream);

Review Comment:
   JsonGenerator is a Closeable, does it need to be closed 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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


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