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

[GitHub] [arrow] adamreeve opened a new pull request, #34181: GH-25986: [C++] Enable external material and rotation for encryption keys

adamreeve opened a new pull request, #34181:
URL: https://github.com/apache/arrow/pull/34181

   This PR is a replacement for #10491 which appears to be abandoned. I've fixed the issues pointed out in review comments as well as a few more things I noticed. I've also made one significant change to the API; rather than `CryptoFactory::RotateMasterKeys` operating on a whole directory, it rotates keys for a single file, as this is much more flexible and allows users to decide what files to rotate keys for, and there didn't appear to be a good reason for this to work at the whole directory level. This does mean the API diverges slightly from the parquet-mr API though.
   
   ### Rationale for this change
   
   Use of external key material allows rotating master encryption keys without having to rewrite Parquet file data. See https://docs.google.com/document/d/1bEu903840yb95k9q2X-BlsYKuXoygE4VnMDl9xz_zhk/edit?usp=sharing for more details.
   
   ### What changes are included in this PR?
   
   Adds support for writing and reading external key material for Parquet files from C++, as well as a new `CryptoFactory::RotateMasterKeys` function that allows re-encrypting key encryption keys or data encryption keys with latest versions of master keys.
   
   ### Are these changes tested?
   
   Yes, unit tests are included. I've added an additional test that reads a file generated with parquet-mr from the parquet-testing repository. This requires merging the PR at https://github.com/apache/parquet-testing/pull/36 and updating the parquet-testing submodule before the new test will pass.
   
   ### Are there any user-facing changes?
   
   Yes, the existing `internal_key_material` option in `parquet::encryption::EncryptionConfiguration` will now work and use external key material. This requires using two new parameters (`file_path` and `file_system`) in `CryptoFactory::GetFileEncryptionProperties` and `CryptoFactory::GetFileDecryptionProperties`, which are needed so that we know where to write/read the external key material. Note that this means external key material won't work from Python until the new parameters are exposed in Python too.
   
   This changes the `CryptoFactory` ABI but the API is still source compatible.
   
   The `CryptoFactory::RotateMasterKeys` function is also a new public facing API.


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


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

Posted by "wjones127 (via GitHub)" <gi...@apache.org>.
wjones127 commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1439163077

   > Thanks for the feedback @wjones127, I've addressed all of these points except for the one about using overloads as I think it might be more important to have overloads on the CryptoFactory methods instead. I'm not sure how important ABI stability is though, should anything not in an internal namespace use overloads when adding parameters to avoid breaking ABI compatibility?
   
   We don't aim for ABI stability as much as maintaining usability right now. But we do try to maintain a sort of API stability for C++ code when we can, including adding deprecation messages to functions that will be removed in future versions.
   
   I think overloads seem appropriate for the methods that take _either_ `KeyMaterialStore` or file path + `FileSystem`. It seems weird to have one function that expects only one of either parameters to be passed.


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


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

Posted by "ursabot (via GitHub)" <gi...@apache.org>.
ursabot commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1448044045

   Benchmark runs are scheduled for baseline = c2e8f253e6b675a7caed61449b0c720f675911a1 and contender = 33ed1dbbf799bb2df01c1fe70b867acecca247f1. 33ed1dbbf799bb2df01c1fe70b867acecca247f1 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/e2caed8699bb4a2891789ea1498b0794...852725ca07dc4d178fa25797e09777aa/)
   [Failed :arrow_down:2.02% :arrow_up:0.0%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/98febf8b60b44eb3a0a633ab12682036...6652ae063e2c4ff38b04e64c0eacd289/)
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/11c0fb9d571e42689c790a3589d582ef...82201db3242342a28b2d119e65502e8e/)
   [Finished :arrow_down:0.06% :arrow_up:0.03%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/84b120c3b30e491ca94b3532de53549c...274ce726a96e4e7bba188d76601764c8/)
   Buildkite builds:
   [Finished] [`33ed1dbb` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2445)
   [Finished] [`33ed1dbb` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2475)
   [Finished] [`33ed1dbb` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2442)
   [Finished] [`33ed1dbb` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2466)
   [Finished] [`c2e8f253` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2444)
   [Failed] [`c2e8f253` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2474)
   [Finished] [`c2e8f253` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2441)
   [Finished] [`c2e8f253` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2465)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


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


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

Posted by "adamreeve (via GitHub)" <gi...@apache.org>.
adamreeve commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1436089135

   Thanks for the feedback @wjones127, I've addressed all of these points except for the one about using overloads as I think it might be more important to have overloads on the `CryptoFactory` methods instead. I'm not sure how important ABI stability is though, should anything not in an internal namespace use overloads when adding parameters to avoid breaking ABI compatibility?
   
   I've also updated the parquet-testing submodule now that https://github.com/apache/parquet-testing/pull/36 is merged, so all tests should pass :crossed_fingers: 


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


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

Posted by "wjones127 (via GitHub)" <gi...@apache.org>.
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


[GitHub] [arrow] github-actions[bot] commented on pull request #34181: GH-25986: [C++] Enable external material and rotation for encryption keys

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1430356497

   :warning: GitHub issue #25986 **has been automatically assigned in GitHub** to PR creator.


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


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

Posted by "adamreeve (via GitHub)" <gi...@apache.org>.
adamreeve commented on code in PR #34181:
URL: https://github.com/apache/arrow/pull/34181#discussion_r1111311533


##########
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:
   It might make sense for `CryptoFactory::GetFileEncryptionProperties` and `CryptoFactory::GetFileDecryptionProperties` to have separate overloads as they seem closer to the public API that most people will interact with? I'm not so sure about this constructor, it's in the public encryption namespace but does seem closer to the internals, and is only called from `CryptoFactory::GetFileDecryptionProperties` within the arrow source, where we pass all parameters.



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


[GitHub] [arrow] wjones127 merged pull request #34181: GH-25986: [C++] Enable external material and rotation for encryption keys

Posted by "wjones127 (via GitHub)" <gi...@apache.org>.
wjones127 merged PR #34181:
URL: https://github.com/apache/arrow/pull/34181


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


[GitHub] [arrow] github-actions[bot] commented on pull request #34181: GH-25986: [C++] Enable external material and rotation for encryption keys

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1430356460

   * Closes: #25986


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


Re: [PR] GH-25986: [C++] Enable external material and rotation for encryption keys [arrow]

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1992202178

   @adamreeve Thank you very much for this PR. Do you know if there are plans to make this work from Python? (at least the read part)


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


[GitHub] [arrow] github-actions[bot] commented on pull request #34181: GH-25986: [C++] Enable external material and rotation for encryption keys

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1430357186

   :warning: GitHub issue #25986 **has been automatically assigned in GitHub** to PR creator.


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


Re: [PR] GH-25986: [C++] Enable external material and rotation for encryption keys [arrow]

Posted by "adamreeve (via GitHub)" <gi...@apache.org>.
adamreeve commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1993793577

   Hi @ggershinsky, no, I don't have any plans to make this work from Python, and I'm not aware of any. It's a while since I worked on this, but I don't think there would be too much work required, mainly just adding the file path and filesystem arguments to `CryptoFactory.file_decryption_properties` and  `CryptoFactory.file_encryption_properties`.


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


Re: [PR] GH-25986: [C++] Enable external material and rotation for encryption keys [arrow]

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1993876234

   Sounds good, we'll have a look.


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


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

Posted by "adamreeve (via GitHub)" <gi...@apache.org>.
adamreeve commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1439293583

   > I think overloads seem appropriate for the methods that take _either_ `KeyMaterialStore` or file path + `FileSystem`. It seems weird to have one function that expects only one of either parameters to be passed.
   
   Right, sorry I totally missed the point of your comment. I've split this constructor into two overloads now.


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


[GitHub] [arrow] github-actions[bot] commented on pull request #34181: GH-25986: [C++] Enable external material and rotation for encryption keys

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34181:
URL: https://github.com/apache/arrow/pull/34181#issuecomment-1430421350

   :warning: GitHub issue #25986 **has been automatically assigned in GitHub** to PR creator.


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


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

Posted by "adamreeve (via GitHub)" <gi...@apache.org>.
adamreeve commented on code in PR #34181:
URL: https://github.com/apache/arrow/pull/34181#discussion_r1111311533


##########
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:
   It might make sense for `CryptoFactory::GetFileEncryptionProperties` and `CryptoFactory::GetFileDecryptionProperties` to have separate overloads as they are part of the public API that most people will interact with? I'm not so sure about this constructor, it's in the public encryption namespace but does seem closer to the internals, and all parameters are used in the places where it's called in the Arrow source.



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