You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "wjones127 (via GitHub)" <gi...@apache.org> on 2023/02/17 20:12:21 UTC

[GitHub] [arrow] wjones127 commented on a diff in pull request #34181: GH-25986: [C++] Enable external material and rotation for encryption keys

wjones127 commented on code in PR #34181:
URL: https://github.com/apache/arrow/pull/34181#discussion_r1110136687


##########
cpp/src/parquet/encryption/key_wrapping_test.cc:
##########
@@ -27,16 +31,40 @@ namespace encryption {
 namespace test {
 
 class KeyWrappingTest : public ::testing::Test {
- public:
+ protected:
+  std::unique_ptr<TemporaryDir> temp_dir_;
+
   void SetUp() {
     key_list_ = BuildKeyMap(kColumnMasterKeyIds, kColumnMasterKeys, kFooterMasterKeyId,
                             kFooterMasterKey);
+    temp_dir_ = temp_data_dir().ValueOrDie();
   }
 
- protected:
-  void WrapThenUnwrap(std::shared_ptr<FileKeyMaterialStore> key_material_store,
-                      bool double_wrapping, bool is_wrap_locally) {
+  void WrapThenUnwrap(bool internal_key_material, bool double_wrapping,
+                      bool is_wrap_locally) {
     double cache_entry_lifetime_seconds = 600;
+    std::shared_ptr<FileKeyMaterialStore> key_material_store;
+    std::shared_ptr<::arrow::fs::FileSystem> file_system = nullptr;
+    std::string file_name;
+    if (internal_key_material) {
+      key_material_store = nullptr;
+    } else {
+      file_name += double_wrapping ? "double_wrapping" : "no_double_wrapping";
+      file_name += is_wrap_locally ? "-wrap_locally" : "-wrap_on_server";
+      file_name +=
+          internal_key_material ? "-internal_key_material" : "-external_key_metrial";

Review Comment:
   ```suggestion
             internal_key_material ? "-internal_key_material" : "-external_key_material";
   ```



##########
cpp/src/parquet/encryption/file_system_key_material_store.h:
##########
@@ -0,0 +1,91 @@
+// 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.
+
+#pragma once
+
+#include <set>
+#include <string>
+#include <unordered_map>
+
+#include "arrow/filesystem/filesystem.h"
+
+#include "parquet/encryption/file_key_material_store.h"
+
+namespace parquet {
+namespace encryption {
+
+/// A FileKeyMaterialStore that stores key material in a file system file in the same
+/// folder as the Parquet file.
+class PARQUET_EXPORT FileSystemKeyMaterialStore : public FileKeyMaterialStore {
+ public:
+  static constexpr const char kKeyMaterialFilePrefix[] = "_KEY_MATERIAL_FOR_";
+  static constexpr const char kTempFilePrefix[] = "_TMP";
+  static constexpr const char kKeyMaterialFileSuffix[] = ".json";
+
+  FileSystemKeyMaterialStore() {}
+  FileSystemKeyMaterialStore(const std::string& key_material_file_path,
+                             const std::shared_ptr<::arrow::fs::FileSystem>& file_system);
+
+  /// Creates a new file system key material store for a parquet file.
+  /// When temp_store is true, files are saved with an extra _TMP prefix so they don't
+  /// conflict with existing external material files. This is useful during key rotation
+  /// so that temporary key material files can be created while using the existing key
+  /// material, before moving the key material to the non-temporary location.
+  static std::shared_ptr<FileSystemKeyMaterialStore> Make(
+      const std::string& parquet_file_path,
+      const std::shared_ptr<::arrow::fs::FileSystem>& file_system, bool temp_store);

Review Comment:
   Perhaps `use_tmp_prefix` is a more accurate name?
   
   ```suggestion
         const std::shared_ptr<::arrow::fs::FileSystem>& file_system, bool use_tmp_prefix);
   ```



##########
cpp/src/parquet/encryption/key_management_test.cc:
##########
@@ -122,45 +132,87 @@ class TestEncryptionKeyManagement : public ::testing::Test {
         ARROW_LOG(FATAL) << "Invalid encryption_no";
     }
 
+    encryption.internal_key_material = internal_key_material;
     return encryption;
   }
 
   DecryptionConfiguration GetDecryptionConfiguration() {
     return DecryptionConfiguration();
   }
 
-  void WriteEncryptedParquetFile(bool double_wrapping, int encryption_no) {
-    std::string file_name = GetFileName(double_wrapping, wrap_locally_, encryption_no);
-    auto encryption_config = GetEncryptionConfiguration(double_wrapping, encryption_no);
-
-    auto file_encryption_properties = crypto_factory_.GetFileEncryptionProperties(
-        kms_connection_config_, encryption_config);
-    std::string file = temp_dir->path().ToString() + file_name;
-
-    encryptor_.EncryptFile(file, file_encryption_properties);
+  void WriteEncryptedParquetFile(bool double_wrapping, bool internal_key_material,
+                                 int encryption_no) {
+    std::string file_name =
+        GetFileName(double_wrapping, wrap_locally_, internal_key_material, encryption_no);
+
+    auto encryption_config =
+        GetEncryptionConfiguration(double_wrapping, internal_key_material, encryption_no);
+
+    std::string file = temp_dir_->path().ToString() + file_name;
+    if (internal_key_material) {
+      auto file_encryption_properties = crypto_factory_.GetFileEncryptionProperties(
+          kms_connection_config_, encryption_config);
+      encryptor_.EncryptFile(file, file_encryption_properties);
+    } else {
+      auto file_system = std::make_shared<::arrow::fs::LocalFileSystem>();
+      std::string file_path = temp_dir_->path().ToString() + file_name;

Review Comment:
   This also the same thing as `file` above?



##########
cpp/src/parquet/encryption/file_key_material_store.h:
##########
@@ -17,15 +17,43 @@
 
 #pragma once
 
+#include <set>
+#include <string>
+#include <unordered_map>
+
+#include "arrow/filesystem/filesystem.h"
+#include "parquet/platform.h"
+
 namespace parquet {
 namespace encryption {
 
-// Key material can be stored outside the Parquet file, for example in a separate small
-// file in the same folder. This is important for “key rotation”, when MEKs have to be
-// changed (if compromised; or periodically, just in case) - without modifying the Parquet
-// files (often  immutable).
-// TODO: details will be implemented later
-class FileKeyMaterialStore {};
+/// Stores encryption key material outside the Parquet file, for example in a separate
+/// small file in the same folder. This is important for “key rotation”, when MEKs have to
+/// be changed (if compromised; or periodically, just in case) - without modifying the
+/// Parquet files (often  immutable).
+class PARQUET_EXPORT FileKeyMaterialStore {
+ public:
+  /// Add key material for one encryption key.
+  virtual void AddKeyMaterial(std::string key_id_in_file, std::string key_material) = 0;
+
+  /// Get key material
+  virtual std::string GetKeyMaterial(std::string key_id_in_file) = 0;
+
+  /// After key material was added for all keys in the given Parquet file,
+  /// save material in persistent store.
+  virtual void SaveMaterial() = 0;
+
+  /// Remove key material from persistent store. Used in key rotation.
+  virtual void RemoveMaterial() = 0;
+
+  /// Move key material to another store. Used in key rotation.
+  virtual void MoveMaterialTo(std::shared_ptr<FileKeyMaterialStore> target_key_store) = 0;
+
+  ///  Returns the Set of all key IDs in this store (for the given Parquet file)

Review Comment:
   ```suggestion
     /// Returns the Set of all key IDs in this store (for the given Parquet file)
   ```



##########
cpp/src/parquet/encryption/file_key_unwrapper.h:
##########
@@ -44,14 +43,21 @@ class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
   /// key_toolkit and kms_connection_config is to get KmsClient from cache or create
   /// KmsClient if it's not in the cache yet. cache_entry_lifetime_seconds is life time of
   /// KmsClient in the cache.
+  /// If the file uses external key material then either the Parquet file path and file
+  /// system must be specified, or a key material store provided.
   FileKeyUnwrapper(KeyToolkit* key_toolkit,
                    const KmsConnectionConfig& kms_connection_config,
-                   double cache_lifetime_seconds);
+                   double cache_lifetime_seconds, const std::string& file_path,
+                   const std::shared_ptr<::arrow::fs::FileSystem>& file_system = NULLPTR,
+                   std::shared_ptr<FileKeyMaterialStore> key_material_store = NULLPTR);

Review Comment:
   Could these be two different overloads perhaps?



##########
cpp/src/parquet/encryption/key_management_test.cc:
##########
@@ -22,55 +22,64 @@
 #include <thread>
 #include <unordered_map>
 
+#include <arrow/io/file.h>
+#include "arrow/filesystem/filesystem.h"
+#include "arrow/filesystem/localfs.h"
+#include "arrow/status.h"
+#include "arrow/testing/gtest_util.h"
 #include "arrow/testing/util.h"
 #include "arrow/util/logging.h"
 
 #include "parquet/encryption/crypto_factory.h"
 #include "parquet/encryption/key_toolkit.h"
 #include "parquet/encryption/test_encryption_util.h"
 #include "parquet/encryption/test_in_memory_kms.h"
+#include "parquet/file_reader.h"
 #include "parquet/test_util.h"
 
 namespace parquet {
 namespace encryption {
 namespace test {
 
-std::unique_ptr<TemporaryDir> temp_dir;
-
 class TestEncryptionKeyManagement : public ::testing::Test {
- public:
-  void SetUp() {
-#ifndef ARROW_WITH_SNAPPY
-    GTEST_SKIP() << "Test requires Snappy compression";
-#endif
-    key_list_ = BuildKeyMap(kColumnMasterKeyIds, kColumnMasterKeys, kFooterMasterKeyId,
-                            kFooterMasterKey);
-    column_key_mapping_ = BuildColumnKeyMapping();
-  }
-
-  static void SetUpTestCase();
-
  protected:
+  std::unique_ptr<TemporaryDir> temp_dir_;
   FileEncryptor encryptor_;
   FileDecryptor decryptor_;
 
   std::unordered_map<std::string, std::string> key_list_;
+  std::unordered_map<std::string, std::string> new_key_list_;
   std::string column_key_mapping_;
   KmsConnectionConfig kms_connection_config_;
   CryptoFactory crypto_factory_;
   bool wrap_locally_;
 
+  void SetUp() {
+#ifndef ARROW_WITH_SNAPPY
+    GTEST_SKIP() << "Test requires Snappy compression";
+#endif
+    key_list_ = BuildKeyMap(kColumnMasterKeyIds, kColumnMasterKeys, kFooterMasterKeyId,
+                            kFooterMasterKey);
+    new_key_list_ = BuildKeyMap(kColumnMasterKeyIds, kNewColumnMasterKeys,
+                                kFooterMasterKeyId, kNewFooterMasterKey);
+    column_key_mapping_ = BuildColumnKeyMapping();
+    temp_dir_ = temp_data_dir().ValueOrDie();
+  }
+
   void SetupCryptoFactory(bool wrap_locally) {
     wrap_locally_ = wrap_locally;
     std::shared_ptr<KmsClientFactory> kms_client_factory =
         std::make_shared<TestOnlyInMemoryKmsClientFactory>(wrap_locally, key_list_);
     crypto_factory_.RegisterKmsClientFactory(kms_client_factory);
   }
 
-  std::string GetFileName(bool double_wrapping, bool wrap_locally, int encryption_no) {
+  std::string GetFileName(bool double_wrapping, bool wrap_locally,
+                          bool internal_key_material, int encryption_no) {
     std::string file_name;
     file_name += double_wrapping ? "double_wrapping" : "no_double_wrapping";
     file_name += wrap_locally ? "-wrap_locally" : "-wrap_on_server";
+    file_name +=
+        internal_key_material ? "-internal_key_material" : "-external_key_metrial";

Review Comment:
   ```suggestion
           internal_key_material ? "-internal_key_material" : "-external_key_material";
   ```



##########
cpp/src/parquet/encryption/key_management_test.cc:
##########
@@ -122,45 +132,87 @@ class TestEncryptionKeyManagement : public ::testing::Test {
         ARROW_LOG(FATAL) << "Invalid encryption_no";
     }
 
+    encryption.internal_key_material = internal_key_material;
     return encryption;
   }
 
   DecryptionConfiguration GetDecryptionConfiguration() {
     return DecryptionConfiguration();
   }
 
-  void WriteEncryptedParquetFile(bool double_wrapping, int encryption_no) {
-    std::string file_name = GetFileName(double_wrapping, wrap_locally_, encryption_no);
-    auto encryption_config = GetEncryptionConfiguration(double_wrapping, encryption_no);
-
-    auto file_encryption_properties = crypto_factory_.GetFileEncryptionProperties(
-        kms_connection_config_, encryption_config);
-    std::string file = temp_dir->path().ToString() + file_name;
-
-    encryptor_.EncryptFile(file, file_encryption_properties);
+  void WriteEncryptedParquetFile(bool double_wrapping, bool internal_key_material,
+                                 int encryption_no) {
+    std::string file_name =
+        GetFileName(double_wrapping, wrap_locally_, internal_key_material, encryption_no);
+
+    auto encryption_config =
+        GetEncryptionConfiguration(double_wrapping, internal_key_material, encryption_no);
+
+    std::string file = temp_dir_->path().ToString() + file_name;
+    if (internal_key_material) {
+      auto file_encryption_properties = crypto_factory_.GetFileEncryptionProperties(
+          kms_connection_config_, encryption_config);
+      encryptor_.EncryptFile(file, file_encryption_properties);
+    } else {
+      auto file_system = std::make_shared<::arrow::fs::LocalFileSystem>();
+      std::string file_path = temp_dir_->path().ToString() + file_name;
+      auto file_encryption_properties = crypto_factory_.GetFileEncryptionProperties(
+          kms_connection_config_, encryption_config, file_path, file_system);
+      encryptor_.EncryptFile(file, file_encryption_properties);
+    }
   }
 
-  void ReadEncryptedParquetFile(bool double_wrapping, int encryption_no) {
+  void ReadEncryptedParquetFile(bool double_wrapping, bool internal_key_material,
+                                int encryption_no) {
     auto decryption_config = GetDecryptionConfiguration();
-    std::string file_name = GetFileName(double_wrapping, wrap_locally_, encryption_no);
-
-    auto file_decryption_properties = crypto_factory_.GetFileDecryptionProperties(
-        kms_connection_config_, decryption_config);
-    std::string file = temp_dir->path().ToString() + file_name;
+    std::string file_name =
+        GetFileName(double_wrapping, wrap_locally_, internal_key_material, encryption_no);
+    std::string file = temp_dir_->path().ToString() + file_name;
+    if (internal_key_material) {
+      auto file_decryption_properties = crypto_factory_.GetFileDecryptionProperties(
+          kms_connection_config_, decryption_config);
+
+      decryptor_.DecryptFile(file, file_decryption_properties);
+    } else {
+      auto file_system = std::make_shared<::arrow::fs::LocalFileSystem>();
+      std::string file_path = temp_dir_->path().ToString() +
+                              GetFileName(double_wrapping, wrap_locally_,
+                                          internal_key_material, encryption_no);

Review Comment:
   Isn't this the same as `file` 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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