You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/08/22 07:43:47 UTC

[GitHub] [arrow] thamht4190 opened a new pull request #8023: Arrow 9318 encryption key management

thamht4190 opened a new pull request #8023:
URL: https://github.com/apache/arrow/pull/8023


   This PR is C++ implementation for parquet key tool, based on [the Java implementation](https://github.com/apache/parquet-mr/pull/615) and [the design doc](https://docs.google.com/document/d/1boH6HPkG0ZhgxcaRkGk3QpZ8X_J91uXZwVGwYN45St4/edit?usp=sharing).
   
   The major parts of this PR are:
   * higher level of encryption/decryption configuration, including key management configuration.
   * KMS connection configuration
   * Abstract class KmsClient, KmsClientFactory
   * PropertiesDrivenCryptoFactory class to convert these above configurations to the lower level FileEncryptionProperties, FileDecryptionProperties
   * unit test using InMemoryKms (an sample of KmsClient).
   
   Comparing to Java version, this C++ pull doesn't contain externally storing key material using hadoop file system (only storing key material internally in parquet file is supported for now). The reason is lack of understanding about Hadoop file system, can be implemented it later in another pull.
   
   Thanks!


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



[GitHub] [arrow] bkietz commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-702082359


   @thamht4190 thanks for your flexibility!


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



[GitHub] [arrow] pitrou commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-796770371


   As for testing, I'd still welcome unit tests with actual pieces of JSON as per the spec, but I won't insist on it here :-)
   
   Last question: do we want to mark those APIs for experimental so that we feel free to change them in the future?


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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497971688



##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)
+      : cached_item_(cached_item) {
+    expiration_timestamp_ =
+        CurrentTimePoint() + std::chrono::milliseconds(expiration_interval_millis);
+  }
+
+  bool IsExpired() {
+    auto now = CurrentTimePoint();
+    return (now > expiration_timestamp_);
+  }
+
+  E cached_item() { return cached_item_; }
+
+ private:
+  TimePoint expiration_timestamp_;
+  E cached_item_;
+};
+
+// This class is to avoid the below warning when compiling KeyToolkit class with VS2015
+// warning C4503: decorated name length exceeded, name was truncated
+template <typename V>
+class ExpiringCacheMapEntry {
+ public:
+  ExpiringCacheMapEntry() = default;
+
+  explicit ExpiringCacheMapEntry(std::shared_ptr<ConcurrentMap<V>> cached_item,
+                                 uint64_t expiration_interval_millis)
+      : map_cache_(cached_item, expiration_interval_millis) {}
+
+  bool IsExpired() { return map_cache_.IsExpired(); }
+
+  std::shared_ptr<ConcurrentMap<V>> cached_item() { return map_cache_.cached_item(); }
+
+ private:
+  ExpiringCacheEntry<std::shared_ptr<ConcurrentMap<V>>> map_cache_;

Review comment:
       The lock in `TwoLevelCacheWithExpiration` is for the outside map `cache_` and also `last_cache_cleanup_timestamp_`. While this `ConcurrentMap` can be accessed and modified outside `TwoLevelCacheWithExpiration`, for example: https://github.com/apache/arrow/pull/8023/files#diff-0006a54749b9fee8573f28939bab6eb0R76




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497999593



##########
File path: cpp/src/parquet/encryption/key_metadata.h
##########
@@ -0,0 +1,92 @@
+// 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 <string>
+
+#include "arrow/util/variant.h"
+
+#include "parquet/encryption/key_material.h"
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer. The "key metadata" field is made available to file readers
+// to enable recovery of the key. This interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows integration with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details). This class is implemented to support version 1 of the parquet key
+// management tools specification.
+//
+// KeyMetadata writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material.
+// 2. "internalStorage" - a boolean. If true, means that "key material" is kept inside the
+// "key metadata" field. If false, "key material" is kept externally (outside Parquet
+// files) - in this case, "key metadata" keeps a reference to the external "key material".
+// 3. "keyReference" - a String, with the reference to the external "key material".
+// Written only if internalStorage is false.
+//
+// If internalStorage is true, "key material" is a part of "key metadata", and the json
+// keeps additional fields, described in the KeyMaterial class.
+class PARQUET_EXPORT KeyMetadata {
+ public:
+  static constexpr const char kKeyMaterialInternalStorageField[] = "internalStorage";
+  static constexpr const char kKeyReferenceField[] = "keyReference";

Review comment:
       These are used in `key_material.cc`




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486400053



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;

Review comment:
       Eventually I think 2 above ways are identical. The user have to keep an object alive along the time of their program and these both needs to be well documented.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486397802



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       I guess the purpose to put `EncryptKeyLocally`/`DecryptKeyLocally` method in `KeyToolkit` is that user can make use of this function to encrypt/decrypt the key locally without writing the code again. An example: We have used these methods in `TestOnlyInMemoryKms` class. So it's not really internal stuff. cc @ggershinsky 




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498078569



##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED
+  fields.push_back(PrimitiveNode::Make(kBooleanFieldName, Repetition::REQUIRED,
+                                       Type::BOOLEAN, ConvertedType::NONE));
+
+  // Create a primitive node named 'int32_field' with type:INT32, repetition:REQUIRED,
+  // logical type:TIME_MILLIS
+  fields.push_back(PrimitiveNode::Make(kInt32FieldName, Repetition::REQUIRED, Type::INT32,
+                                       ConvertedType::TIME_MILLIS));
+
+  // Create a primitive node named 'int64_field' with type:INT64, repetition:REPEATED
+  fields.push_back(PrimitiveNode::Make(kInt64FieldName, Repetition::REPEATED, Type::INT64,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kInt96FieldName, Repetition::REQUIRED, Type::INT96,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kFloatFieldName, Repetition::REQUIRED, Type::FLOAT,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kDoubleFieldName, Repetition::REQUIRED,
+                                       Type::DOUBLE, ConvertedType::NONE));
+
+  // Create a primitive node named 'ba_field' with type:BYTE_ARRAY, repetition:OPTIONAL
+  fields.push_back(PrimitiveNode::Make(kByteArrayFieldName, Repetition::OPTIONAL,
+                                       Type::BYTE_ARRAY, ConvertedType::NONE));
+
+  // Create a primitive node named 'flba_field' with type:FIXED_LEN_BYTE_ARRAY,
+  // repetition:REQUIRED, field_length = kFixedLength
+  fields.push_back(PrimitiveNode::Make(kFixedLenByteArrayFieldName, Repetition::REQUIRED,
+                                       Type::FIXED_LEN_BYTE_ARRAY, ConvertedType::NONE,
+                                       kFixedLength));
+
+  // Create a GroupNode named 'schema' using the primitive nodes defined above
+  // This GroupNode is the root node of the schema tree
+  return std::static_pointer_cast<GroupNode>(
+      GroupNode::Make("schema", Repetition::REQUIRED, fields));
+}
+
+void FileEncryptor::EncryptFile(
+    std::string file,
+    std::shared_ptr<parquet::FileEncryptionProperties> encryption_configurations) {
+  WriterProperties::Builder prop_builder;
+  prop_builder.compression(parquet::Compression::SNAPPY);
+  prop_builder.encryption(encryption_configurations);
+  std::shared_ptr<WriterProperties> writer_properties = prop_builder.build();
+
+  PARQUET_ASSIGN_OR_THROW(auto out_file, FileClass::Open(file));
+  // Create a ParquetFileWriter instance
+  std::shared_ptr<parquet::ParquetFileWriter> file_writer =
+      parquet::ParquetFileWriter::Open(out_file, schema_, writer_properties);
+
+  for (int r = 0; r < num_rgs; r++) {
+    bool buffered_mode = r % 2 == 0;
+    auto row_group_writer = buffered_mode ? file_writer->AppendBufferedRowGroup()
+                                          : file_writer->AppendRowGroup();
+
+    int column_index = 0;
+    // Captures i by reference; increments it by one
+    auto get_next_column = [&]() {
+      return buffered_mode ? row_group_writer->column(column_index++)
+                           : row_group_writer->NextColumn();
+    };
+
+    // Write the Bool column
+    parquet::BoolWriter* bool_writer =
+        static_cast<parquet::BoolWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      bool value = ((i % 2) == 0) ? true : false;
+      bool_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Int32 column
+    parquet::Int32Writer* int32_writer =
+        static_cast<parquet::Int32Writer*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      int32_t value = i;
+      int32_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Int64 column. Each row has repeats twice.
+    parquet::Int64Writer* int64_writer =
+        static_cast<parquet::Int64Writer*>(get_next_column());
+    for (int i = 0; i < 2 * rows_per_rowgroup_; i++) {
+      int64_t value = i * 1000 * 1000;
+      value *= 1000 * 1000;
+      int16_t definition_level = 1;
+      int16_t repetition_level = 0;
+      if ((i % 2) == 0) {
+        repetition_level = 1;  // start of a new record
+      }
+      int64_writer->WriteBatch(1, &definition_level, &repetition_level, &value);
+    }
+
+    // Write the INT96 column.
+    parquet::Int96Writer* int96_writer =
+        static_cast<parquet::Int96Writer*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::Int96 value;
+      value.value[0] = i;
+      value.value[1] = i + 1;
+      value.value[2] = i + 2;
+      int96_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Float column
+    parquet::FloatWriter* float_writer =
+        static_cast<parquet::FloatWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      float value = static_cast<float>(i) * 1.1f;
+      float_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Double column
+    parquet::DoubleWriter* double_writer =
+        static_cast<parquet::DoubleWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      double value = i * 1.1111111;
+      double_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the ByteArray column. Make every alternate values NULL
+    parquet::ByteArrayWriter* ba_writer =
+        static_cast<parquet::ByteArrayWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::ByteArray value;
+      char hello[kFixedLength] = "parquet";
+      hello[7] = static_cast<char>(static_cast<int>('0') + i / 100);
+      hello[8] = static_cast<char>(static_cast<int>('0') + (i / 10) % 10);
+      hello[9] = static_cast<char>(static_cast<int>('0') + i % 10);
+      if (i % 2 == 0) {
+        int16_t definition_level = 1;
+        value.ptr = reinterpret_cast<const uint8_t*>(&hello[0]);
+        value.len = kFixedLength;
+        ba_writer->WriteBatch(1, &definition_level, nullptr, &value);
+      } else {
+        int16_t definition_level = 0;
+        ba_writer->WriteBatch(1, &definition_level, nullptr, nullptr);
+      }
+    }
+
+    // Write the FixedLengthByteArray column
+    parquet::FixedLenByteArrayWriter* flba_writer =
+        static_cast<parquet::FixedLenByteArrayWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::FixedLenByteArray value;
+      char v = static_cast<char>(i);
+      char flba[kFixedLength] = {v, v, v, v, v, v, v, v, v, v};
+      value.ptr = reinterpret_cast<const uint8_t*>(&flba[0]);
+      flba_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+  }
+
+  // Close the ParquetFileWriter
+  file_writer->Close();
+
+  return;
+}
+
+void FileDecryptor::DecryptFile(
+    std::string file,
+    std::shared_ptr<FileDecryptionProperties> file_decryption_properties) {
+  std::string exception_msg;
+  parquet::ReaderProperties reader_properties = parquet::default_reader_properties();
+  if (file_decryption_properties) {
+    reader_properties.file_decryption_properties(file_decryption_properties->DeepClone());
+  }
+
+  auto file_reader = parquet::ParquetFileReader::OpenFile(file, false, reader_properties);
+
+  // Get the File MetaData
+  std::shared_ptr<parquet::FileMetaData> file_metadata = file_reader->metadata();
+
+  // Get the number of RowGroups
+  int num_row_groups = file_metadata->num_row_groups();
+
+  // Get the number of Columns
+  int num_columns = file_metadata->num_columns();
+  ASSERT_EQ(num_columns, 8);
+
+  // Iterate over all the RowGroups in the file
+  for (int r = 0; r < num_row_groups; ++r) {
+    // Get the RowGroup Reader
+    std::shared_ptr<parquet::RowGroupReader> row_group_reader = file_reader->RowGroup(r);
+
+    // Get the RowGroupMetaData
+    std::unique_ptr<RowGroupMetaData> rg_metadata = file_metadata->RowGroup(r);
+
+    int64_t values_read = 0;
+    int64_t rows_read = 0;
+    int16_t definition_level;
+    int16_t repetition_level;
+    int i;
+    std::shared_ptr<parquet::ColumnReader> column_reader;
+
+    // Get the Column Reader for the boolean column
+    column_reader = row_group_reader->Column(0);
+    parquet::BoolReader* bool_reader =
+        static_cast<parquet::BoolReader*>(column_reader.get());
+
+    // Get the ColumnChunkMetaData for the boolean column
+    std::unique_ptr<ColumnChunkMetaData> boolean_md = rg_metadata->ColumnChunk(0);
+
+    // Read all the rows in the column
+    i = 0;
+    while (bool_reader->HasNext()) {
+      bool value;
+      // Read one value at a time. The number of rows read is returned. values_read
+      // contains the number of non-null rows
+      rows_read = bool_reader->ReadBatch(1, nullptr, nullptr, &value, &values_read);
+      // Ensure only one value is read
+      ASSERT_EQ(rows_read, 1);
+      // There are no NULL values in the rows written
+      ASSERT_EQ(values_read, 1);
+      // Verify the value written
+      bool expected_value = ((i % 2) == 0) ? true : false;
+      ASSERT_EQ(value, expected_value);
+      i++;
+    }
+    // make sure we got the same number of values the metadata says
+    ASSERT_EQ(boolean_md->num_values(), i);
+
+    // Get the Column Reader for the Int32 column
+    column_reader = row_group_reader->Column(1);
+    parquet::Int32Reader* int32_reader =
+        static_cast<parquet::Int32Reader*>(column_reader.get());
+
+    // Get the ColumnChunkMetaData for the Int32 column
+    std::unique_ptr<ColumnChunkMetaData> int32_md = rg_metadata->ColumnChunk(1);
+
+    // Read all the rows in the column

Review comment:
       Ah, sorry, I hadn't noticed it was just moved around. But thanks for looking at 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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;

Review comment:
       +1 from me. I mean, the builder pattern guarantees the object will be immutable; while with a struct it's easier for a developer to overlook unintended changes after the object is created. But if Arrow / parquet-cpp doesn't use the builder pattern, no reason to start using it now. This is one of the places where we already have a difference between Java and C++ encryption APIs - due to the substantial language differences.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486380916



##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       @ggershinsky any suggestion for the comment?




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498079983



##########
File path: cpp/src/parquet/encryption/file_key_unwrapper.h
##########
@@ -0,0 +1,66 @@
+// 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 "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/key_material.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class will retrieve the key from "key metadata", following these steps:
+// 1. Parse "key metadata" (see structure in KeyMetadata class).
+// 2. Retrieve "key material" which can be stored inside or outside "key metadata"
+//    Currently we don't support the case "key material" stores outside "key metadata"
+//    yet.
+// 3. Unwrap the "data encryption key" from "key material". There are 2 modes:
+// 3.1. single wrapping: decrypt the wrapped "data encryption key" directly with "master
+// encryption key" 3.2. double wrapping: 2 steps: 3.2.1. "key encryption key" is decrypted
+// with "master encryption key" 3.2.2. "data encryption key" is decrypted with the above
+// "key encryption key"
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  /// 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.
+  FileKeyUnwrapper(KeyToolkit* key_toolkit,

Review comment:
       I see, thanks.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,111 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT RemoteKmsClient : public KmsClient {
+ public:
+  static constexpr const char kLocalWrapNoKeyVersion[] = "NO_VERSION";
+
+  void Initialize(const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally);

Review comment:
       Since this is an API function, could you please add a comment above it; something like
   "/// If is_wrap_locally is true, master keys will be fetched from the KMS server and used to encrypt other keys (DEKs or KEKs).
   /// If is_wrap_locally is false, keys will be encrypted inside KMS server. This is the preferred mode, since it is safer and also allows for key rotation.

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,223 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Wrap keys locally - master keys are fetched from the KMS server and used to
+    /// encrypt other keys (DEKs or KEKs).
+    Builder* wrap_locally(bool wrap_locally);

Review comment:
       I think we need to remove this option. We won't be able to encrypt some of the files with local wrapping, and other files with remote wrapping.

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,223 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Wrap keys locally - master keys are fetched from the KMS server and used to
+    /// encrypt other keys (DEKs or KEKs).
+    Builder* wrap_locally(bool wrap_locally);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    bool wrap_locally_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  bool wrap_locally() const { return wrap_locally_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, bool wrap_locally,
+                          uint64_t cache_lifetime_seconds, bool internal_key_material,
+                          bool uniform_encryption, int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        wrap_locally_(wrap_locally),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+ private:
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  bool wrap_locally_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder()
+        : wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Wrap keys locally - master keys are fetched from the KMS server and used to
+    /// encrypt other keys (DEKs or KEKs).
+    Builder* wrap_locally(bool wrap_locally);

Review comment:
       As before (we need to remove this option. We won't be able to decrypt some of the files with local wrapping, and other files with remote wrapping.)

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,223 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Wrap keys locally - master keys are fetched from the KMS server and used to
+    /// encrypt other keys (DEKs or KEKs).
+    Builder* wrap_locally(bool wrap_locally);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    bool wrap_locally_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  bool wrap_locally() const { return wrap_locally_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, bool wrap_locally,
+                          uint64_t cache_lifetime_seconds, bool internal_key_material,
+                          bool uniform_encryption, int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        wrap_locally_(wrap_locally),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+ private:
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  bool wrap_locally_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder()
+        : wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Wrap keys locally - master keys are fetched from the KMS server and used to
+    /// encrypt other keys (DEKs or KEKs).
+    Builder* wrap_locally(bool wrap_locally);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    bool wrap_locally_;

Review comment:
       As before (we need to remove this option. We won't be able to decrypt some of the files with local wrapping, and other files with remote wrapping.)

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,223 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Wrap keys locally - master keys are fetched from the KMS server and used to
+    /// encrypt other keys (DEKs or KEKs).
+    Builder* wrap_locally(bool wrap_locally);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    bool wrap_locally_;

Review comment:
       As before (we need to remove this option. We won't be able to encrypt some of the files with local wrapping, and other files with remote wrapping.)

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,223 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Wrap keys locally - master keys are fetched from the KMS server and used to
+    /// encrypt other keys (DEKs or KEKs).
+    Builder* wrap_locally(bool wrap_locally);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    bool wrap_locally_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  bool wrap_locally() const { return wrap_locally_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, bool wrap_locally,
+                          uint64_t cache_lifetime_seconds, bool internal_key_material,
+                          bool uniform_encryption, int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        wrap_locally_(wrap_locally),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+ private:
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  bool wrap_locally_;

Review comment:
       As before (we need to remove this option. We won't be able to encrypt some of the files with local wrapping, and other files with remote wrapping.)

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,223 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Wrap keys locally - master keys are fetched from the KMS server and used to
+    /// encrypt other keys (DEKs or KEKs).
+    Builder* wrap_locally(bool wrap_locally);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    bool wrap_locally_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  bool wrap_locally() const { return wrap_locally_; }

Review comment:
       As before (we need to remove this option. We won't be able to encrypt some of the files with local wrapping, and other files with remote wrapping.)




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498326446



##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {

Review comment:
       @pitrou Yes, in this case, only the map is accessed by multiple threads.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497994581



##########
File path: cpp/src/parquet/encryption/file_key_unwrapper.h
##########
@@ -0,0 +1,66 @@
+// 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 "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/key_material.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class will retrieve the key from "key metadata", following these steps:
+// 1. Parse "key metadata" (see structure in KeyMetadata class).
+// 2. Retrieve "key material" which can be stored inside or outside "key metadata"
+//    Currently we don't support the case "key material" stores outside "key metadata"
+//    yet.
+// 3. Unwrap the "data encryption key" from "key material". There are 2 modes:
+// 3.1. single wrapping: decrypt the wrapped "data encryption key" directly with "master
+// encryption key" 3.2. double wrapping: 2 steps: 3.2.1. "key encryption key" is decrypted
+// with "master encryption key" 3.2.2. "data encryption key" is decrypted with the above
+// "key encryption key"
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  /// 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.
+  FileKeyUnwrapper(KeyToolkit* key_toolkit,

Review comment:
       It references to `PropertiesDrivenCryptoFactory::key_toolkit_` which should remain alive during with `PropertiesDrivenCryptoFactory` object. `FileKeyUnwrapper` is used internally in `PropertiesDrivenCryptoFactory`, so I think it's ok to pass a raw pointer.




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r485046887



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;

Review comment:
       Also, this implementation may not be return an actual singleton, as that method is inlinable?




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



[GitHub] [arrow] emkornfield commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r475337736



##########
File path: cpp/src/parquet/encryption.h
##########
@@ -47,15 +47,15 @@ using ColumnPathToEncryptionPropertiesMap =
 
 class PARQUET_EXPORT DecryptionKeyRetriever {
  public:
-  virtual std::string GetKey(const std::string& key_metadata) const = 0;

Review comment:
       Why are you removing const?  

##########
File path: cpp/src/parquet/file_key_material_store.h
##########
@@ -0,0 +1,51 @@
+// 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") = 0; 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>
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyMaterialStore {
+ public:
+  // Initializes key material store for a parquet file.
+  virtual void Initialize(const std::string& parquet_file_path, bool temp_store) = 0;
+
+  virtual void AddKeyMaterial(const std::string& key_id_in_file,

Review comment:
       is it an error to repeat a key?  please add documentation.

##########
File path: cpp/src/parquet/key_encryption_key.h
##########
@@ -0,0 +1,57 @@
+// 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 <cstdint>
+#include <vector>
+
+#include "arrow/util/base64.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyEncryptionKey {
+ public:
+  KeyEncryptionKey() = default;

Review comment:
       given the base64 encoding done in the other constructor, should this really be default?  Is it ok for all the fields to be empty?

##########
File path: cpp/src/parquet/file_key_unwrapper.h
##########
@@ -0,0 +1,65 @@
+// 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 "parquet/encryption.h"
+#include "parquet/key_material.h"
+#include "parquet/key_toolkit.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  FileKeyUnwrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                   const KmsConnectionConfig& kms_connection_config,
+                   uint64_t cache_lifetime, bool is_wrap_locally);
+
+  std::string GetKey(const std::string& key_metadata) override;
+
+ private:
+  class KeyWithMasterID {

Review comment:
       ```suggestion
     class KeyWithMasterId {
   ```
   The code base generally uses camel case even for acronyms.

##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());

Review comment:
       I thought there was a document parser that takes a string pointer and a size, that is probably better to use if it is available.

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers

Review comment:
       ```suggestion
   // storage in the file footer. The "key metadata" field is made available to file readers
   ```

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,

Review comment:
       ```suggestion
   // KeyMetadata writes (and reads) the "key metadata" field as a flat json object,
   ```

##########
File path: cpp/src/parquet/file_key_material_store.h
##########
@@ -0,0 +1,51 @@
+// 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") = 0; 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>
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyMaterialStore {
+ public:
+  // Initializes key material store for a parquet file.

Review comment:
       For headers we use doxygen triple slash comments "///"

##########
File path: cpp/src/parquet/file_key_material_store.h
##########
@@ -0,0 +1,51 @@
+// 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") = 0; 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>
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyMaterialStore {

Review comment:
       needs documentation.

##########
File path: cpp/src/parquet/encryption_key_management_test.cc
##########
@@ -0,0 +1,238 @@
+// 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.
+
+#include <gtest/gtest.h>
+
+#include <iostream>
+#include <string>
+
+#include "arrow/testing/util.h"
+
+#include "parquet/in_memory_kms.h"
+#include "parquet/key_toolkit.h"
+#include "parquet/properties_driven_crypto_factory.h"
+#include "parquet/test_encryption_util.h"
+#include "parquet/test_util.h"
+
+namespace parquet {
+
+namespace test {
+
+using encryption::DecryptionConfiguration;
+using encryption::EncryptionConfiguration;
+using encryption::InMemoryKmsClientFactory;
+using encryption::KeyAccessToken;
+using encryption::KeyToolkit;
+using encryption::KmsClient;
+using encryption::KmsClientFactory;
+using encryption::KmsConnectionConfig;
+using encryption::PropertiesDrivenCryptoFactory;
+
+const char FOOTER_MASTER_KEY[] = "0123456789112345";
+const char* const COLUMN_MASTER_KEYS[] = {"1234567890123450", "1234567890123451",
+                                          "1234567890123452", "1234567890123453",
+                                          "1234567890123454", "1234567890123455"};
+const char* const COLUMN_MASTER_KEY_IDS[] = {"kc1", "kc2", "kc3", "kc4", "kc5", "kc6"};
+const char FOOTER_MASTER_KEY_ID[] = "kf";
+
+std::map<std::string, std::string> BuildKeyMap(const char* const* column_ids,
+                                               const char* const* column_keys,
+                                               const char* footer_id,
+                                               const char* footer_key) {
+  std::map<std::string, std::string> key_map;
+  // add column keys
+  for (int i = 0; i < 6; i++) {
+    key_map.insert({column_ids[i], column_keys[i]});
+  }
+  // add footer key
+  key_map.insert({footer_id, footer_key});
+
+  return key_map;
+}
+
+std::string BuildColumnKeyMapping() {
+  std::ostringstream stream;
+  stream << COLUMN_MASTER_KEY_IDS[0] << ":" << DOUBLE_FIELD_NAME << ";"
+         << COLUMN_MASTER_KEY_IDS[1] << ":" << FLOAT_FIELD_NAME << ";"
+         << COLUMN_MASTER_KEY_IDS[2] << ":" << BOOLEAN_FIELD_NAME << ";"
+         << COLUMN_MASTER_KEY_IDS[3] << ":" << INT32_FIELD_NAME << ";"
+         << COLUMN_MASTER_KEY_IDS[4] << ":" << BA_FIELD_NAME << ";"
+         << COLUMN_MASTER_KEY_IDS[5] << ":" << FLBA_FIELD_NAME << ";";
+  return stream.str();
+}
+
+class TestEncrytionKeyManagement : public ::testing::Test {
+ public:
+  void SetUp() {
+    key_list_ = BuildKeyMap(COLUMN_MASTER_KEY_IDS, COLUMN_MASTER_KEYS,
+                            FOOTER_MASTER_KEY_ID, FOOTER_MASTER_KEY);
+    column_key_mapping_ = BuildColumnKeyMapping();
+
+    kms_connection_config_.refreshable_key_access_token =
+        std::make_shared<KeyAccessToken>();
+  }
+
+ protected:
+  FileEncryptor encryptor_;
+  FileDecryptor decryptor_;
+
+  std::map<std::string, std::string> key_list_;
+  std::string column_key_mapping_;
+  KmsConnectionConfig kms_connection_config_;
+
+  void SetupCryptoFactory(PropertiesDrivenCryptoFactory& crypto_factory,
+                          bool wrap_locally) {
+    KeyToolkit::RemoveCacheEntriesForAllTokens();
+    std::shared_ptr<KmsClientFactory> kms_client_factory =
+        std::make_shared<InMemoryKmsClientFactory>(wrap_locally, key_list_);
+    crypto_factory.kms_client_factory(kms_client_factory);
+  }
+
+  std::string GetFileName(bool double_wrapping, bool wrap_locally, int encryption_no) {
+    std::string file_name;
+    file_name += double_wrapping ? "double_wrapping" : "no_double_wrapping";
+    file_name += wrap_locally ? "-wrap_locally" : "-no_wrap_locally";
+    switch (encryption_no) {
+      case 0:
+        file_name += "-encrypt_columns_and_footer_diff_keys";
+        break;
+      case 1:
+        file_name += "-encrypt_columns_not_footer";
+        break;
+      case 2:
+        file_name += "-encrypt_columns_and_footer_same_keys";
+        break;
+      case 3:
+        file_name += "-encrypt_columns_and_footer_ctr";
+        break;
+      default:  // case 4:
+        file_name += "-no_encrypt";
+        break;
+    }
+    file_name += encryption_no == 4 ? ".parquet" : ".parquet.encrypted";
+    return file_name;
+  }
+
+  std::vector<std::shared_ptr<EncryptionConfiguration>> GetEncryptionConfigurations(
+      bool double_wrapping, bool wrap_locally) {
+    std::vector<std::shared_ptr<EncryptionConfiguration>> configs;
+    std::vector<EncryptionConfiguration::Builder*> config_builders;
+
+    // encrypt some columns and footer, different keys
+    EncryptionConfiguration::Builder builder1(FOOTER_MASTER_KEY_ID);
+    builder1.column_keys(column_key_mapping_);
+    config_builders.push_back(&builder1);
+
+    // encrypt columns, plaintext footer, different keys
+    EncryptionConfiguration::Builder builder2(FOOTER_MASTER_KEY_ID);
+    builder2.column_keys(column_key_mapping_)->plaintext_footer(true);
+    config_builders.push_back(&builder2);
+
+    // encrypt some columns and footer, same key
+    EncryptionConfiguration::Builder builder3(FOOTER_MASTER_KEY_ID);
+    builder3.uniform_encryption();
+    config_builders.push_back(&builder3);
+
+    // Encrypt two columns and the footer, with different keys.
+    // Use AES_GCM_CTR_V1 algorithm.
+    EncryptionConfiguration::Builder builder4(FOOTER_MASTER_KEY_ID);
+    builder4.column_keys(column_key_mapping_)
+        ->encryption_algorithm(ParquetCipher::AES_GCM_CTR_V1);
+    config_builders.push_back(&builder4);
+
+    for (EncryptionConfiguration::Builder* builder : config_builders) {
+      auto config =
+          builder->double_wrapping(double_wrapping)->wrap_locally(wrap_locally)->build();
+      configs.push_back(config);
+    }
+
+    // non encryption
+    configs.push_back(NULL);
+
+    return configs;
+  }
+
+  std::shared_ptr<DecryptionConfiguration> GetDecryptionConfiguration(bool wrap_locally) {
+    DecryptionConfiguration::Builder builder;
+    return builder.wrap_locally(wrap_locally)->build();
+  }
+
+  void WriteEncryptedParquetFiles() {
+    for (int i = 0; i < 2; i++) {
+      bool wrap_locally = (i == 0);
+
+      PropertiesDrivenCryptoFactory crypto_factory;
+      SetupCryptoFactory(crypto_factory, wrap_locally);
+
+      for (int j = 0; j < 2; j++) {
+        bool double_wrapping = (j == 0);
+        auto encryption_configs =
+            this->GetEncryptionConfigurations(double_wrapping, wrap_locally);
+        for (size_t encryption_no = 0; encryption_no < encryption_configs.size();
+             encryption_no++) {
+          std::string file_name =
+              GetFileName(double_wrapping, wrap_locally, encryption_no);
+          std::cout << "Writing file: " << file_name << std::endl;
+
+          auto encryption_config = encryption_configs[encryption_no];
+          std::shared_ptr<FileEncryptionProperties> file_encryption_properties =
+              crypto_factory.GetFileEncryptionProperties(kms_connection_config_,
+                                                         encryption_config);
+
+          std::string file = data_file(file_name.c_str());
+          encryptor_.EncryptFile(file, file_encryption_properties);
+        }
+      }
+    }
+  }
+
+  void ReadEncryptedParquetFiles() {
+    for (int i = 0; i < 2; i++) {
+      bool wrap_locally = (i == 0);
+
+      PropertiesDrivenCryptoFactory crypto_factory;
+      SetupCryptoFactory(crypto_factory, wrap_locally);
+
+      auto decryption_config = this->GetDecryptionConfiguration(wrap_locally);
+      for (int j = 0; j < 2; j++) {
+        bool double_wrapping = (j == 0);
+        for (size_t encryption_no = 0; encryption_no < 5; encryption_no++) {
+          std::string file_name =
+              GetFileName(double_wrapping, wrap_locally, encryption_no);
+          std::cout << "Reading file: " << file_name << std::endl;
+
+          std::shared_ptr<FileDecryptionProperties> file_decryption_properties =
+              crypto_factory.GetFileDecryptionProperties(kms_connection_config_,
+                                                         decryption_config);
+
+          std::string file = data_file(file_name.c_str());
+          decryptor_.DecryptFile(file, file_decryption_properties);
+        }
+      }
+    }
+  }
+};
+
+TEST_F(TestEncrytionKeyManagement, TestWriteReadEncryptedParquetFiles) {

Review comment:
       It would be better to structure this as a parameterized test or have a separate test case for each type of encryption.  If there is also a need to verify different types of files can be written together (shared state) this should be its own explicit test.

##########
File path: cpp/src/parquet/file_key_material_store.h
##########
@@ -0,0 +1,51 @@
+// 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") = 0; 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>
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyMaterialStore {
+ public:
+  // Initializes key material store for a parquet file.
+  virtual void Initialize(const std::string& parquet_file_path, bool temp_store) = 0;

Review comment:
       nit: places where `const std::string&` are used as parameters should probably  use arrow::util::string_view instead.

##########
File path: cpp/src/parquet/file_key_material_store.h
##########
@@ -0,0 +1,51 @@
+// 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") = 0; 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>
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyMaterialStore {
+ public:
+  // Initializes key material store for a parquet file.
+  virtual void Initialize(const std::string& parquet_file_path, bool temp_store) = 0;
+
+  virtual void AddKeyMaterial(const std::string& key_id_in_file,
+                              const std::string& key_material) = 0;
+
+  // After key material was added for all keys in the given Parquet file,
+  // save material in persistent store.
+  virtual void SaveMaterial() = 0;
+
+  virtual std::string GetKeyMaterial(const std::string& key_id_in_file) = 0;
+
+  // return set of all key IDs in this store (for the given Parquet file)

Review comment:
       ```suggestion
     /// Returns a set of all key IDs in this store (for the given Parquet file).
   ```

##########
File path: cpp/src/parquet/kms_client.h
##########
@@ -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.
+
+#pragma once
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;
+  std::unordered_map<std::string, std::string> custom_kms_conf;
+
+  const std::string& key_access_token() const {
+    if (refreshable_key_access_token == NULL ||
+        refreshable_key_access_token->value().empty()) {
+      throw ParquetException("key access token is not set!");
+    }
+    return refreshable_key_access_token->value();
+  }
+
+  void SetDefaultIfEmpty();
+};
+
+class PARQUET_EXPORT KmsClient {
+ public:
+  static constexpr char KMS_INSTANCE_ID_DEFAULT[] = "DEFAULT";
+  static constexpr char KMS_INSTANCE_URL_DEFAULT[] = "DEFAULT";
+  static constexpr char KEY_ACCESS_TOKEN_DEFAULT[] = "DEFAULT";
+
+  // Wraps a key - encrypts it with the master key, encodes the result
+  // and potentially adds a KMS-specific metadata.
+  virtual std::string WrapKey(const std::string& key_bytes,
+                              const std::string& master_key_identifier) = 0;

Review comment:
       It might be useful to wrap master_key_identifier in a struct, to make it clear that is is an ID and not the actual key.

##########
File path: cpp/src/parquet/file_key_material_store.h
##########
@@ -0,0 +1,51 @@
+// 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") = 0; 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>
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyMaterialStore {
+ public:
+  // Initializes key material store for a parquet file.
+  virtual void Initialize(const std::string& parquet_file_path, bool temp_store) = 0;
+
+  virtual void AddKeyMaterial(const std::string& key_id_in_file,
+                              const std::string& key_material) = 0;
+
+  // After key material was added for all keys in the given Parquet file,
+  // save material in persistent store.
+  virtual void SaveMaterial() = 0;
+
+  virtual std::string GetKeyMaterial(const std::string& key_id_in_file) = 0;
+
+  // return set of all key IDs in this store (for the given Parquet file)
+  virtual std::set<std::string> GetKeyIdSet() = 0;

Review comment:
       do you want set or [unordered_set](https://en.cppreference.com/w/cpp/container/unordered_set)  same question where map is used.

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,

Review comment:
       docs please.

##########
File path: cpp/src/parquet/file_key_unwrapper.cc
##########
@@ -0,0 +1,107 @@
+// 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.
+
+#include <iostream>
+
+#include "parquet/file_key_unwrapper.h"
+#include "parquet/key_metadata.h"
+#include "parquet/key_toolkit.h"
+
+namespace parquet {
+namespace encryption {
+
+FileKeyUnwrapper::FileKeyUnwrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                                   const KmsConnectionConfig& kms_connection_config,
+                                   uint64_t cache_lifetime, bool is_wrap_locally)

Review comment:
       its not really clear what this value represents [duration](https://en.cppreference.com/w/cpp/chrono/duration) might be something to consider here.  
   
   If Duration isn't used, use int64_t and spell out the units in the variable name.

##########
File path: cpp/src/parquet/file_key_material_store.h
##########
@@ -0,0 +1,51 @@
+// 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") = 0; 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>
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyMaterialStore {
+ public:
+  // Initializes key material store for a parquet file.
+  virtual void Initialize(const std::string& parquet_file_path, bool temp_store) = 0;

Review comment:
       Also this seems to imply that the MaterialStore isn't thread-safe.  Would a better pattern to have this be a factory method that returns something a like a:
   ```
   FileKeyMaterialStore?
   ```

##########
File path: cpp/src/parquet/file_key_unwrapper.h
##########
@@ -0,0 +1,65 @@
+// 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 "parquet/encryption.h"
+#include "parquet/key_material.h"
+#include "parquet/key_toolkit.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  FileKeyUnwrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,

Review comment:
       is there a reason to use shared_ptr here?  Or would unique_ptr be more appropriate?

##########
File path: cpp/src/parquet/file_key_unwrapper.h
##########
@@ -0,0 +1,65 @@
+// 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 "parquet/encryption.h"
+#include "parquet/key_material.h"
+#include "parquet/key_toolkit.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  FileKeyUnwrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                   const KmsConnectionConfig& kms_connection_config,
+                   uint64_t cache_lifetime, bool is_wrap_locally);
+
+  std::string GetKey(const std::string& key_metadata) override;
+
+ private:
+  class KeyWithMasterID {
+   public:
+    KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+        : key_bytes_(key_bytes), master_id_(master_id) {}
+
+    const std::string& data_key() const { return key_bytes_; }
+    const std::string& master_id() const { return master_id_; }
+
+   private:
+    std::string key_bytes_;
+    std::string master_id_;
+  };
+
+  KeyWithMasterID GetDEKandMasterID(const KeyMaterial& key_material);

Review comment:
       I think spelling out DEK here makes sense.  Not everyone will have these acronym on the top of there mind and a google search doesn't list it as a top result.  If the name gets too long I  think dropping andMasterId would be fine (considerer doing it anyways, since it is implied by the return type).

##########
File path: cpp/src/parquet/file_key_unwrapper.h
##########
@@ -0,0 +1,65 @@
+// 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 "parquet/encryption.h"
+#include "parquet/key_material.h"
+#include "parquet/key_toolkit.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  FileKeyUnwrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,

Review comment:
       please document parameters.

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;

Review comment:
       preferred style is kKekLength.  I think it would be better to spell out KEK

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key, std::string key_id_in_file);
+
+ private:
+  KeyEncryptionKey CreateKeyEncryptionKey(const std::string& master_key_id);
+
+  // A map of MEK_ID -> KeyEncryptionKey, for the current token
+  std::map<std::string, KeyEncryptionKey> kek_per_master_key_id_;

Review comment:
       map versus unordered map?

##########
File path: cpp/src/parquet/file_key_unwrapper.h
##########
@@ -0,0 +1,65 @@
+// 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 "parquet/encryption.h"
+#include "parquet/key_material.h"
+#include "parquet/key_toolkit.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {

Review comment:
       needs documentation.

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,

Review comment:
       doc please.  also how does this differ then the method below.

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;

Review comment:
       spell out RND please.

##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());
+
+  if (document.HasParseError() || !document.IsObject()) {
+    throw ParquetException("Failed to parse key metadata " + key_material_string);
+  }
+
+  // External key material - extract "key material type", and make sure it is supported
+  std::string key_material_type = document[KEY_MATERIAL_TYPE_FIELD].GetString();
+  if (KEY_MATERIAL_TYPE1 != key_material_type) {
+    throw ParquetException("Wrong key material type: " + key_material_type + " vs " +
+                           KEY_MATERIAL_TYPE1);
+  }
+  // Parse other fields (common to internal and external key material)
+  return Parse(document);
+}
+
+KeyMaterial KeyMaterial::Parse(const rapidjson::Document& key_material_json) {
+  // 2. Check if "key material" belongs to file footer key
+  bool is_footer_key = key_material_json[IS_FOOTER_KEY_FIELD].GetBool();
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  if (is_footer_key) {
+    // 3.  For footer key, extract KMS Instance ID
+    kms_instance_id = key_material_json[KMS_INSTANCE_ID_FIELD].GetString();

Review comment:
       i haven't used rapidjson much, what happens if this field doesn't exist or it is not a string type?

##########
File path: cpp/src/parquet/file_key_unwrapper.h
##########
@@ -0,0 +1,65 @@
+// 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 "parquet/encryption.h"
+#include "parquet/key_material.h"
+#include "parquet/key_toolkit.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  FileKeyUnwrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                   const KmsConnectionConfig& kms_connection_config,
+                   uint64_t cache_lifetime, bool is_wrap_locally);
+
+  std::string GetKey(const std::string& key_metadata) override;
+
+ private:
+  class KeyWithMasterID {
+   public:
+    KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+        : key_bytes_(key_bytes), master_id_(master_id) {}
+
+    const std::string& data_key() const { return key_bytes_; }
+    const std::string& master_id() const { return master_id_; }
+
+   private:
+    std::string key_bytes_;
+    std::string master_id_;
+  };
+
+  KeyWithMasterID GetDEKandMasterID(const KeyMaterial& key_material);
+  std::shared_ptr<KmsClient> GetKmsClientFromConfigOrKeyMaterial(
+      const KeyMaterial& key_material);
+
+  // A map of KEK_ID -> KEK bytes, for the current token
+  std::map<std::string, std::string> kek_per_kek_id_;

Review comment:
       unordered_map? 

##########
File path: cpp/src/parquet/file_key_unwrapper.h
##########
@@ -0,0 +1,65 @@
+// 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 "parquet/encryption.h"
+#include "parquet/key_material.h"
+#include "parquet/key_toolkit.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  FileKeyUnwrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                   const KmsConnectionConfig& kms_connection_config,
+                   uint64_t cache_lifetime, bool is_wrap_locally);
+
+  std::string GetKey(const std::string& key_metadata) override;
+
+ private:
+  class KeyWithMasterID {
+   public:
+    KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+        : key_bytes_(key_bytes), master_id_(master_id) {}
+
+    const std::string& data_key() const { return key_bytes_; }

Review comment:
       given this is a private class is there a reason to not use a struct and to make key_bytes and master_id constant?

##########
File path: cpp/src/parquet/in_memory_kms.h
##########
@@ -0,0 +1,70 @@
+// 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 "arrow/util/base64.h"
+
+#include "parquet/key_toolkit.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/remote_kms_client.h"
+#include "parquet/string_util.h"
+
+namespace parquet {
+namespace encryption {
+
+// This is a mock class, built for testing only. Don't use it as an example of KmsClient

Review comment:
       maybe name the file test_in_memory_kms?  and the class TestOnlyInmemoryKms?

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,

Review comment:
       it isn't clear what the contents of string is.  Should this be wrapped in a struct or have an alias name to better communicate what is returned?

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key, std::string key_id_in_file);
+
+ private:
+  KeyEncryptionKey CreateKeyEncryptionKey(const std::string& master_key_id);
+
+  // A map of MEK_ID -> KeyEncryptionKey, for the current token

Review comment:
       spell out MEK.

##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());
+
+  if (document.HasParseError() || !document.IsObject()) {
+    throw ParquetException("Failed to parse key metadata " + key_material_string);
+  }
+
+  // External key material - extract "key material type", and make sure it is supported
+  std::string key_material_type = document[KEY_MATERIAL_TYPE_FIELD].GetString();
+  if (KEY_MATERIAL_TYPE1 != key_material_type) {
+    throw ParquetException("Wrong key material type: " + key_material_type + " vs " +
+                           KEY_MATERIAL_TYPE1);
+  }
+  // Parse other fields (common to internal and external key material)
+  return Parse(document);
+}
+
+KeyMaterial KeyMaterial::Parse(const rapidjson::Document& key_material_json) {
+  // 2. Check if "key material" belongs to file footer key
+  bool is_footer_key = key_material_json[IS_FOOTER_KEY_FIELD].GetBool();
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  if (is_footer_key) {
+    // 3.  For footer key, extract KMS Instance ID
+    kms_instance_id = key_material_json[KMS_INSTANCE_ID_FIELD].GetString();
+    // 4.  For footer key, extract KMS Instance URL
+    kms_instance_url = key_material_json[KMS_INSTANCE_URL_FIELD].GetString();
+  }
+  // 5. Extract master key ID
+  std::string master_key_id = key_material_json[MASTER_KEY_ID_FIELD].GetString();
+  // 6. Extract wrapped DEK
+  std::string encoded_wrapped_dek = key_material_json[WRAPPED_DEK_FIELD].GetString();
+  std::string kek_id;
+  std::string encoded_wrapped_kek;
+  // 7. Check if "key material" was generated in double wrapping mode
+  bool is_double_wrapped = key_material_json[DOUBLE_WRAPPING_FIELD].GetBool();
+  if (is_double_wrapped) {
+    // 8. In double wrapping mode, extract KEK ID
+    kek_id = key_material_json[KEK_ID_FIELD].GetString();
+    // 9. In double wrapping mode, extract wrapped KEK
+    encoded_wrapped_kek = key_material_json[WRAPPED_KEK_FIELD].GetString();
+  }
+
+  return KeyMaterial(is_footer_key, kms_instance_id, kms_instance_url, master_key_id,
+                     is_double_wrapped, kek_id, encoded_wrapped_kek, encoded_wrapped_dek);
+}
+
+std::string KeyMaterial::CreateSerialized(
+    bool is_footer_key, const std::string& kms_instance_id,
+    const std::string& kms_instance_url, const std::string& master_key_id,
+    bool is_double_wrapped, const std::string& kek_id,
+    const std::string& encoded_wrapped_kek, const std::string& encoded_wrapped_dek,
+    bool is_internal_storage) {
+  rapidjson::Document d;
+  auto& allocator = d.GetAllocator();

Review comment:
       please spell out types.  Generally auto should only be used if the type is already available on the same line (or very close by).

##########
File path: cpp/src/parquet/key_encryption_key.h
##########
@@ -0,0 +1,57 @@
+// 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 <cstdint>
+#include <vector>
+
+#include "arrow/util/base64.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyEncryptionKey {

Review comment:
       docs please.

##########
File path: cpp/src/parquet/key_encryption_key.h
##########
@@ -0,0 +1,57 @@
+// 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 <cstdint>
+#include <vector>
+
+#include "arrow/util/base64.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyEncryptionKey {
+ public:
+  KeyEncryptionKey() = default;
+
+  KeyEncryptionKey(const std::string& kek_bytes, const std::string& kek_id,
+                   const std::string& encoded_wrapped_kek)
+      : kek_bytes_(kek_bytes),
+        kek_id_(kek_id),
+        encoded_wrapped_kek_(encoded_wrapped_kek) {
+    encoded_kek_id_ = arrow::util::base64_encode(reinterpret_cast<uint8_t*>(&kek_id_[0]),

Review comment:
       kek_id_.data().

##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());
+
+  if (document.HasParseError() || !document.IsObject()) {
+    throw ParquetException("Failed to parse key metadata " + key_material_string);
+  }
+
+  // External key material - extract "key material type", and make sure it is supported
+  std::string key_material_type = document[KEY_MATERIAL_TYPE_FIELD].GetString();
+  if (KEY_MATERIAL_TYPE1 != key_material_type) {
+    throw ParquetException("Wrong key material type: " + key_material_type + " vs " +
+                           KEY_MATERIAL_TYPE1);
+  }
+  // Parse other fields (common to internal and external key material)
+  return Parse(document);
+}
+
+KeyMaterial KeyMaterial::Parse(const rapidjson::Document& key_material_json) {
+  // 2. Check if "key material" belongs to file footer key
+  bool is_footer_key = key_material_json[IS_FOOTER_KEY_FIELD].GetBool();

Review comment:
       what happens if this isn't a boolean field?

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key, std::string key_id_in_file);
+
+ private:
+  KeyEncryptionKey CreateKeyEncryptionKey(const std::string& master_key_id);
+
+  // A map of MEK_ID -> KeyEncryptionKey, for the current token
+  std::map<std::string, KeyEncryptionKey> kek_per_master_key_id_;
+
+  std::shared_ptr<KmsClient> kms_client_;

Review comment:
       i'd guess at least a few of these should be const?

##########
File path: cpp/src/parquet/key_material.h
##########
@@ -0,0 +1,120 @@
+// 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 <map>
+#include <string>
+
+#include <rapidjson/document.h>
+
+namespace parquet {
+namespace encryption {
+
+// KeyMaterial class represents the "key material", keeping the information that allows
+// readers to recover an encryption key (see description of the KeyMetadata class). The
+// keytools package (PARQUET-1373) implements the "envelope encryption" pattern, in a
+// "single wrapping" or "double wrapping" mode. In the single wrapping mode, the key
+// material is generated by encrypting the "data encryption key" (DEK) by a "master key".
+// In the double wrapping mode, the key material is generated by encrypting the DEK by a
+// "key encryption key" (KEK), that in turn is encrypted by a "master key".
+//
+// Key material is kept in a flat json object, with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current
+// version, only one value is allowed - "PKMT1" (stands
+//     for "parquet key management tools, version 1"). For external key material storage,
+//     this field is written in both "key metadata" and "key material" jsons. For internal
+//     key material storage, this field is written only once in the common json.
+// 2. "isFooterKey" - a boolean. If true, means that the material belongs to a file footer
+// key, and keeps additional information (such as
+//     KMS instance ID and URL). If false, means that the material belongs to a column
+//     key.
+// 3. "kmsInstanceID" - a String, with the KMS Instance ID. Written only in footer key
+// material.
+// 4. "kmsInstanceURL" - a String, with the KMS Instance URL. Written only in footer key
+// material.
+// 5. "masterKeyID" - a String, with the ID of the master key used to generate the
+// material.
+// 6. "wrappedDEK" - a String, with the wrapped DEK (base64 encoding).
+// 7. "doubleWrapping" - a boolean. If true, means that the material was generated in
+// double wrapping mode.
+//     If false - in single wrapping mode.
+// 8. "keyEncryptionKeyID" - a String, with the ID of the KEK used to generate the
+// material. Written only in double wrapping mode.
+// 9. "wrappedKEK" - a String, with the wrapped KEK (base64 encoding). Written only in
+// double wrapping mode.
+class KeyMaterial {
+ public:
+  static constexpr char KEY_MATERIAL_TYPE_FIELD[] = "keyMaterialType";

Review comment:
       kKeyMaterialTypeField

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key, std::string key_id_in_file);
+
+ private:
+  KeyEncryptionKey CreateKeyEncryptionKey(const std::string& master_key_id);
+
+  // A map of MEK_ID -> KeyEncryptionKey, for the current token
+  std::map<std::string, KeyEncryptionKey> kek_per_master_key_id_;
+
+  std::shared_ptr<KmsClient> kms_client_;
+  KmsConnectionConfig kms_connection_config_;
+  std::shared_ptr<FileKeyMaterialStore> key_material_store_;
+  uint64_t cache_entry_lifetime_;
+  bool double_wrapping_;
+  uint16_t key_counter_;

Review comment:
       generally signed types are preferred even if the values can't logically be negative.  why is this limited to 65K values?

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {

Review comment:
       document please.  Does this need to be exported?

##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());
+
+  if (document.HasParseError() || !document.IsObject()) {
+    throw ParquetException("Failed to parse key metadata " + key_material_string);
+  }
+
+  // External key material - extract "key material type", and make sure it is supported
+  std::string key_material_type = document[KEY_MATERIAL_TYPE_FIELD].GetString();
+  if (KEY_MATERIAL_TYPE1 != key_material_type) {
+    throw ParquetException("Wrong key material type: " + key_material_type + " vs " +
+                           KEY_MATERIAL_TYPE1);
+  }
+  // Parse other fields (common to internal and external key material)
+  return Parse(document);
+}
+
+KeyMaterial KeyMaterial::Parse(const rapidjson::Document& key_material_json) {
+  // 2. Check if "key material" belongs to file footer key
+  bool is_footer_key = key_material_json[IS_FOOTER_KEY_FIELD].GetBool();
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  if (is_footer_key) {
+    // 3.  For footer key, extract KMS Instance ID
+    kms_instance_id = key_material_json[KMS_INSTANCE_ID_FIELD].GetString();
+    // 4.  For footer key, extract KMS Instance URL
+    kms_instance_url = key_material_json[KMS_INSTANCE_URL_FIELD].GetString();
+  }
+  // 5. Extract master key ID
+  std::string master_key_id = key_material_json[MASTER_KEY_ID_FIELD].GetString();
+  // 6. Extract wrapped DEK
+  std::string encoded_wrapped_dek = key_material_json[WRAPPED_DEK_FIELD].GetString();
+  std::string kek_id;
+  std::string encoded_wrapped_kek;
+  // 7. Check if "key material" was generated in double wrapping mode
+  bool is_double_wrapped = key_material_json[DOUBLE_WRAPPING_FIELD].GetBool();
+  if (is_double_wrapped) {
+    // 8. In double wrapping mode, extract KEK ID
+    kek_id = key_material_json[KEK_ID_FIELD].GetString();
+    // 9. In double wrapping mode, extract wrapped KEK
+    encoded_wrapped_kek = key_material_json[WRAPPED_KEK_FIELD].GetString();
+  }
+
+  return KeyMaterial(is_footer_key, kms_instance_id, kms_instance_url, master_key_id,
+                     is_double_wrapped, kek_id, encoded_wrapped_kek, encoded_wrapped_dek);
+}
+
+std::string KeyMaterial::CreateSerialized(
+    bool is_footer_key, const std::string& kms_instance_id,

Review comment:
       this seems very easy to mess up, consider creating a struct to pass through the necessary information. 

##########
File path: cpp/src/parquet/key_material.h
##########
@@ -0,0 +1,120 @@
+// 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 <map>
+#include <string>
+
+#include <rapidjson/document.h>
+
+namespace parquet {
+namespace encryption {
+
+// KeyMaterial class represents the "key material", keeping the information that allows
+// readers to recover an encryption key (see description of the KeyMetadata class). The
+// keytools package (PARQUET-1373) implements the "envelope encryption" pattern, in a
+// "single wrapping" or "double wrapping" mode. In the single wrapping mode, the key
+// material is generated by encrypting the "data encryption key" (DEK) by a "master key".
+// In the double wrapping mode, the key material is generated by encrypting the DEK by a
+// "key encryption key" (KEK), that in turn is encrypted by a "master key".
+//
+// Key material is kept in a flat json object, with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current
+// version, only one value is allowed - "PKMT1" (stands
+//     for "parquet key management tools, version 1"). For external key material storage,
+//     this field is written in both "key metadata" and "key material" jsons. For internal
+//     key material storage, this field is written only once in the common json.
+// 2. "isFooterKey" - a boolean. If true, means that the material belongs to a file footer
+// key, and keeps additional information (such as
+//     KMS instance ID and URL). If false, means that the material belongs to a column
+//     key.
+// 3. "kmsInstanceID" - a String, with the KMS Instance ID. Written only in footer key
+// material.
+// 4. "kmsInstanceURL" - a String, with the KMS Instance URL. Written only in footer key
+// material.
+// 5. "masterKeyID" - a String, with the ID of the master key used to generate the
+// material.
+// 6. "wrappedDEK" - a String, with the wrapped DEK (base64 encoding).
+// 7. "doubleWrapping" - a boolean. If true, means that the material was generated in
+// double wrapping mode.
+//     If false - in single wrapping mode.
+// 8. "keyEncryptionKeyID" - a String, with the ID of the KEK used to generate the
+// material. Written only in double wrapping mode.
+// 9. "wrappedKEK" - a String, with the wrapped KEK (base64 encoding). Written only in
+// double wrapping mode.
+class KeyMaterial {
+ public:
+  static constexpr char KEY_MATERIAL_TYPE_FIELD[] = "keyMaterialType";

Review comment:
       you should comment that all these fields are defined in a specification and should never be changed.

##########
File path: cpp/src/parquet/key_material.h
##########
@@ -0,0 +1,120 @@
+// 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 <map>
+#include <string>
+
+#include <rapidjson/document.h>
+
+namespace parquet {
+namespace encryption {
+
+// KeyMaterial class represents the "key material", keeping the information that allows
+// readers to recover an encryption key (see description of the KeyMetadata class). The
+// keytools package (PARQUET-1373) implements the "envelope encryption" pattern, in a
+// "single wrapping" or "double wrapping" mode. In the single wrapping mode, the key
+// material is generated by encrypting the "data encryption key" (DEK) by a "master key".
+// In the double wrapping mode, the key material is generated by encrypting the DEK by a
+// "key encryption key" (KEK), that in turn is encrypted by a "master key".
+//
+// Key material is kept in a flat json object, with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current
+// version, only one value is allowed - "PKMT1" (stands
+//     for "parquet key management tools, version 1"). For external key material storage,
+//     this field is written in both "key metadata" and "key material" jsons. For internal
+//     key material storage, this field is written only once in the common json.
+// 2. "isFooterKey" - a boolean. If true, means that the material belongs to a file footer
+// key, and keeps additional information (such as
+//     KMS instance ID and URL). If false, means that the material belongs to a column
+//     key.
+// 3. "kmsInstanceID" - a String, with the KMS Instance ID. Written only in footer key
+// material.
+// 4. "kmsInstanceURL" - a String, with the KMS Instance URL. Written only in footer key
+// material.
+// 5. "masterKeyID" - a String, with the ID of the master key used to generate the
+// material.
+// 6. "wrappedDEK" - a String, with the wrapped DEK (base64 encoding).
+// 7. "doubleWrapping" - a boolean. If true, means that the material was generated in
+// double wrapping mode.
+//     If false - in single wrapping mode.
+// 8. "keyEncryptionKeyID" - a String, with the ID of the KEK used to generate the
+// material. Written only in double wrapping mode.
+// 9. "wrappedKEK" - a String, with the wrapped KEK (base64 encoding). Written only in
+// double wrapping mode.
+class KeyMaterial {
+ public:
+  static constexpr char KEY_MATERIAL_TYPE_FIELD[] = "keyMaterialType";
+  static constexpr char KEY_MATERIAL_TYPE1[] = "PKMT1";
+
+  static constexpr char FOOTER_KEY_ID_IN_FILE[] = "footerKey";
+  static constexpr char COLUMN_KEY_ID_IN_FILE_PREFIX[] = "columnKey";
+
+  static constexpr char IS_FOOTER_KEY_FIELD[] = "isFooterKey";
+  static constexpr char DOUBLE_WRAPPING_FIELD[] = "doubleWrapping";
+  static constexpr char KMS_INSTANCE_ID_FIELD[] = "kmsInstanceID";
+  static constexpr char KMS_INSTANCE_URL_FIELD[] = "kmsInstanceURL";
+  static constexpr char MASTER_KEY_ID_FIELD[] = "masterKeyID";
+  static constexpr char WRAPPED_DEK_FIELD[] = "wrappedDEK";
+  static constexpr char KEK_ID_FIELD[] = "keyEncryptionKeyID";
+  static constexpr char WRAPPED_KEK_FIELD[] = "wrappedKEK";
+
+ public:
+  KeyMaterial() = default;
+
+  static KeyMaterial Parse(const std::string& key_material_string);
+
+  static KeyMaterial Parse(const rapidjson::Document& key_material_json);
+
+  static std::string CreateSerialized(bool is_footer_key,

Review comment:
       What is the use-case for this method?  Can this method be called SerializeToJson?

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation

Review comment:
       ```suggestion
   // to enable recovery of the key. This  interface can be utilized for implementation
   ```

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).

Review comment:
       ```suggestion
   // class for details).  This class is implemented to support version 1 of the parquet key management tools specification.
   ```

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       reference the version directly please.

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current
+// version, only one value is allowed - "PKMT1" (stands
+//     for "parquet key management tools, version 1")
+// 2. "internalStorage" - a boolean. If true, means that "key material" is kept inside the
+// "key metadata" field. If false, "key material"
+//     is kept externally (outside Parquet files) - in this case, "key metadata" keeps a
+//     reference to the external "key material".
+// 3. "keyReference" - a String, with the reference to the external "key material".
+// Written only if internalStorage is false.
+//
+// If internalStorage is true, "key material" is a part of "key metadata", and the json
+// keeps additional fields, described in the KeyMaterial class.
+class KeyMetadata {
+ public:
+  static constexpr char KEY_MATERIAL_INTERNAL_STORAGE_FIELD[] = "internalStorage";
+  static constexpr char KEY_REFERENCE_FIELD[] = "keyReference";
+
+  static KeyMetadata Parse(const std::string& key_metadata_bytes);
+
+  static std::string CreateSerializedForExternalMaterial(
+      const std::string& key_reference);
+
+  bool key_material_stored_internally() const { return is_internal_storage_; }
+
+  const KeyMaterial& key_material() const {
+    if (!is_internal_storage_) {
+      throw ParquetException("key material is stored externally.");
+    }
+    return key_material_;
+  }
+
+  const std::string& key_reference() const {
+    if (is_internal_storage_) {
+      throw ParquetException("key material is stored internally.");
+    }
+    return key_reference_;
+  }
+
+ private:
+  explicit KeyMetadata(const KeyMaterial& key_material);
+  explicit KeyMetadata(const std::string& key_reference);
+
+  bool is_internal_storage_;

Review comment:
       a better way of modelling this is with arrow::util::variant.

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual

Review comment:
       ```suggestion
   // "envelope encryption" pattern, allows integration with KMS servers. It keeps the actual
   ```

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current
+// version, only one value is allowed - "PKMT1" (stands
+//     for "parquet key management tools, version 1")
+// 2. "internalStorage" - a boolean. If true, means that "key material" is kept inside the
+// "key metadata" field. If false, "key material"
+//     is kept externally (outside Parquet files) - in this case, "key metadata" keeps a
+//     reference to the external "key material".
+// 3. "keyReference" - a String, with the reference to the external "key material".
+// Written only if internalStorage is false.
+//
+// If internalStorage is true, "key material" is a part of "key metadata", and the json
+// keeps additional fields, described in the KeyMaterial class.
+class KeyMetadata {
+ public:
+  static constexpr char KEY_MATERIAL_INTERNAL_STORAGE_FIELD[] = "internalStorage";

Review comment:
       kKeyMaterialInternalStorageField.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {

Review comment:
       docs please.

##########
File path: cpp/src/parquet/kms_client.h
##########
@@ -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.
+
+#pragma once
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KeyAccessToken {

Review comment:
       docs please.

##########
File path: cpp/src/parquet/kms_client_factory.h
##########
@@ -0,0 +1,42 @@
+// 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 "parquet/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KmsClientFactory {
+ public:
+  explicit KmsClientFactory(bool wrap_locally) : wrap_locally_(wrap_locally) {}
+
+  KmsClientFactory() : KmsClientFactory(false) {}
+
+  virtual ~KmsClientFactory() {}
+
+  virtual std::shared_ptr<KmsClient> CreateKmsClient(

Review comment:
       why a shared client?

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;

Review comment:
       const?

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {

Review comment:
       The design here isn't super idiomatic as I read it for caches (which generally one might more then one per process?  Could you discuss in more detail why this was chosen.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;

Review comment:
       could you discuss why a singleton is used here?  Is there never a case where two caches could be useful?

##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());

Review comment:
       Is the json allowed to be utf-8 encoded?

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {

Review comment:
       KekWriterCache, but better to spell out KEK.

##########
File path: cpp/src/parquet/string_util.h
##########
@@ -0,0 +1,29 @@
+// 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 <string>
+#include <vector>
+
+namespace parquet {
+
+std::vector<std::string> SplitString(const std::string& v, char delim);

Review comment:
       I would recommend moving this to arrow::util::string, and take arrow::util::string_view as an input.




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r485557938



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,

Review comment:
       Why isn't this a method on `KmsClientFactory`?




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486372095



##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,

Review comment:
       @ggershinsky I removed the method with `key_id_in_file`. It will be added when working with external storage.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r477067068



##########
File path: cpp/src/parquet/encryption_internal.h
##########
@@ -45,6 +45,9 @@ constexpr int8_t kOffsetIndex = 7;
 /// Performs AES encryption operations with GCM or CTR ciphers.
 class AesEncryptor {
  public:
+  /// Can serve one key length only. Possible values: 16, 24, 32 bytes.
+  explicit AesEncryptor(ParquetCipher::type alg_id, int key_len, bool metadata);

Review comment:
       @emkornfield I want to create a local variable of AesEncryptor, without using smart pointers, so it will store in stack rather than in heap.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498709142



##########
File path: cpp/src/parquet/encryption/remote_kms_client.cc
##########
@@ -0,0 +1,127 @@
+// 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.
+
+#include "arrow/json/object_parser.h"
+#include "arrow/json/object_writer.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/remote_kms_client.h"
+#include "parquet/exception.h"
+
+using arrow::json::ObjectParser;
+using arrow::json::ObjectWriter;
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char RemoteKmsClient::kLocalWrapNoKeyVersion[];
+
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapKeyVersionField[];
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapEncryptedKeyField[];
+
+RemoteKmsClient::LocalKeyWrap::LocalKeyWrap(const std::string& master_key_version,
+                                            const std::string& encrypted_encoded_key)
+    : encrypted_encoded_key_(encrypted_encoded_key),
+      master_key_version_(master_key_version) {}
+
+std::string RemoteKmsClient::LocalKeyWrap::CreateSerialized(
+    const std::string& encrypted_encoded_key) {
+  ObjectWriter json_writer;
+
+  json_writer.SetString(kLocalWrapKeyVersionField, kLocalWrapNoKeyVersion);
+  json_writer.SetString(kLocalWrapEncryptedKeyField, encrypted_encoded_key);
+
+  return json_writer.Serialize();
+}
+
+RemoteKmsClient::LocalKeyWrap RemoteKmsClient::LocalKeyWrap::Parse(
+    const std::string& wrapped_key) {
+  ObjectParser json_parser;
+  if (!json_parser.Parse(wrapped_key)) {
+    throw ParquetException("Failed to parse local key wrap json " + wrapped_key);
+  }
+  std::string master_key_version;
+  PARQUET_ASSIGN_OR_THROW(master_key_version,
+                          json_parser.GetString(kLocalWrapKeyVersionField));
+
+  std::string encrypted_encoded_key;
+  PARQUET_ASSIGN_OR_THROW(encrypted_encoded_key,
+                          json_parser.GetString(kLocalWrapEncryptedKeyField));
+
+  return RemoteKmsClient::LocalKeyWrap(master_key_version, encrypted_encoded_key);
+}
+
+void RemoteKmsClient::Initialize(const KmsConnectionConfig& kms_connection_config,
+                                 bool is_wrap_locally) {
+  kms_connection_config_ = kms_connection_config;
+  is_wrap_locally_ = is_wrap_locally;
+  if (is_wrap_locally_) {
+    master_key_cache_.Clear();
+  }
+
+  is_default_token_ =
+      kms_connection_config_.key_access_token() == KmsClient::kKeyAccessTokenDefault;
+
+  InitializeInternal();
+}
+
+std::string RemoteKmsClient::WrapKey(const std::string& key_bytes,
+                                     const std::string& master_key_identifier) {
+  if (is_wrap_locally_) {

Review comment:
       Thanks!




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498334175



##########
File path: cpp/src/parquet/encryption/remote_kms_client.cc
##########
@@ -0,0 +1,127 @@
+// 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.
+
+#include "arrow/json/object_parser.h"
+#include "arrow/json/object_writer.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/remote_kms_client.h"
+#include "parquet/exception.h"
+
+using arrow::json::ObjectParser;
+using arrow::json::ObjectWriter;
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char RemoteKmsClient::kLocalWrapNoKeyVersion[];
+
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapKeyVersionField[];
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapEncryptedKeyField[];
+
+RemoteKmsClient::LocalKeyWrap::LocalKeyWrap(const std::string& master_key_version,
+                                            const std::string& encrypted_encoded_key)
+    : encrypted_encoded_key_(encrypted_encoded_key),
+      master_key_version_(master_key_version) {}
+
+std::string RemoteKmsClient::LocalKeyWrap::CreateSerialized(
+    const std::string& encrypted_encoded_key) {
+  ObjectWriter json_writer;
+
+  json_writer.SetString(kLocalWrapKeyVersionField, kLocalWrapNoKeyVersion);
+  json_writer.SetString(kLocalWrapEncryptedKeyField, encrypted_encoded_key);
+
+  return json_writer.Serialize();
+}
+
+RemoteKmsClient::LocalKeyWrap RemoteKmsClient::LocalKeyWrap::Parse(
+    const std::string& wrapped_key) {
+  ObjectParser json_parser;
+  if (!json_parser.Parse(wrapped_key)) {
+    throw ParquetException("Failed to parse local key wrap json " + wrapped_key);
+  }
+  std::string master_key_version;
+  PARQUET_ASSIGN_OR_THROW(master_key_version,
+                          json_parser.GetString(kLocalWrapKeyVersionField));
+
+  std::string encrypted_encoded_key;
+  PARQUET_ASSIGN_OR_THROW(encrypted_encoded_key,
+                          json_parser.GetString(kLocalWrapEncryptedKeyField));
+
+  return RemoteKmsClient::LocalKeyWrap(master_key_version, encrypted_encoded_key);
+}
+
+void RemoteKmsClient::Initialize(const KmsConnectionConfig& kms_connection_config,
+                                 bool is_wrap_locally) {
+  kms_connection_config_ = kms_connection_config;
+  is_wrap_locally_ = is_wrap_locally;
+  if (is_wrap_locally_) {
+    master_key_cache_.Clear();
+  }
+
+  is_default_token_ =
+      kms_connection_config_.key_access_token() == KmsClient::kKeyAccessTokenDefault;
+
+  InitializeInternal();
+}
+
+std::string RemoteKmsClient::WrapKey(const std::string& key_bytes,
+                                     const std::string& master_key_identifier) {
+  if (is_wrap_locally_) {

Review comment:
       What about keeping `RemoteKmsClient` API the same, but write more 2 internal classes, like below:
   
   ```
   namespace internal {
    class LocalWrappingRemoteKmsClient {};
    class InServerWrappingRemoteKmsClient {};
   }
   
   class RemoteKmsClient {
    private:
     bool wrap_locally_;
     arrow::util::variant<LocalWrappingRemoteKmsClient,InServerRemoteKmsClient> internal_kms_client_;
   }
   ```
   Any idea? @pitrou @ggershinsky 




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r579596186



##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED

Review comment:
       I removed them.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497970716



##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)
+      : cached_item_(cached_item) {
+    expiration_timestamp_ =
+        CurrentTimePoint() + std::chrono::milliseconds(expiration_interval_millis);
+  }
+
+  bool IsExpired() {
+    auto now = CurrentTimePoint();
+    return (now > expiration_timestamp_);
+  }
+
+  E cached_item() { return cached_item_; }
+
+ private:
+  TimePoint expiration_timestamp_;
+  E cached_item_;
+};
+
+// This class is to avoid the below warning when compiling KeyToolkit class with VS2015
+// warning C4503: decorated name length exceeded, name was truncated
+template <typename V>
+class ExpiringCacheMapEntry {
+ public:
+  ExpiringCacheMapEntry() = default;
+
+  explicit ExpiringCacheMapEntry(std::shared_ptr<ConcurrentMap<V>> cached_item,
+                                 uint64_t expiration_interval_millis)
+      : map_cache_(cached_item, expiration_interval_millis) {}
+
+  bool IsExpired() { return map_cache_.IsExpired(); }
+
+  std::shared_ptr<ConcurrentMap<V>> cached_item() { return map_cache_.cached_item(); }
+
+ private:
+  ExpiringCacheEntry<std::shared_ptr<ConcurrentMap<V>>> map_cache_;
+};
+
+}  // namespace internal
+
+// Two-level cache with expiration of internal caches according to token lifetime.
+// External cache is per token, internal is per string key.
+// Wrapper class around:
+//    std::unordered_map<std::string,
+//    internal::ExpiringCacheEntry<std::unordered_map<std::string, V>>>
+// This cache is safe to be shared between threads.
+template <typename V>
+class TwoLevelCacheWithExpiration {
+ public:
+  TwoLevelCacheWithExpiration() {
+    last_cache_cleanup_timestamp_ = internal::CurrentTimePoint();
+  }
+
+  std::shared_ptr<ConcurrentMap<V>> GetOrCreateInternalCache(
+      const std::string& access_token, uint64_t cache_entry_lifetime_ms) {

Review comment:
       I follow the code in Java: https://github.com/apache/parquet-mr/pull/615/files#diff-6926b4ba3bb623d4467b3752c5846f47R42. So maybe @ggershinsky can explain 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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497980844



##########
File path: cpp/src/parquet/encryption/key_material.h
##########
@@ -0,0 +1,125 @@
+// 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 <map>
+#include <string>
+
+#include "arrow/json/object_parser.h"
+
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KeyMaterial class represents the "key material", keeping the information that allows
+// readers to recover an encryption key (see description of the KeyMetadata class). The
+// keytools package (PARQUET-1373) implements the "envelope encryption" pattern, in a
+// "single wrapping" or "double wrapping" mode. In the single wrapping mode, the key
+// material is generated by encrypting the "data encryption key" (DEK) by a "master key".
+// In the double wrapping mode, the key material is generated by encrypting the DEK by a
+// "key encryption key" (KEK), that in turn is encrypted by a "master key".
+//
+// Key material is kept in a flat json object, with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current
+// version, only one value is allowed - "PKMT1" (stands
+//     for "parquet key management tools, version 1"). For external key material storage,
+//     this field is written in both "key metadata" and "key material" jsons. For internal
+//     key material storage, this field is written only once in the common json.
+// 2. "isFooterKey" - a boolean. If true, means that the material belongs to a file footer
+// key, and keeps additional information (such as
+//     KMS instance ID and URL). If false, means that the material belongs to a column
+//     key.
+// 3. "kmsInstanceID" - a String, with the KMS Instance ID. Written only in footer key
+// material.
+// 4. "kmsInstanceURL" - a String, with the KMS Instance URL. Written only in footer key
+// material.
+// 5. "masterKeyID" - a String, with the ID of the master key used to generate the
+// material.
+// 6. "wrappedDEK" - a String, with the wrapped DEK (base64 encoding).
+// 7. "doubleWrapping" - a boolean. If true, means that the material was generated in
+// double wrapping mode.
+//     If false - in single wrapping mode.
+// 8. "keyEncryptionKeyID" - a String, with the ID of the KEK used to generate the
+// material. Written only in double wrapping mode.
+// 9. "wrappedKEK" - a String, with the wrapped KEK (base64 encoding). Written only in
+// double wrapping mode.
+class PARQUET_EXPORT KeyMaterial {
+ public:
+  // these fields are defined in a specification and should never be changed
+  static constexpr const char kKeyMaterialTypeField[] = "keyMaterialType";
+  static constexpr const char kKeyMaterialType1[] = "PKMT1";
+
+  static constexpr const char kFooterKeyIdInFile[] = "footerKey";
+  static constexpr const char kColumnKeyIdInFilePrefix[] = "columnKey";
+
+  static constexpr const char kIsFooterKeyField[] = "isFooterKey";
+  static constexpr const char kDoubleWrappingField[] = "doubleWrapping";
+  static constexpr const char kKmsInstanceIdField[] = "kmsInstanceID";
+  static constexpr const char kKmsInstanceUrlField[] = "kmsInstanceURL";
+  static constexpr const char kMasterKeyIdField[] = "masterKeyID";
+  static constexpr const char kWrappedDataEncryptionKeyField[] = "wrappedDEK";
+  static constexpr const char kKeyEncryptionKeyIdField[] = "keyEncryptionKeyID";
+  static constexpr const char kWrappedKeyEncryptionKeyField[] = "wrappedKEK";
+
+ public:
+  KeyMaterial() = default;
+
+  static KeyMaterial Parse(const std::string& key_material_string);
+
+  static KeyMaterial Parse(const arrow::json::ObjectParser& key_material_json);
+
+  /// This method returns a json string that will be stored either inside a parquet file
+  /// or in a key material store outside the parquet file.

Review comment:
       No, I meant "store" (or more clear, "file store", "file storage"). Sorry if my English is not very good.




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r582007270



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {

Review comment:
       Perhaps. I can't really give an opinion since I don't really understand what the purpose of this class it, sorry. I may be mistaken, but it seems to be a mix of unrelated functionality.




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



[GitHub] [arrow] bkietz commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-702112110


   > What about keeping this PR to see an overview, I'll create some smaller PRs to review?
   
   FWIW, I think that using JIRA would be more accessible for overview purposes:
   - Make ARROW-9318 into a tracking issue for the JIRAs of the smaller PRs
   - Paste subsets of the design doc into each of the JIRAs, supplemented by a summary of the interfaces/classes which will be added to address each subset of the requirements


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



[GitHub] [arrow] emkornfield commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r481294829



##########
File path: cpp/src/parquet/kms_client.h
##########
@@ -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.
+
+#pragma once
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;
+  std::unordered_map<std::string, std::string> custom_kms_conf;
+
+  const std::string& key_access_token() const {
+    if (refreshable_key_access_token == NULL ||
+        refreshable_key_access_token->value().empty()) {
+      throw ParquetException("key access token is not set!");
+    }
+    return refreshable_key_access_token->value();
+  }
+
+  void SetDefaultIfEmpty();
+};
+
+class PARQUET_EXPORT KmsClient {
+ public:
+  static constexpr char KMS_INSTANCE_ID_DEFAULT[] = "DEFAULT";
+  static constexpr char KMS_INSTANCE_URL_DEFAULT[] = "DEFAULT";
+  static constexpr char KEY_ACCESS_TOKEN_DEFAULT[] = "DEFAULT";
+
+  // Wraps a key - encrypts it with the master key, encodes the result
+  // and potentially adds a KMS-specific metadata.
+  virtual std::string WrapKey(const std::string& key_bytes,
+                              const std::string& master_key_identifier) = 0;

Review comment:
       @ggershinsky what type of interop are you thinking of?




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



[GitHub] [arrow] emkornfield commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r475376926



##########
File path: cpp/src/parquet/encryption.h
##########
@@ -47,15 +47,15 @@ using ColumnPathToEncryptionPropertiesMap =
 
 class PARQUET_EXPORT DecryptionKeyRetriever {
  public:
-  virtual std::string GetKey(const std::string& key_metadata) const = 0;

Review comment:
       would making those members `mutable` make sense instead?




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first part.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public API.

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first part.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public API.

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first part.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public API.

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first part.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public 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.

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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486382968



##########
File path: cpp/src/parquet/key_material.h
##########
@@ -0,0 +1,120 @@
+// 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 <map>
+#include <string>
+
+#include <rapidjson/document.h>
+
+namespace parquet {
+namespace encryption {
+
+// KeyMaterial class represents the "key material", keeping the information that allows
+// readers to recover an encryption key (see description of the KeyMetadata class). The
+// keytools package (PARQUET-1373) implements the "envelope encryption" pattern, in a
+// "single wrapping" or "double wrapping" mode. In the single wrapping mode, the key
+// material is generated by encrypting the "data encryption key" (DEK) by a "master key".
+// In the double wrapping mode, the key material is generated by encrypting the DEK by a
+// "key encryption key" (KEK), that in turn is encrypted by a "master key".
+//
+// Key material is kept in a flat json object, with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current
+// version, only one value is allowed - "PKMT1" (stands
+//     for "parquet key management tools, version 1"). For external key material storage,
+//     this field is written in both "key metadata" and "key material" jsons. For internal
+//     key material storage, this field is written only once in the common json.
+// 2. "isFooterKey" - a boolean. If true, means that the material belongs to a file footer
+// key, and keeps additional information (such as
+//     KMS instance ID and URL). If false, means that the material belongs to a column
+//     key.
+// 3. "kmsInstanceID" - a String, with the KMS Instance ID. Written only in footer key
+// material.
+// 4. "kmsInstanceURL" - a String, with the KMS Instance URL. Written only in footer key
+// material.
+// 5. "masterKeyID" - a String, with the ID of the master key used to generate the
+// material.
+// 6. "wrappedDEK" - a String, with the wrapped DEK (base64 encoding).
+// 7. "doubleWrapping" - a boolean. If true, means that the material was generated in
+// double wrapping mode.
+//     If false - in single wrapping mode.
+// 8. "keyEncryptionKeyID" - a String, with the ID of the KEK used to generate the
+// material. Written only in double wrapping mode.
+// 9. "wrappedKEK" - a String, with the wrapped KEK (base64 encoding). Written only in
+// double wrapping mode.
+class KeyMaterial {
+ public:
+  static constexpr char KEY_MATERIAL_TYPE_FIELD[] = "keyMaterialType";
+  static constexpr char KEY_MATERIAL_TYPE1[] = "PKMT1";
+
+  static constexpr char FOOTER_KEY_ID_IN_FILE[] = "footerKey";
+  static constexpr char COLUMN_KEY_ID_IN_FILE_PREFIX[] = "columnKey";
+
+  static constexpr char IS_FOOTER_KEY_FIELD[] = "isFooterKey";
+  static constexpr char DOUBLE_WRAPPING_FIELD[] = "doubleWrapping";
+  static constexpr char KMS_INSTANCE_ID_FIELD[] = "kmsInstanceID";
+  static constexpr char KMS_INSTANCE_URL_FIELD[] = "kmsInstanceURL";
+  static constexpr char MASTER_KEY_ID_FIELD[] = "masterKeyID";
+  static constexpr char WRAPPED_DEK_FIELD[] = "wrappedDEK";
+  static constexpr char KEK_ID_FIELD[] = "keyEncryptionKeyID";
+  static constexpr char WRAPPED_KEK_FIELD[] = "wrappedKEK";
+
+ public:
+  KeyMaterial() = default;
+
+  static KeyMaterial Parse(const std::string& key_material_string);
+
+  static KeyMaterial Parse(const rapidjson::Document& key_material_json);
+
+  static std::string CreateSerialized(bool is_footer_key,

Review comment:
       I updated it to `SerializeToJson`




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)
+      : cached_item_(cached_item) {
+    expiration_timestamp_ =
+        CurrentTimePoint() + std::chrono::milliseconds(expiration_interval_millis);
+  }
+
+  bool IsExpired() {
+    auto now = CurrentTimePoint();
+    return (now > expiration_timestamp_);
+  }
+
+  E cached_item() { return cached_item_; }
+
+ private:
+  TimePoint expiration_timestamp_;
+  E cached_item_;
+};
+
+// This class is to avoid the below warning when compiling KeyToolkit class with VS2015
+// warning C4503: decorated name length exceeded, name was truncated
+template <typename V>
+class ExpiringCacheMapEntry {
+ public:
+  ExpiringCacheMapEntry() = default;
+
+  explicit ExpiringCacheMapEntry(std::shared_ptr<ConcurrentMap<V>> cached_item,
+                                 uint64_t expiration_interval_millis)
+      : map_cache_(cached_item, expiration_interval_millis) {}
+
+  bool IsExpired() { return map_cache_.IsExpired(); }
+
+  std::shared_ptr<ConcurrentMap<V>> cached_item() { return map_cache_.cached_item(); }
+
+ private:
+  ExpiringCacheEntry<std::shared_ptr<ConcurrentMap<V>>> map_cache_;
+};
+
+}  // namespace internal
+
+// Two-level cache with expiration of internal caches according to token lifetime.
+// External cache is per token, internal is per string key.
+// Wrapper class around:
+//    std::unordered_map<std::string,
+//    internal::ExpiringCacheEntry<std::unordered_map<std::string, V>>>
+// This cache is safe to be shared between threads.
+template <typename V>
+class TwoLevelCacheWithExpiration {
+ public:
+  TwoLevelCacheWithExpiration() {
+    last_cache_cleanup_timestamp_ = internal::CurrentTimePoint();
+  }
+
+  std::shared_ptr<ConcurrentMap<V>> GetOrCreateInternalCache(
+      const std::string& access_token, uint64_t cache_entry_lifetime_ms) {

Review comment:
       The public API uses seconds, so it should be ok to use seconds in the internal functions.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497981549



##########
File path: cpp/src/parquet/encryption/file_key_wrapper.h
##########
@@ -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.
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/file_key_material_store.h"
+#include "parquet/encryption/key_encryption_key.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class will generate "key metadata" from "data encryption key" and "master key",
+// following these steps:
+// 1. Wrap "data encryption key". There are 2 modes:
+// 1.1. single wrapping: encrypt "data encryption key" directly with "master encryption
+// key" 1.2. double wrapping: 2 steps: 1.2.1. "key encryption key" is randomized (see
+// structure of KeyEncryptionKey class) 1.2.2. "data encryption key" is encrypted with the
+// above "key encryption key"
+// 2. Create "key material" (see structure in KeyMaterial class)
+// 3. Create "key metadata" with "key material" inside or a reference to outside "key
+// material" (see structure in KeyMetadata class).
+// Currently we don't support the case "key material" stores outside "key metadata"
+// yet.
+class PARQUET_EXPORT FileKeyWrapper {
+ public:
+  static constexpr int kKeyEncryptionKeyLength = 16;
+  static constexpr int kKeyEncryptionKeyIdLength = 16;
+
+  /// key_toolkit and kms_connection_config is to get KmsClient from the cache or create
+  /// KmsClient if it's not in the cache yet. cache_entry_lifetime_seconds is life time of
+  /// KmsClient in the cache. key_material_store is to store "key material" outside
+  /// parquet file, NULL if "key material" is stored inside parquet file.
+  FileKeyWrapper(KeyToolkit* key_toolkit,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime_seconds, bool double_wrapping);
+
+  /// Creates key_metadata field for a given data key, via wrapping the key with the
+  /// master key
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+ private:
+  KeyEncryptionKey CreateKeyEncryptionKey(const std::string& master_key_id);
+
+  /// A map of Master Encryption Key ID -> KeyEncryptionKey, for the current token
+  std::shared_ptr<arrow::util::ConcurrentMap<KeyEncryptionKey>> kek_per_master_key_id_;

Review comment:
       That variable references the item in cache. When the item is expired, a thread can remove it from cache, while this thread can still access it. So I use `shared_ptr` here to make sure this variable still alive when the item in cache is removed.




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



[GitHub] [arrow] ggershinsky commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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


   > > In general, this PR is difficult to review due to its size. Would it be possible to break it into a two or more smaller PRs?
   > > In theory, I think yes, we can create each PR for a group of some classes but it may be hard for reviewers to see an overview. What about keeping this PR to see an overview, I'll create some smaller PRs to review? @ggershinsky
   
   sgtm


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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,

Review comment:
       yep, I agree with you both, this is a bit confusing. Fortunately, this is not a public API, only an internal function; but still.. The "key metadata", returned here, refers to this term in the most basic sense - that is, a free-form byte array field, as defined in the encryption spec / thrift format. In the Java version, we simply return a "byte[]" here. On the other hand, the KeyMetadata class encapsulates a more detailed implementation of this term, that assumes the PKMT-specific structure, so it can not be used as the return parameter.
   Like @thamht4190 , I'd prefer the current approach, and agree with @emkornfield that it would be good to have a comment here (and in the next function). Something like 
   "Creates key_metadata field for a given data key, via wrapping the key with the master key"

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,

Review comment:
       a comment, something like
   "Creates key_metadata field for a given data key, via wrapping the key with the master key. This function signature is only called from the key rotation procedure, where the key_id_in_file is already available (retrieved from the key material being rotated), so the function does not have to generate these ids."

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;

Review comment:
       There is some cost, associated with creating a KmsClient instance object, including a potential interaction with a remote KMS Server. Therefore, the cache is a singleton, so only one thread has to pay this cost, others will get the instance from the cache.

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key, std::string key_id_in_file);
+
+ private:
+  KeyEncryptionKey CreateKeyEncryptionKey(const std::string& master_key_id);
+
+  // A map of MEK_ID -> KeyEncryptionKey, for the current token
+  std::map<std::string, KeyEncryptionKey> kek_per_master_key_id_;
+
+  std::shared_ptr<KmsClient> kms_client_;
+  KmsConnectionConfig kms_connection_config_;
+  std::shared_ptr<FileKeyMaterialStore> key_material_store_;
+  uint64_t cache_entry_lifetime_;
+  bool double_wrapping_;
+  uint16_t key_counter_;

Review comment:
       the maximal value of this counter is the number of columns in a file (if every one is encrypted with a different key), so int16 is reasonable. Moreover, there is a deeper limit on this, described under https://github.com/apache/parquet-format/pull/114#discussion_r232941138 




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r475446337



##########
File path: cpp/src/parquet/file_key_material_store.h
##########
@@ -0,0 +1,51 @@
+// 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") = 0; 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>
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyMaterialStore {
+ public:
+  // Initializes key material store for a parquet file.
+  virtual void Initialize(const std::string& parquet_file_path, bool temp_store) = 0;

Review comment:
       For simplicity, I will remove members of this class since they are not used anywhere yet and keeps only the class name. I will add later when implementing with hadoop file system.




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



[GitHub] [arrow] pitrou commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-795824822


   Ok, I've pushed some updates to try and conform with coding conventions a bit more.
   
   Some high-level comments:
   * In general, I'm not particularly happy with the API (for example, why is the configuration mutated while in use?)
   * There are some roundtrip tests but no tests with reference data; so there's no guarantee that this actually implements the spec faithfully
   * I'm concerned that the crypto code isn't very mindful of safety (see https://issues.apache.org/jira/browse/PARQUET-1997); this PR builds on that and also seems to keep sensitive data around (decryption keys) in various places
   
   I also don't have a personal interest in this currently and I recognize that the PR implements a useful functionality.
   @ggershinsky @thamht4190 Do you plan to address some of these concerns in this PR? Otherwise, I may merge this and users of this code will have to be mindful of potential issues.
   


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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/key_toolkit_internal.h
##########
@@ -0,0 +1,52 @@
+// 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 <chrono>
+#include <string>
+
+namespace parquet {
+namespace encryption {
+namespace internal {
+
+// "data encryption key" and "master key identifier" are paired together as output when
+// parsing from "key material"
+class KeyWithMasterId {
+ public:
+  KeyWithMasterId(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  const std::string key_bytes_;
+  const std::string master_id_;
+};
+
+/// Encrypts "key" with "master_key", using AES-GCM and the "aad"

Review comment:
       We're using two ciphers in Parquet encryption spec - AES-GCM and AES-CTR. Out of these two, GCM is more secure, because it prevents tampering with the cipher text (encrypted key in this case).  
   The "key" size is configurable. The "master_key" is fetched from KMS, so beyond our control. But it would be good indeed to make sure it has an allowed length. @thamht4190 , could you add a check of the master keys, right after they are returned from GetMasterKeyFromServer, to verify their length is either 16, 24 or 32 bytes?




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r475440024



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {

Review comment:
       I based on Java version, so maybe @ggershinsky will help to answer this question.




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498073329



##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {

Review comment:
       That depends if _only_ the map is accessed by multiple threads. Generally, the map is only one field inside a class and other fields must be synchronized as well, so the class has its own mutex.
   




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer

Review comment:
       It would be useful to spell out the default values in the public API documentation.  E.g.,
   ". The default is false - files are written with encrypted footer".

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key

Review comment:
       please add "all" - "encrypt footer and all columns with the same encryption key"

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).

Review comment:
       "The default is 600 (10 minutes)"

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).

Review comment:
       "The default is 600 (10 minutes)"

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.

Review comment:
       "The default is 128 bits"

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.

Review comment:
       "The default is true. If set to false, use single wrapping - where DEKs are encrypted directly with master keys"




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r475442393



##########
File path: cpp/src/parquet/kms_client_factory.h
##########
@@ -0,0 +1,42 @@
+// 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 "parquet/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KmsClientFactory {
+ public:
+  explicit KmsClientFactory(bool wrap_locally) : wrap_locally_(wrap_locally) {}
+
+  KmsClientFactory() : KmsClientFactory(false) {}
+
+  virtual ~KmsClientFactory() {}
+
+  virtual std::shared_ptr<KmsClient> CreateKmsClient(

Review comment:
       The code accesses a KmsClient instance at many places, so I think a shared client is a must.




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r485575668



##########
File path: cpp/src/parquet/kms_client_factory.h
##########
@@ -0,0 +1,42 @@
+// 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 "parquet/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KmsClientFactory {
+ public:
+  explicit KmsClientFactory(bool wrap_locally) : wrap_locally_(wrap_locally) {}
+
+  KmsClientFactory() : KmsClientFactory(false) {}
+
+  virtual ~KmsClientFactory() {}
+
+  virtual std::shared_ptr<KmsClient> CreateKmsClient(

Review comment:
       Also, why is this method pure virtual? The user can't create a `KmsClientFactory` directly?
   If this is only for internal use, put this in an `internal` namespace or something.




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r485569777



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,

Review comment:
       (note you can use `enable_shared_from_this` if the `shared_ptr` is the concern...)




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



[GitHub] [arrow] thamht4190 commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-701817028


   > In general, this PR is difficult to review due to its size. Would it be possible to break it into a two or more smaller PRs?
   In theory, I think yes, we can create each PR for a group of some classes but it may be hard for reviewers to see an overview. What about keeping this PR to see an overview, I'll create some smaller PRs to review? @ggershinsky
   
   


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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/properties_driven_crypto_factory.cc
##########
@@ -0,0 +1,278 @@
+// 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.
+
+#include <sstream>
+
+#include "arrow/result.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+#include "parquet/encryption_internal.h"
+#include "parquet/file_key_material_store.h"
+#include "parquet/file_key_unwrapper.h"
+#include "parquet/properties_driven_crypto_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr const int32_t PropertiesDrivenCryptoFactory::kAcceptableDataKeyLengths[];
+
+EncryptionConfiguration::Builder* EncryptionConfiguration::Builder::column_keys(
+    const std::string& column_keys) {
+  DCHECK(!column_keys.empty());
+  if (uniform_encryption_) {
+    throw ParquetException(
+        "Cannot call both column_keys(const std::string&) and uniform_encryption()");
+  }
+  column_keys_ = column_keys;
+  return this;
+}
+
+EncryptionConfiguration::Builder* EncryptionConfiguration::Builder::uniform_encryption() {
+  if (!column_keys_.empty()) {
+    throw ParquetException(
+        "Cannot call both column_keys(const std::string&) and uniform_encryption()");
+  }
+  uniform_encryption_ = true;
+  return this;
+}
+
+EncryptionConfiguration::Builder* EncryptionConfiguration::Builder::encryption_algorithm(
+    ParquetCipher::type algo) {
+  encryption_algorithm_ = algo;
+  return this;
+}
+
+EncryptionConfiguration::Builder* EncryptionConfiguration::Builder::plaintext_footer(
+    bool plaintext_footer) {
+  plaintext_footer_ = plaintext_footer;
+  return this;
+}
+
+EncryptionConfiguration::Builder* EncryptionConfiguration::Builder::double_wrapping(
+    bool double_wrapping) {
+  double_wrapping_ = double_wrapping;
+  return this;
+}
+
+EncryptionConfiguration::Builder* EncryptionConfiguration::Builder::wrap_locally(
+    bool wrap_locally) {
+  wrap_locally_ = wrap_locally;
+  return this;
+}
+
+EncryptionConfiguration::Builder*
+EncryptionConfiguration::Builder::cache_lifetime_seconds(
+    uint64_t cache_lifetime_seconds) {
+  cache_lifetime_seconds_ = cache_lifetime_seconds;
+  return this;
+}
+
+EncryptionConfiguration::Builder* EncryptionConfiguration::Builder::internal_key_material(
+    bool internal_key_material) {
+  internal_key_material_ = internal_key_material;
+  return this;
+}
+
+EncryptionConfiguration::Builder* EncryptionConfiguration::Builder::data_key_length_bits(
+    int32_t data_key_length_bits) {
+  data_key_length_bits_ = data_key_length_bits;
+  return this;
+}
+
+std::shared_ptr<EncryptionConfiguration> EncryptionConfiguration::Builder::build() {
+  if (!uniform_encryption_ && column_keys_.empty()) {
+    throw ParquetException(
+        "Either column_keys(const std::string&) or uniform_encryption() must be "
+        "called.");
+  }
+
+  return std::make_shared<EncryptionConfiguration>(
+      footer_key_, column_keys_, encryption_algorithm_, plaintext_footer_,
+      double_wrapping_, wrap_locally_, cache_lifetime_seconds_, internal_key_material_,
+      uniform_encryption_, data_key_length_bits_);
+}
+
+DecryptionConfiguration::Builder* DecryptionConfiguration::Builder::wrap_locally(
+    bool wrap_locally) {
+  wrap_locally_ = wrap_locally;
+  return this;
+}
+
+DecryptionConfiguration::Builder*
+DecryptionConfiguration::Builder::cache_lifetime_seconds(
+    uint64_t cache_lifetime_seconds) {
+  cache_lifetime_seconds_ = cache_lifetime_seconds;
+  return this;
+}
+
+std::shared_ptr<DecryptionConfiguration> DecryptionConfiguration::Builder::build() {
+  return std::make_shared<DecryptionConfiguration>(wrap_locally_,
+                                                   cache_lifetime_seconds_);
+}
+
+void PropertiesDrivenCryptoFactory::kms_client_factory(

Review comment:
       I think it's better called "register_kms_client_factory" or "set_kms_client_factory". Also, can be made to throw an exception if already set, so a user won't override it by mistake.




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



[GitHub] [arrow] pitrou commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-799528319


   @emkornfield Feel free to take a last 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.

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



[GitHub] [arrow] emkornfield commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-800863407


   Sorry didn't get a chance to look.  @pitrou if you are comfortable with changes please go ahead and merge


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



[GitHub] [arrow] emkornfield commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-799936764


   It looks like there are JSON changes in here? does this need a rebase?


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



[GitHub] [arrow] emkornfield commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-796862072


    > I also don't have a personal interest in this currently and I recognize that the PR implements a useful functionality.
   @ggershinsky @thamht4190 Do you plan to address some of these concerns in this PR? Otherwise, I may merge this and users of this code will have to be mindful of potential issues.
   
   I'll try to look tonight but this makes it sound like there are general concerns around quality here, and that doesn't seem like something we want to merge (especially if it is security related).  If I get distracted and don't give feedback by monday, it is OK not to block on me.


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



[GitHub] [arrow] github-actions[bot] commented on pull request #8023: Arrow 9318 encryption key management

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-678610345


   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   Could you open an issue for this pull request on JIRA?
   https://issues.apache.org/jira/browse/ARROW
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497996333



##########
File path: cpp/src/arrow/json/object_parser.h
##########
@@ -0,0 +1,45 @@
+// 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 "arrow/json/rapidjson_defs.h"  // IWYU pragma: keep
+
+#include <rapidjson/document.h>
+
+#include "arrow/result.h"
+#include "arrow/util/string_view.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace json {
+
+namespace rj = arrow::rapidjson;
+
+class ARROW_EXPORT ObjectParser {
+ public:
+  bool Parse(arrow::util::string_view json);
+
+  Result<std::string> GetString(const char* key) const;
+  Result<bool> GetBool(const char* key) const;
+
+ private:
+  rj::Document _document;

Review comment:
       @bkietz Sorry I don't understand this part "For example, the only public mention of it ObjectParser is KeyMaterial::Parse where it could be replaced by a string.". Can you explain more? Thanks!




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



[GitHub] [arrow] roee88 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
roee88 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r490958753



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,223 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be

Review comment:
       Doesn't the low level API allow encrypting only the footer? I think this is useful even for codebases that use PME without actually encrypting any column. For example, if the columns to encrypt are decided according to policies, this limitations introduces a lot of complexity.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498141296



##########
File path: cpp/src/parquet/encryption/kms_client.h
##########
@@ -0,0 +1,84 @@
+// 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 <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class wraps the key access token of a KMS server. If your token changes over time,
+// you should keep the reference to the KeyAccessToken object and call Refresh() method
+// every time you have a new token.
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;

Review comment:
       After parsing this `KmsConnectionConfig` object into `PropertiesDrivenCryptoFactory`, the token can be changed in the future, so we need `shared_ptr` to change its value later.
   `KeyAccessToken` can be accessed by multiple threads. I will add a mutex. Sorry, I forgot 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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498008512



##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED
+  fields.push_back(PrimitiveNode::Make(kBooleanFieldName, Repetition::REQUIRED,
+                                       Type::BOOLEAN, ConvertedType::NONE));
+
+  // Create a primitive node named 'int32_field' with type:INT32, repetition:REQUIRED,
+  // logical type:TIME_MILLIS
+  fields.push_back(PrimitiveNode::Make(kInt32FieldName, Repetition::REQUIRED, Type::INT32,
+                                       ConvertedType::TIME_MILLIS));
+
+  // Create a primitive node named 'int64_field' with type:INT64, repetition:REPEATED
+  fields.push_back(PrimitiveNode::Make(kInt64FieldName, Repetition::REPEATED, Type::INT64,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kInt96FieldName, Repetition::REQUIRED, Type::INT96,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kFloatFieldName, Repetition::REQUIRED, Type::FLOAT,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kDoubleFieldName, Repetition::REQUIRED,
+                                       Type::DOUBLE, ConvertedType::NONE));
+
+  // Create a primitive node named 'ba_field' with type:BYTE_ARRAY, repetition:OPTIONAL
+  fields.push_back(PrimitiveNode::Make(kByteArrayFieldName, Repetition::OPTIONAL,
+                                       Type::BYTE_ARRAY, ConvertedType::NONE));
+
+  // Create a primitive node named 'flba_field' with type:FIXED_LEN_BYTE_ARRAY,
+  // repetition:REQUIRED, field_length = kFixedLength
+  fields.push_back(PrimitiveNode::Make(kFixedLenByteArrayFieldName, Repetition::REQUIRED,
+                                       Type::FIXED_LEN_BYTE_ARRAY, ConvertedType::NONE,
+                                       kFixedLength));
+
+  // Create a GroupNode named 'schema' using the primitive nodes defined above
+  // This GroupNode is the root node of the schema tree
+  return std::static_pointer_cast<GroupNode>(
+      GroupNode::Make("schema", Repetition::REQUIRED, fields));
+}
+
+void FileEncryptor::EncryptFile(
+    std::string file,
+    std::shared_ptr<parquet::FileEncryptionProperties> encryption_configurations) {
+  WriterProperties::Builder prop_builder;
+  prop_builder.compression(parquet::Compression::SNAPPY);
+  prop_builder.encryption(encryption_configurations);
+  std::shared_ptr<WriterProperties> writer_properties = prop_builder.build();
+
+  PARQUET_ASSIGN_OR_THROW(auto out_file, FileClass::Open(file));
+  // Create a ParquetFileWriter instance
+  std::shared_ptr<parquet::ParquetFileWriter> file_writer =
+      parquet::ParquetFileWriter::Open(out_file, schema_, writer_properties);
+
+  for (int r = 0; r < num_rgs; r++) {
+    bool buffered_mode = r % 2 == 0;
+    auto row_group_writer = buffered_mode ? file_writer->AppendBufferedRowGroup()
+                                          : file_writer->AppendRowGroup();
+
+    int column_index = 0;
+    // Captures i by reference; increments it by one
+    auto get_next_column = [&]() {
+      return buffered_mode ? row_group_writer->column(column_index++)
+                           : row_group_writer->NextColumn();
+    };
+
+    // Write the Bool column
+    parquet::BoolWriter* bool_writer =
+        static_cast<parquet::BoolWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      bool value = ((i % 2) == 0) ? true : false;
+      bool_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Int32 column
+    parquet::Int32Writer* int32_writer =
+        static_cast<parquet::Int32Writer*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      int32_t value = i;
+      int32_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Int64 column. Each row has repeats twice.
+    parquet::Int64Writer* int64_writer =
+        static_cast<parquet::Int64Writer*>(get_next_column());
+    for (int i = 0; i < 2 * rows_per_rowgroup_; i++) {
+      int64_t value = i * 1000 * 1000;
+      value *= 1000 * 1000;
+      int16_t definition_level = 1;
+      int16_t repetition_level = 0;
+      if ((i % 2) == 0) {
+        repetition_level = 1;  // start of a new record
+      }
+      int64_writer->WriteBatch(1, &definition_level, &repetition_level, &value);
+    }
+
+    // Write the INT96 column.
+    parquet::Int96Writer* int96_writer =
+        static_cast<parquet::Int96Writer*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::Int96 value;
+      value.value[0] = i;
+      value.value[1] = i + 1;
+      value.value[2] = i + 2;
+      int96_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Float column
+    parquet::FloatWriter* float_writer =
+        static_cast<parquet::FloatWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      float value = static_cast<float>(i) * 1.1f;
+      float_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Double column
+    parquet::DoubleWriter* double_writer =
+        static_cast<parquet::DoubleWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      double value = i * 1.1111111;
+      double_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the ByteArray column. Make every alternate values NULL
+    parquet::ByteArrayWriter* ba_writer =
+        static_cast<parquet::ByteArrayWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::ByteArray value;
+      char hello[kFixedLength] = "parquet";
+      hello[7] = static_cast<char>(static_cast<int>('0') + i / 100);
+      hello[8] = static_cast<char>(static_cast<int>('0') + (i / 10) % 10);
+      hello[9] = static_cast<char>(static_cast<int>('0') + i % 10);
+      if (i % 2 == 0) {
+        int16_t definition_level = 1;
+        value.ptr = reinterpret_cast<const uint8_t*>(&hello[0]);
+        value.len = kFixedLength;
+        ba_writer->WriteBatch(1, &definition_level, nullptr, &value);
+      } else {
+        int16_t definition_level = 0;
+        ba_writer->WriteBatch(1, &definition_level, nullptr, nullptr);
+      }
+    }
+
+    // Write the FixedLengthByteArray column
+    parquet::FixedLenByteArrayWriter* flba_writer =
+        static_cast<parquet::FixedLenByteArrayWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::FixedLenByteArray value;
+      char v = static_cast<char>(i);
+      char flba[kFixedLength] = {v, v, v, v, v, v, v, v, v, v};
+      value.ptr = reinterpret_cast<const uint8_t*>(&flba[0]);
+      flba_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+  }
+
+  // Close the ParquetFileWriter
+  file_writer->Close();
+
+  return;
+}
+
+void FileDecryptor::DecryptFile(
+    std::string file,
+    std::shared_ptr<FileDecryptionProperties> file_decryption_properties) {
+  std::string exception_msg;
+  parquet::ReaderProperties reader_properties = parquet::default_reader_properties();
+  if (file_decryption_properties) {
+    reader_properties.file_decryption_properties(file_decryption_properties->DeepClone());
+  }
+
+  auto file_reader = parquet::ParquetFileReader::OpenFile(file, false, reader_properties);
+
+  // Get the File MetaData
+  std::shared_ptr<parquet::FileMetaData> file_metadata = file_reader->metadata();
+
+  // Get the number of RowGroups
+  int num_row_groups = file_metadata->num_row_groups();
+
+  // Get the number of Columns
+  int num_columns = file_metadata->num_columns();
+  ASSERT_EQ(num_columns, 8);
+
+  // Iterate over all the RowGroups in the file
+  for (int r = 0; r < num_row_groups; ++r) {
+    // Get the RowGroup Reader
+    std::shared_ptr<parquet::RowGroupReader> row_group_reader = file_reader->RowGroup(r);
+
+    // Get the RowGroupMetaData
+    std::unique_ptr<RowGroupMetaData> rg_metadata = file_metadata->RowGroup(r);
+
+    int64_t values_read = 0;
+    int64_t rows_read = 0;
+    int16_t definition_level;
+    int16_t repetition_level;
+    int i;
+    std::shared_ptr<parquet::ColumnReader> column_reader;
+
+    // Get the Column Reader for the boolean column
+    column_reader = row_group_reader->Column(0);
+    parquet::BoolReader* bool_reader =
+        static_cast<parquet::BoolReader*>(column_reader.get());
+
+    // Get the ColumnChunkMetaData for the boolean column
+    std::unique_ptr<ColumnChunkMetaData> boolean_md = rg_metadata->ColumnChunk(0);
+
+    // Read all the rows in the column
+    i = 0;
+    while (bool_reader->HasNext()) {
+      bool value;
+      // Read one value at a time. The number of rows read is returned. values_read
+      // contains the number of non-null rows
+      rows_read = bool_reader->ReadBatch(1, nullptr, nullptr, &value, &values_read);
+      // Ensure only one value is read
+      ASSERT_EQ(rows_read, 1);
+      // There are no NULL values in the rows written
+      ASSERT_EQ(values_read, 1);
+      // Verify the value written
+      bool expected_value = ((i % 2) == 0) ? true : false;
+      ASSERT_EQ(value, expected_value);
+      i++;
+    }
+    // make sure we got the same number of values the metadata says
+    ASSERT_EQ(boolean_md->num_values(), i);
+
+    // Get the Column Reader for the Int32 column
+    column_reader = row_group_reader->Column(1);
+    parquet::Int32Reader* int32_reader =
+        static_cast<parquet::Int32Reader*>(column_reader.get());
+
+    // Get the ColumnChunkMetaData for the Int32 column
+    std::unique_ptr<ColumnChunkMetaData> int32_md = rg_metadata->ColumnChunk(1);
+
+    // Read all the rows in the column

Review comment:
       This code is not newly written in this pull, I only move here for common use. It's kind of difficult for option 1, since we use this code to read parquet files generated by Java for interop-test, or we have to change the Java unit tests as well.
   I will look for some solution for option 2.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,106 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KMS systems wrap keys by encrypting them by master keys, and attaching additional
+// information (such as the version number of the masker key) to the result of encryption.
+// The master key version is required in  key rotation. Currently, the local wrapping mode
+// does not support key rotation (because not all KMS systems allow to fetch a master key
+// by its ID and version number). Still, the local wrapping mode adds a placeholder for
+// the master key version, that will enable support for key rotation in this mode in the
+// future, with appropriate KMS systems. This will also enable backward compatibility,
+// where future readers will be able to extract master key version in the files written by
+// the current code.
+//
+// LocalKeyWrap class writes (and reads) the "key wrap" as a flat json with the following
+// fields:
+// 1. "masterKeyVersion" - a String, with the master key version. In the current version,
+// only one value is allowed - "NO_VERSION".
+// 2. "encryptedKey" - a String, with the key encrypted by the master key
+// (base64-encoded).
+class PARQUET_EXPORT RemoteKmsClient : public KmsClient {
+ public:
+  static constexpr const char kLocalWrapNoKeyVersion[] = "NO_VERSION";
+
+  void Initialize(const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally);
+
+  std::string WrapKey(const std::string& key_bytes,
+                      const std::string& master_key_identifier) override;
+
+  std::string UnwrapKey(const std::string& wrapped_key,
+                        const std::string& master_key_identifier) override;
+
+ protected:
+  // Wrap a key with the master key in the remote KMS server.
+  virtual std::string WrapKeyInServer(const std::string& key_bytes,
+                                      const std::string& master_key_identifier) = 0;
+
+  // Unwrap a key with the master key in the remote KMS server.
+  virtual std::string UnwrapKeyInServer(const std::string& wrapped_key,
+                                        const std::string& master_key_identifier) = 0;
+
+  // Get master key from the remote KMS server.
+  // Required only for local wrapping. No need to implement if KMS supports in-server
+  // wrapping/unwrapping.

Review comment:
       "Note: this function might be called by multiple threads"

##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,106 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KMS systems wrap keys by encrypting them by master keys, and attaching additional
+// information (such as the version number of the masker key) to the result of encryption.
+// The master key version is required in  key rotation. Currently, the local wrapping mode
+// does not support key rotation (because not all KMS systems allow to fetch a master key
+// by its ID and version number). Still, the local wrapping mode adds a placeholder for
+// the master key version, that will enable support for key rotation in this mode in the
+// future, with appropriate KMS systems. This will also enable backward compatibility,
+// where future readers will be able to extract master key version in the files written by
+// the current code.
+//
+// LocalKeyWrap class writes (and reads) the "key wrap" as a flat json with the following
+// fields:
+// 1. "masterKeyVersion" - a String, with the master key version. In the current version,
+// only one value is allowed - "NO_VERSION".
+// 2. "encryptedKey" - a String, with the key encrypted by the master key
+// (base64-encoded).
+class PARQUET_EXPORT RemoteKmsClient : public KmsClient {
+ public:
+  static constexpr const char kLocalWrapNoKeyVersion[] = "NO_VERSION";
+
+  void Initialize(const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally);
+
+  std::string WrapKey(const std::string& key_bytes,
+                      const std::string& master_key_identifier) override;
+
+  std::string UnwrapKey(const std::string& wrapped_key,
+                        const std::string& master_key_identifier) override;
+
+ protected:
+  // Wrap a key with the master key in the remote KMS server.

Review comment:
       Suggest adding this to the comment: "Note: this function might be called by multiple threads"

##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,106 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KMS systems wrap keys by encrypting them by master keys, and attaching additional
+// information (such as the version number of the masker key) to the result of encryption.
+// The master key version is required in  key rotation. Currently, the local wrapping mode
+// does not support key rotation (because not all KMS systems allow to fetch a master key
+// by its ID and version number). Still, the local wrapping mode adds a placeholder for
+// the master key version, that will enable support for key rotation in this mode in the
+// future, with appropriate KMS systems. This will also enable backward compatibility,
+// where future readers will be able to extract master key version in the files written by
+// the current code.
+//
+// LocalKeyWrap class writes (and reads) the "key wrap" as a flat json with the following
+// fields:
+// 1. "masterKeyVersion" - a String, with the master key version. In the current version,
+// only one value is allowed - "NO_VERSION".
+// 2. "encryptedKey" - a String, with the key encrypted by the master key
+// (base64-encoded).
+class PARQUET_EXPORT RemoteKmsClient : public KmsClient {
+ public:
+  static constexpr const char kLocalWrapNoKeyVersion[] = "NO_VERSION";
+
+  void Initialize(const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally);
+
+  std::string WrapKey(const std::string& key_bytes,
+                      const std::string& master_key_identifier) override;
+
+  std::string UnwrapKey(const std::string& wrapped_key,
+                        const std::string& master_key_identifier) override;
+
+ protected:
+  // Wrap a key with the master key in the remote KMS server.
+  virtual std::string WrapKeyInServer(const std::string& key_bytes,
+                                      const std::string& master_key_identifier) = 0;
+
+  // Unwrap a key with the master key in the remote KMS server.

Review comment:
       "Note: this function might be called by multiple threads"

##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,106 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KMS systems wrap keys by encrypting them by master keys, and attaching additional
+// information (such as the version number of the masker key) to the result of encryption.
+// The master key version is required in  key rotation. Currently, the local wrapping mode
+// does not support key rotation (because not all KMS systems allow to fetch a master key
+// by its ID and version number). Still, the local wrapping mode adds a placeholder for
+// the master key version, that will enable support for key rotation in this mode in the
+// future, with appropriate KMS systems. This will also enable backward compatibility,
+// where future readers will be able to extract master key version in the files written by
+// the current code.
+//
+// LocalKeyWrap class writes (and reads) the "key wrap" as a flat json with the following
+// fields:
+// 1. "masterKeyVersion" - a String, with the master key version. In the current version,
+// only one value is allowed - "NO_VERSION".
+// 2. "encryptedKey" - a String, with the key encrypted by the master key
+// (base64-encoded).

Review comment:
       Should this comment (lines 29-44) be moved to the LocalWrapKey class at the line 75?

##########
File path: cpp/src/parquet/encryption/kms_client.h
##########
@@ -0,0 +1,84 @@
+// 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 <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class wraps the key access token of a KMS server. If your token changes over time,
+// you should keep the reference to the KeyAccessToken object and call Refresh() method
+// every time you have a new token.
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;
+  std::unordered_map<std::string, std::string> custom_kms_conf;
+
+  const std::string& key_access_token() const {
+    if (refreshable_key_access_token == NULL ||
+        refreshable_key_access_token->value().empty()) {
+      throw ParquetException("key access token is not set!");
+    }
+    return refreshable_key_access_token->value();
+  }
+
+  void SetDefaultIfEmpty();
+};
+
+class PARQUET_EXPORT KmsClient {
+ public:
+  static constexpr const char kKmsInstanceIdDefault[] = "DEFAULT";
+  static constexpr const char kKmsInstanceUrlDefault[] = "DEFAULT";
+  static constexpr const char kKeyAccessTokenDefault[] = "DEFAULT";
+
+  // Wraps a key - encrypts it with the master key, encodes the result
+  // and potentially adds a KMS-specific metadata.
+  virtual std::string WrapKey(const std::string& key_bytes,
+                              const std::string& master_key_identifier) = 0;
+
+  // Decrypts (unwraps) a key with the master key.

Review comment:
       "Note: this function might be called by multiple threads"

##########
File path: cpp/src/parquet/encryption/kms_client.h
##########
@@ -0,0 +1,84 @@
+// 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 <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class wraps the key access token of a KMS server. If your token changes over time,
+// you should keep the reference to the KeyAccessToken object and call Refresh() method
+// every time you have a new token.
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;
+  std::unordered_map<std::string, std::string> custom_kms_conf;
+
+  const std::string& key_access_token() const {
+    if (refreshable_key_access_token == NULL ||
+        refreshable_key_access_token->value().empty()) {
+      throw ParquetException("key access token is not set!");
+    }
+    return refreshable_key_access_token->value();
+  }
+
+  void SetDefaultIfEmpty();
+};
+
+class PARQUET_EXPORT KmsClient {
+ public:
+  static constexpr const char kKmsInstanceIdDefault[] = "DEFAULT";
+  static constexpr const char kKmsInstanceUrlDefault[] = "DEFAULT";
+  static constexpr const char kKeyAccessTokenDefault[] = "DEFAULT";
+
+  // Wraps a key - encrypts it with the master key, encodes the result
+  // and potentially adds a KMS-specific metadata.

Review comment:
       Suggest adding this to the comment: "Note: this function might be called by multiple threads"




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498154775



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {
+ public:
+  /// KmsClientFactory must be registered before calling
+  /// GetFileEncryptionProperties()/GetFileDecryptionProperties() methods.
+  void RegisterKmsClientFactory(std::shared_ptr<KmsClientFactory> kms_client_factory);
+
+  std::shared_ptr<FileEncryptionProperties> GetFileEncryptionProperties(
+      const KmsConnectionConfig& kms_connection_config,
+      std::shared_ptr<EncryptionConfiguration> encryption_config);
+
+  /// The returned FileDecryptionProperties object will use the cache inside this
+  /// PropertiesDrivenCryptoFactory object, so please keep this
+  /// PropertiesDrivenCryptoFactory object alive along with the returned
+  /// FileDecryptionProperties object.
+  std::shared_ptr<FileDecryptionProperties> GetFileDecryptionProperties(
+      const KmsConnectionConfig& kms_connection_config,
+      std::shared_ptr<DecryptionConfiguration> decryption_config);
+
+  void RemoveCacheEntriesForToken(const std::string& access_token) {
+    key_toolkit_.RemoveCacheEntriesForToken(access_token);
+  }
+
+  void RemoveCacheEntriesForAllTokens() { key_toolkit_.RemoveCacheEntriesForAllTokens(); }
+
+ private:
+  /// Acceptable data key lengths in number of bits
+  static constexpr const int32_t kAcceptableDataKeyLengths[] = {128, 192, 256};

Review comment:
       No, the order doesn't matter. Yes, it's just a set of constants to lookup.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/remote_kms_client.cc
##########
@@ -0,0 +1,127 @@
+// 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.
+
+#include "arrow/json/object_parser.h"
+#include "arrow/json/object_writer.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/remote_kms_client.h"
+#include "parquet/exception.h"
+
+using arrow::json::ObjectParser;
+using arrow::json::ObjectWriter;
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char RemoteKmsClient::kLocalWrapNoKeyVersion[];
+
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapKeyVersionField[];
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapEncryptedKeyField[];
+
+RemoteKmsClient::LocalKeyWrap::LocalKeyWrap(const std::string& master_key_version,
+                                            const std::string& encrypted_encoded_key)
+    : encrypted_encoded_key_(encrypted_encoded_key),
+      master_key_version_(master_key_version) {}
+
+std::string RemoteKmsClient::LocalKeyWrap::CreateSerialized(
+    const std::string& encrypted_encoded_key) {
+  ObjectWriter json_writer;
+
+  json_writer.SetString(kLocalWrapKeyVersionField, kLocalWrapNoKeyVersion);
+  json_writer.SetString(kLocalWrapEncryptedKeyField, encrypted_encoded_key);
+
+  return json_writer.Serialize();
+}
+
+RemoteKmsClient::LocalKeyWrap RemoteKmsClient::LocalKeyWrap::Parse(
+    const std::string& wrapped_key) {
+  ObjectParser json_parser;
+  if (!json_parser.Parse(wrapped_key)) {
+    throw ParquetException("Failed to parse local key wrap json " + wrapped_key);
+  }
+  std::string master_key_version;
+  PARQUET_ASSIGN_OR_THROW(master_key_version,
+                          json_parser.GetString(kLocalWrapKeyVersionField));
+
+  std::string encrypted_encoded_key;
+  PARQUET_ASSIGN_OR_THROW(encrypted_encoded_key,
+                          json_parser.GetString(kLocalWrapEncryptedKeyField));
+
+  return RemoteKmsClient::LocalKeyWrap(master_key_version, encrypted_encoded_key);
+}
+
+void RemoteKmsClient::Initialize(const KmsConnectionConfig& kms_connection_config,
+                                 bool is_wrap_locally) {
+  kms_connection_config_ = kms_connection_config;
+  is_wrap_locally_ = is_wrap_locally;
+  if (is_wrap_locally_) {
+    master_key_cache_.Clear();
+  }
+
+  is_default_token_ =
+      kms_connection_config_.key_access_token() == KmsClient::kKeyAccessTokenDefault;
+
+  InitializeInternal();
+}
+
+std::string RemoteKmsClient::WrapKey(const std::string& key_bytes,
+                                     const std::string& master_key_identifier) {
+  if (is_wrap_locally_) {

Review comment:
       This is a public API, so it would be good to keep it as similar to the merged parquet-mr version as possible. Also, the names like LocalWrappingRemoteKmsClient and InServerWrappingRemoteKmsClient don't sound great for a public API, and can be confusing.. The current RemoteKmsClient (with the default value of false for the `is_wrap_locally` parameter) is easier for a user.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first part.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public API.

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first part.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public API.

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first part.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public API.

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first part.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public API.

##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first part.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public 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.

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



[GitHub] [arrow] thamht4190 commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-716594944


   @pitrou @bkietz I've created a small pull request from this pull: https://github.com/apache/arrow/pull/8375. Can you take a look on it? If you see this pull is good to start, I will create another small pulls? cc @ggershinsky 


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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r579594227



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {

Review comment:
       What about `PropertiesCryptoFactory`?




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



[GitHub] [arrow] ggershinsky commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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


   @pitrou My personal preference is the latter - merging this pr and letting the users of this code to be mindful of potential issues. This is safer than proceeding with the current low-level interface work, which has many more security and other issues. When this pr is merged, it will unblock the users who'd create the high-level interface in Python and address the API point (I'll add a link to this thread in the Python design doc; the Python API is to be reviewed; the C++ API could be synced accordingly). 
   
   Regarding the second point - we'll be testing the interop with the Java counterpart, to make sure the spec is implemented properly. Regarding the third point, I think the code is safe, as per our discussion in that jira.


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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r479632859



##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       Sorry. I don't get this point. Can you explain?




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



[GitHub] [arrow] ggershinsky commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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


   > > What about keeping this PR to see an overview, I'll create some smaller PRs to review?
   > 
   > FWIW, I think that using JIRA would be more accessible for overview purposes:
   > 
   > * Make [ARROW-9318](https://issues.apache.org/jira/browse/ARROW-9318) into a tracking issue for the JIRAs of the smaller PRs
   > * Paste subsets of the design doc into each of the JIRAs, supplemented by a summary of the interfaces/classes which will be added to address each subset of the requirements
   
   Just to sync on the expectations, before @thamht4190 goes down this path. Decomposing the design into sublayers, each with a fully functioning implementation, built upon each other, looks like a huge effort to me. Doing this to make the review easier is not necessary - if only because the previous pull request (low level encryption) was larger than this one, and has been reviewed/merged as one piece. My understanding of Tham's offer was splitting this PR into a number of source file groups, without a corresponding design subset. Moreover, since making each sub-PR pass the Travis test would require a very significant new dev effort - I'd suggest reviewing each PR without requiring it to pass Travis. Only the overall PR will have to pass the tests. (the overall PR would be updated with each commit to every sub-PR).


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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498079624



##########
File path: cpp/src/parquet/encryption/key_metadata.h
##########
@@ -0,0 +1,92 @@
+// 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 <string>
+
+#include "arrow/util/variant.h"
+
+#include "parquet/encryption/key_material.h"
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer. The "key metadata" field is made available to file readers
+// to enable recovery of the key. This interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows integration with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details). This class is implemented to support version 1 of the parquet key
+// management tools specification.
+//
+// KeyMetadata writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material.
+// 2. "internalStorage" - a boolean. If true, means that "key material" is kept inside the
+// "key metadata" field. If false, "key material" is kept externally (outside Parquet
+// files) - in this case, "key metadata" keeps a reference to the external "key material".
+// 3. "keyReference" - a String, with the reference to the external "key material".
+// Written only if internalStorage is false.
+//
+// If internalStorage is true, "key material" is a part of "key metadata", and the json
+// keeps additional fields, described in the KeyMaterial class.
+class PARQUET_EXPORT KeyMetadata {
+ public:
+  static constexpr const char kKeyMaterialInternalStorageField[] = "internalStorage";
+  static constexpr const char kKeyReferenceField[] = "keyReference";

Review comment:
       Ok, thanks.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/remote_kms_client.cc
##########
@@ -0,0 +1,127 @@
+// 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.
+
+#include "arrow/json/object_parser.h"
+#include "arrow/json/object_writer.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/remote_kms_client.h"
+#include "parquet/exception.h"
+
+using arrow::json::ObjectParser;
+using arrow::json::ObjectWriter;
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char RemoteKmsClient::kLocalWrapNoKeyVersion[];
+
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapKeyVersionField[];
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapEncryptedKeyField[];
+
+RemoteKmsClient::LocalKeyWrap::LocalKeyWrap(const std::string& master_key_version,
+                                            const std::string& encrypted_encoded_key)
+    : encrypted_encoded_key_(encrypted_encoded_key),
+      master_key_version_(master_key_version) {}
+
+std::string RemoteKmsClient::LocalKeyWrap::CreateSerialized(
+    const std::string& encrypted_encoded_key) {
+  ObjectWriter json_writer;
+
+  json_writer.SetString(kLocalWrapKeyVersionField, kLocalWrapNoKeyVersion);
+  json_writer.SetString(kLocalWrapEncryptedKeyField, encrypted_encoded_key);
+
+  return json_writer.Serialize();
+}
+
+RemoteKmsClient::LocalKeyWrap RemoteKmsClient::LocalKeyWrap::Parse(
+    const std::string& wrapped_key) {
+  ObjectParser json_parser;
+  if (!json_parser.Parse(wrapped_key)) {
+    throw ParquetException("Failed to parse local key wrap json " + wrapped_key);
+  }
+  std::string master_key_version;
+  PARQUET_ASSIGN_OR_THROW(master_key_version,
+                          json_parser.GetString(kLocalWrapKeyVersionField));
+
+  std::string encrypted_encoded_key;
+  PARQUET_ASSIGN_OR_THROW(encrypted_encoded_key,
+                          json_parser.GetString(kLocalWrapEncryptedKeyField));
+
+  return RemoteKmsClient::LocalKeyWrap(master_key_version, encrypted_encoded_key);
+}
+
+void RemoteKmsClient::Initialize(const KmsConnectionConfig& kms_connection_config,
+                                 bool is_wrap_locally) {
+  kms_connection_config_ = kms_connection_config;
+  is_wrap_locally_ = is_wrap_locally;
+  if (is_wrap_locally_) {
+    master_key_cache_.Clear();
+  }
+
+  is_default_token_ =
+      kms_connection_config_.key_access_token() == KmsClient::kKeyAccessTokenDefault;
+
+  InitializeInternal();
+}
+
+std::string RemoteKmsClient::WrapKey(const std::string& key_bytes,
+                                     const std::string& master_key_identifier) {
+  if (is_wrap_locally_) {

Review comment:
       Yep, not sure how common this is. Some large organizations work with both Spark/Java and Arrow/C++ libraries (I know ours does), then it's easier to design the data pipelines if the API is similar. 




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r479626558



##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());

Review comment:
       > I thought there was a document parser that takes a string pointer and a size, that is probably better to use if it is available.
   
   I've seen something for json (under `arrow/json` folder), but they seems not fit to the usage here.
   
   > Is the json allowed to be utf-8 encoded?
   Do you mean if rapidjson supports utf-8 encodings? If so, yes.

##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());

Review comment:
       > I thought there was a document parser that takes a string pointer and a size, that is probably better to use if it is available.
   
   I've seen something for json (under `arrow/json` folder), but they seems not fit to the usage here.
   
   > Is the json allowed to be utf-8 encoded?
   
   Do you mean if rapidjson supports utf-8 encodings? If so, yes.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer

Review comment:
       It would be useful to spell out the default values in the public API documentation.  E.g.,
   ". The default is false - files are written with encrypted footer".

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key

Review comment:
       please add "all" - "encrypt footer and all columns with the same encryption key"

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).

Review comment:
       "The default is 600 (10 minutes)"

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).

Review comment:
       "The default is 600 (10 minutes)"

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.

Review comment:
       "The default is 128 bits"

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,205 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and columns with the same encryption key
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.

Review comment:
       "The default is true. If set to false, use single wrapping - where DEKs are encrypted directly with master keys"




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498676039



##########
File path: cpp/src/parquet/encryption/test_encryption_util.h
##########
@@ -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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#pragma once
+
+#include <algorithm>
+#include <chrono>
+#include <limits>
+#include <memory>
+#include <random>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/status.h"
+#include "arrow/testing/util.h"
+#include "arrow/util/io_util.h"
+
+#include "parquet/column_page.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
+#include "parquet/encoding.h"
+#include "parquet/encryption/encryption.h"
+#include "parquet/platform.h"
+#include "parquet/test_util.h"
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+using arrow::internal::TemporaryDir;
+
+using parquet::ConvertedType;
+using parquet::Repetition;
+using parquet::Type;
+using schema::GroupNode;
+using schema::NodePtr;
+using schema::PrimitiveNode;
+
+constexpr int kFixedLength = 10;
+
+const char kFooterEncryptionKey[] = "0123456789012345";  // 128bit/16
+const char kColumnEncryptionKey1[] = "1234567890123450";
+const char kColumnEncryptionKey2[] = "1234567890123451";
+const char kFileName[] = "tester";
+
+inline std::string data_file(const char* file) {
+  std::string dir_string(parquet::test::get_data_dir());
+  std::stringstream ss;
+  ss << dir_string << "/" << file;
+  return ss.str();
+}
+
+// A temporary directory that contains the encrypted files generated in the tests.
+extern std::unique_ptr<TemporaryDir> temp_dir;
+
+inline arrow::Result<std::unique_ptr<TemporaryDir>> temp_data_dir() {
+  arrow::Result<std::unique_ptr<TemporaryDir>> dir;
+  ARROW_ASSIGN_OR_RAISE(dir, TemporaryDir::Make("parquet-encryption-test-"));
+  return dir;
+}
+
+static constexpr const char kDoubleFieldName[] = "double_field";
+static constexpr const char kFloatFieldName[] = "float_field";
+static constexpr const char kBooleanFieldName[] = "boolean_field";
+static constexpr const char kInt32FieldName[] = "int32_field";
+static constexpr const char kInt64FieldName[] = "int64_field";
+static constexpr const char kInt96FieldName[] = "int96_field";
+static constexpr const char kByteArrayFieldName[] = "ba_field";
+static constexpr const char kFixedLenByteArrayFieldName[] = "flba_field";
+
+const char kFooterMasterKey[] = "0123456789112345";

Review comment:
       No special reason. I changed all of them to `const char`.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498102558



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;

Review comment:
       Actually I don't see any advantage to use `Builder` pattern compared to a simple struct in this case. I would love to move to a simple struct then.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public 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.

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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r479632794



##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());
+
+  if (document.HasParseError() || !document.IsObject()) {
+    throw ParquetException("Failed to parse key metadata " + key_material_string);
+  }
+
+  // External key material - extract "key material type", and make sure it is supported
+  std::string key_material_type = document[KEY_MATERIAL_TYPE_FIELD].GetString();
+  if (KEY_MATERIAL_TYPE1 != key_material_type) {
+    throw ParquetException("Wrong key material type: " + key_material_type + " vs " +
+                           KEY_MATERIAL_TYPE1);
+  }
+  // Parse other fields (common to internal and external key material)
+  return Parse(document);
+}
+
+KeyMaterial KeyMaterial::Parse(const rapidjson::Document& key_material_json) {
+  // 2. Check if "key material" belongs to file footer key
+  bool is_footer_key = key_material_json[IS_FOOTER_KEY_FIELD].GetBool();

Review comment:
       I should add more validation before passing.

##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());
+
+  if (document.HasParseError() || !document.IsObject()) {
+    throw ParquetException("Failed to parse key metadata " + key_material_string);
+  }
+
+  // External key material - extract "key material type", and make sure it is supported
+  std::string key_material_type = document[KEY_MATERIAL_TYPE_FIELD].GetString();
+  if (KEY_MATERIAL_TYPE1 != key_material_type) {
+    throw ParquetException("Wrong key material type: " + key_material_type + " vs " +
+                           KEY_MATERIAL_TYPE1);
+  }
+  // Parse other fields (common to internal and external key material)
+  return Parse(document);
+}
+
+KeyMaterial KeyMaterial::Parse(const rapidjson::Document& key_material_json) {
+  // 2. Check if "key material" belongs to file footer key
+  bool is_footer_key = key_material_json[IS_FOOTER_KEY_FIELD].GetBool();

Review comment:
       I should add more validation before parsing.




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



[GitHub] [arrow] emkornfield commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r475351570



##########
File path: cpp/src/parquet/test_encryption_util.h
##########
@@ -65,5 +66,36 @@ inline std::string data_file(const char* file) {
   return ss.str();
 }
 
+static constexpr char DOUBLE_FIELD_NAME[] = "double_field";
+static constexpr char FLOAT_FIELD_NAME[] = "float_field";
+static constexpr char BOOLEAN_FIELD_NAME[] = "boolean_field";
+static constexpr char INT32_FIELD_NAME[] = "int32_field";
+static constexpr char INT64_FIELD_NAME[] = "int64_field";
+static constexpr char INT96_FIELD_NAME[] = "int96_field";
+static constexpr char BA_FIELD_NAME[] = "ba_field";
+static constexpr char FLBA_FIELD_NAME[] = "flba_field";
+
+class FileEncryptor {

Review comment:
       docs please.

##########
File path: cpp/src/parquet/CMakeLists.txt
##########
@@ -363,6 +376,12 @@ if(PARQUET_REQUIRE_ENCRYPTION)
                    encryption_write_configurations_test.cc
                    encryption_read_configurations_test.cc
                    encryption_properties_test.cc
+                   test_encryption_util.cc
+                   test_util.cc)
+  add_parquet_test(encryption-key-management-test

Review comment:
       nit: add blank line

##########
File path: cpp/src/parquet/encryption.h
##########
@@ -47,15 +47,15 @@ using ColumnPathToEncryptionPropertiesMap =
 
 class PARQUET_EXPORT DecryptionKeyRetriever {
  public:
-  virtual std::string GetKey(const std::string& key_metadata) const = 0;
+  virtual std::string GetKey(const std::string& key_metadata) = 0;
   virtual ~DecryptionKeyRetriever() {}
 };
 
 /// Simple integer key retriever
 class PARQUET_EXPORT IntegerKeyIdRetriever : public DecryptionKeyRetriever {
  public:
   void PutKey(uint32_t key_id, const std::string& key);
-  std::string GetKey(const std::string& key_metadata) const;
+  std::string GetKey(const std::string& key_metadata);

Review comment:
       why this removal?

##########
File path: cpp/src/parquet/encryption_internal.h
##########
@@ -45,6 +45,9 @@ constexpr int8_t kOffsetIndex = 7;
 /// Performs AES encryption operations with GCM or CTR ciphers.
 class AesEncryptor {
  public:
+  /// Can serve one key length only. Possible values: 16, 24, 32 bytes.
+  explicit AesEncryptor(ParquetCipher::type alg_id, int key_len, bool metadata);

Review comment:
       why isn't static factory sufficient?




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r579596103



##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED
+  fields.push_back(PrimitiveNode::Make(kBooleanFieldName, Repetition::REQUIRED,
+                                       Type::BOOLEAN, ConvertedType::NONE));
+
+  // Create a primitive node named 'int32_field' with type:INT32, repetition:REQUIRED,
+  // logical type:TIME_MILLIS
+  fields.push_back(PrimitiveNode::Make(kInt32FieldName, Repetition::REQUIRED, Type::INT32,
+                                       ConvertedType::TIME_MILLIS));
+
+  // Create a primitive node named 'int64_field' with type:INT64, repetition:REPEATED
+  fields.push_back(PrimitiveNode::Make(kInt64FieldName, Repetition::REPEATED, Type::INT64,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kInt96FieldName, Repetition::REQUIRED, Type::INT96,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kFloatFieldName, Repetition::REQUIRED, Type::FLOAT,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kDoubleFieldName, Repetition::REQUIRED,
+                                       Type::DOUBLE, ConvertedType::NONE));
+
+  // Create a primitive node named 'ba_field' with type:BYTE_ARRAY, repetition:OPTIONAL
+  fields.push_back(PrimitiveNode::Make(kByteArrayFieldName, Repetition::OPTIONAL,
+                                       Type::BYTE_ARRAY, ConvertedType::NONE));
+
+  // Create a primitive node named 'flba_field' with type:FIXED_LEN_BYTE_ARRAY,
+  // repetition:REQUIRED, field_length = kFixedLength
+  fields.push_back(PrimitiveNode::Make(kFixedLenByteArrayFieldName, Repetition::REQUIRED,
+                                       Type::FIXED_LEN_BYTE_ARRAY, ConvertedType::NONE,
+                                       kFixedLength));
+
+  // Create a GroupNode named 'schema' using the primitive nodes defined above
+  // This GroupNode is the root node of the schema tree
+  return std::static_pointer_cast<GroupNode>(
+      GroupNode::Make("schema", Repetition::REQUIRED, fields));
+}
+
+void FileEncryptor::EncryptFile(
+    std::string file,
+    std::shared_ptr<parquet::FileEncryptionProperties> encryption_configurations) {
+  WriterProperties::Builder prop_builder;
+  prop_builder.compression(parquet::Compression::SNAPPY);
+  prop_builder.encryption(encryption_configurations);
+  std::shared_ptr<WriterProperties> writer_properties = prop_builder.build();
+
+  PARQUET_ASSIGN_OR_THROW(auto out_file, FileClass::Open(file));
+  // Create a ParquetFileWriter instance
+  std::shared_ptr<parquet::ParquetFileWriter> file_writer =
+      parquet::ParquetFileWriter::Open(out_file, schema_, writer_properties);
+
+  for (int r = 0; r < num_rgs; r++) {
+    bool buffered_mode = r % 2 == 0;
+    auto row_group_writer = buffered_mode ? file_writer->AppendBufferedRowGroup()
+                                          : file_writer->AppendRowGroup();
+
+    int column_index = 0;
+    // Captures i by reference; increments it by one
+    auto get_next_column = [&]() {
+      return buffered_mode ? row_group_writer->column(column_index++)
+                           : row_group_writer->NextColumn();
+    };
+
+    // Write the Bool column
+    parquet::BoolWriter* bool_writer =
+        static_cast<parquet::BoolWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      bool value = ((i % 2) == 0) ? true : false;
+      bool_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Int32 column
+    parquet::Int32Writer* int32_writer =
+        static_cast<parquet::Int32Writer*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      int32_t value = i;
+      int32_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Int64 column. Each row has repeats twice.
+    parquet::Int64Writer* int64_writer =
+        static_cast<parquet::Int64Writer*>(get_next_column());
+    for (int i = 0; i < 2 * rows_per_rowgroup_; i++) {
+      int64_t value = i * 1000 * 1000;
+      value *= 1000 * 1000;
+      int16_t definition_level = 1;
+      int16_t repetition_level = 0;
+      if ((i % 2) == 0) {
+        repetition_level = 1;  // start of a new record
+      }
+      int64_writer->WriteBatch(1, &definition_level, &repetition_level, &value);
+    }
+
+    // Write the INT96 column.
+    parquet::Int96Writer* int96_writer =
+        static_cast<parquet::Int96Writer*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::Int96 value;
+      value.value[0] = i;
+      value.value[1] = i + 1;
+      value.value[2] = i + 2;
+      int96_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Float column
+    parquet::FloatWriter* float_writer =
+        static_cast<parquet::FloatWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      float value = static_cast<float>(i) * 1.1f;
+      float_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Double column
+    parquet::DoubleWriter* double_writer =
+        static_cast<parquet::DoubleWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      double value = i * 1.1111111;
+      double_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the ByteArray column. Make every alternate values NULL
+    parquet::ByteArrayWriter* ba_writer =
+        static_cast<parquet::ByteArrayWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::ByteArray value;
+      char hello[kFixedLength] = "parquet";
+      hello[7] = static_cast<char>(static_cast<int>('0') + i / 100);
+      hello[8] = static_cast<char>(static_cast<int>('0') + (i / 10) % 10);
+      hello[9] = static_cast<char>(static_cast<int>('0') + i % 10);
+      if (i % 2 == 0) {
+        int16_t definition_level = 1;
+        value.ptr = reinterpret_cast<const uint8_t*>(&hello[0]);
+        value.len = kFixedLength;
+        ba_writer->WriteBatch(1, &definition_level, nullptr, &value);
+      } else {
+        int16_t definition_level = 0;
+        ba_writer->WriteBatch(1, &definition_level, nullptr, nullptr);
+      }
+    }
+
+    // Write the FixedLengthByteArray column
+    parquet::FixedLenByteArrayWriter* flba_writer =
+        static_cast<parquet::FixedLenByteArrayWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::FixedLenByteArray value;
+      char v = static_cast<char>(i);
+      char flba[kFixedLength] = {v, v, v, v, v, v, v, v, v, v};
+      value.ptr = reinterpret_cast<const uint8_t*>(&flba[0]);
+      flba_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+  }
+
+  // Close the ParquetFileWriter
+  file_writer->Close();
+
+  return;
+}
+
+void FileDecryptor::DecryptFile(
+    std::string file,
+    std::shared_ptr<FileDecryptionProperties> file_decryption_properties) {
+  std::string exception_msg;
+  parquet::ReaderProperties reader_properties = parquet::default_reader_properties();
+  if (file_decryption_properties) {
+    reader_properties.file_decryption_properties(file_decryption_properties->DeepClone());
+  }
+
+  auto file_reader = parquet::ParquetFileReader::OpenFile(file, false, reader_properties);
+
+  // Get the File MetaData
+  std::shared_ptr<parquet::FileMetaData> file_metadata = file_reader->metadata();
+
+  // Get the number of RowGroups
+  int num_row_groups = file_metadata->num_row_groups();
+
+  // Get the number of Columns
+  int num_columns = file_metadata->num_columns();
+  ASSERT_EQ(num_columns, 8);
+
+  // Iterate over all the RowGroups in the file
+  for (int r = 0; r < num_row_groups; ++r) {
+    // Get the RowGroup Reader
+    std::shared_ptr<parquet::RowGroupReader> row_group_reader = file_reader->RowGroup(r);
+
+    // Get the RowGroupMetaData
+    std::unique_ptr<RowGroupMetaData> rg_metadata = file_metadata->RowGroup(r);
+
+    int64_t values_read = 0;
+    int64_t rows_read = 0;
+    int16_t definition_level;
+    int16_t repetition_level;
+    int i;
+    std::shared_ptr<parquet::ColumnReader> column_reader;
+
+    // Get the Column Reader for the boolean column
+    column_reader = row_group_reader->Column(0);
+    parquet::BoolReader* bool_reader =
+        static_cast<parquet::BoolReader*>(column_reader.get());
+
+    // Get the ColumnChunkMetaData for the boolean column
+    std::unique_ptr<ColumnChunkMetaData> boolean_md = rg_metadata->ColumnChunk(0);
+
+    // Read all the rows in the column
+    i = 0;
+    while (bool_reader->HasNext()) {
+      bool value;
+      // Read one value at a time. The number of rows read is returned. values_read
+      // contains the number of non-null rows
+      rows_read = bool_reader->ReadBatch(1, nullptr, nullptr, &value, &values_read);

Review comment:
       I did it for other types, except `bool` and byte array.




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486518974



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {

Review comment:
       You can use forward declarations to avoid exposing class definitions when only a reference or pointer is given.
   
   Also, it is good idea to hide implementation details by using the pimpl idiom. Another advantage is limit the growth of compile times.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {

Review comment:
       We'll 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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486367648



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;

Review comment:
       @ggershinsky I worry that if the cache is a member of `PropertiesDrivenCryptoFactory`, users of API must keep the life time of `PropertiesDrivenCryptoFactory` object after getting `FileEncryptionProperties`/`FileDecryptionProperties`. If they forget that, it will be easy to get a crash in their programs.
   I prefer to let users manage the life time of the cache themselves and pass it into `PropertiesDrivenCryptoFactory`.
   Any final decision?




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;

Review comment:
       if this allows a user to set some attributes upon struct "constructions", while the rest of attributes will be automatically set to the default values - then this seems to be as good as builder's immutability stuff. There is an additional builder feature that allows to verify that mandatory parameters are set by the user - but that can be also checked by consumers of the struct. In any case, using a struct sounds good.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r579595188



##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED
+  fields.push_back(PrimitiveNode::Make(kBooleanFieldName, Repetition::REQUIRED,
+                                       Type::BOOLEAN, ConvertedType::NONE));
+
+  // Create a primitive node named 'int32_field' with type:INT32, repetition:REQUIRED,
+  // logical type:TIME_MILLIS
+  fields.push_back(PrimitiveNode::Make(kInt32FieldName, Repetition::REQUIRED, Type::INT32,
+                                       ConvertedType::TIME_MILLIS));
+
+  // Create a primitive node named 'int64_field' with type:INT64, repetition:REPEATED
+  fields.push_back(PrimitiveNode::Make(kInt64FieldName, Repetition::REPEATED, Type::INT64,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kInt96FieldName, Repetition::REQUIRED, Type::INT96,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kFloatFieldName, Repetition::REQUIRED, Type::FLOAT,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kDoubleFieldName, Repetition::REQUIRED,
+                                       Type::DOUBLE, ConvertedType::NONE));
+
+  // Create a primitive node named 'ba_field' with type:BYTE_ARRAY, repetition:OPTIONAL
+  fields.push_back(PrimitiveNode::Make(kByteArrayFieldName, Repetition::OPTIONAL,
+                                       Type::BYTE_ARRAY, ConvertedType::NONE));
+
+  // Create a primitive node named 'flba_field' with type:FIXED_LEN_BYTE_ARRAY,
+  // repetition:REQUIRED, field_length = kFixedLength
+  fields.push_back(PrimitiveNode::Make(kFixedLenByteArrayFieldName, Repetition::REQUIRED,
+                                       Type::FIXED_LEN_BYTE_ARRAY, ConvertedType::NONE,
+                                       kFixedLength));

Review comment:
       These code are not new, I only moved it to a new file inside `encryption/` folder, and reuse it for key management test.
   These input/output files of this test has been used in interop test with java, so the scheme should be kept unchanged.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r479633600



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {

Review comment:
       > Does this need to be exported?
   
   No, it's used internally in FileKeyUnwrapper.cc (implemented) and KeyToolkit.cc (will be implemented later).




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first 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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498079098



##########
File path: cpp/src/parquet/encryption/kms_client.h
##########
@@ -0,0 +1,84 @@
+// 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 <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class wraps the key access token of a KMS server. If your token changes over time,
+// you should keep the reference to the KeyAccessToken object and call Refresh() method
+// every time you have a new token.
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();

Review comment:
       Hmm... is it useful to have a _method_ to do this? Normally, a default value should be used... by default (i.e. if I don't set a value, use the default).




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r479633145



##########
File path: cpp/src/parquet/kms_client.h
##########
@@ -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.
+
+#pragma once
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;
+  std::unordered_map<std::string, std::string> custom_kms_conf;
+
+  const std::string& key_access_token() const {
+    if (refreshable_key_access_token == NULL ||
+        refreshable_key_access_token->value().empty()) {
+      throw ParquetException("key access token is not set!");
+    }
+    return refreshable_key_access_token->value();
+  }
+
+  void SetDefaultIfEmpty();
+};
+
+class PARQUET_EXPORT KmsClient {
+ public:
+  static constexpr char KMS_INSTANCE_ID_DEFAULT[] = "DEFAULT";
+  static constexpr char KMS_INSTANCE_URL_DEFAULT[] = "DEFAULT";
+  static constexpr char KEY_ACCESS_TOKEN_DEFAULT[] = "DEFAULT";
+
+  // Wraps a key - encrypts it with the master key, encodes the result
+  // and potentially adds a KMS-specific metadata.
+  virtual std::string WrapKey(const std::string& key_bytes,
+                              const std::string& master_key_identifier) = 0;

Review comment:
       @ggershinsky I always see `master_key_identifier` goes with a `key`, so should we pass an input param `const KeyWithMasterId& key_with_master_id` instead of 2 input string params? 




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



[GitHub] [arrow] pitrou commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-686599596


   Just a drive-by comment for now, but given the number of files added I think we should create a `src/parquet/encryption` subdirectory.


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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {

Review comment:
       sorry for the delay, just back from a vacation. KeyToolkit is a utility that keeps various tools for key management (such as key rotation, kms client instantiation, cache control, etc), plus a number of auxiliary classes for internal use.

##########
File path: cpp/src/parquet/kms_client.h
##########
@@ -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.
+
+#pragma once
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;
+  std::unordered_map<std::string, std::string> custom_kms_conf;
+
+  const std::string& key_access_token() const {
+    if (refreshable_key_access_token == NULL ||
+        refreshable_key_access_token->value().empty()) {
+      throw ParquetException("key access token is not set!");
+    }
+    return refreshable_key_access_token->value();
+  }
+
+  void SetDefaultIfEmpty();
+};
+
+class PARQUET_EXPORT KmsClient {
+ public:
+  static constexpr char KMS_INSTANCE_ID_DEFAULT[] = "DEFAULT";
+  static constexpr char KMS_INSTANCE_URL_DEFAULT[] = "DEFAULT";
+  static constexpr char KEY_ACCESS_TOKEN_DEFAULT[] = "DEFAULT";
+
+  // Wraps a key - encrypts it with the master key, encodes the result
+  // and potentially adds a KMS-specific metadata.
+  virtual std::string WrapKey(const std::string& key_bytes,
+                              const std::string& master_key_identifier) = 0;

Review comment:
       @thamht4190 this is a public interface, so better off with standard string params. KeyWithMasterId is an auxiliary class for internal usage. Also, there is a value of keeping the C++ public API identical to the Java public API, to make the interop easier in the future. 




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r582057448



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {

Review comment:
       `CryptoFactory` sounds good. You can also add a simple docstring explaining what you just said :-)




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



[GitHub] [arrow] ggershinsky commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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


   Thanks! Yep, I think it's a good idea to mark them as experimental, there is a chance they'll be updated in the course of the Python API work.


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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497980169



##########
File path: cpp/src/parquet/encryption/key_toolkit_internal.h
##########
@@ -0,0 +1,52 @@
+// 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 <chrono>
+#include <string>
+
+namespace parquet {
+namespace encryption {
+namespace internal {
+
+// "data encryption key" and "master key identifier" are paired together as output when
+// parsing from "key material"
+class KeyWithMasterId {
+ public:
+  KeyWithMasterId(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  const std::string key_bytes_;
+  const std::string master_id_;
+};
+
+/// Encrypts "key" with "master_key", using AES-GCM and the "aad"

Review comment:
       cc @ggershinsky 




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497973635



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {
+ public:
+  /// KmsClientFactory must be registered before calling
+  /// GetFileEncryptionProperties()/GetFileDecryptionProperties() methods.
+  void RegisterKmsClientFactory(std::shared_ptr<KmsClientFactory> kms_client_factory);
+
+  std::shared_ptr<FileEncryptionProperties> GetFileEncryptionProperties(
+      const KmsConnectionConfig& kms_connection_config,
+      std::shared_ptr<EncryptionConfiguration> encryption_config);
+
+  /// The returned FileDecryptionProperties object will use the cache inside this
+  /// PropertiesDrivenCryptoFactory object, so please keep this
+  /// PropertiesDrivenCryptoFactory object alive along with the returned
+  /// FileDecryptionProperties object.
+  std::shared_ptr<FileDecryptionProperties> GetFileDecryptionProperties(
+      const KmsConnectionConfig& kms_connection_config,
+      std::shared_ptr<DecryptionConfiguration> decryption_config);
+
+  void RemoveCacheEntriesForToken(const std::string& access_token) {
+    key_toolkit_.RemoveCacheEntriesForToken(access_token);
+  }
+
+  void RemoveCacheEntriesForAllTokens() { key_toolkit_.RemoveCacheEntriesForAllTokens(); }
+
+ private:
+  /// Acceptable data key lengths in number of bits
+  static constexpr const int32_t kAcceptableDataKeyLengths[] = {128, 192, 256};

Review comment:
       Sorry, I don't understand your question. Can you explain?




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498076058



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {
+ public:
+  /// KmsClientFactory must be registered before calling

Review comment:
       Hmm, sorry, I mean: what does it imply exactly to register a KmsClientFactory? Does it associate it with a global name? Something else?




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498685819



##########
File path: cpp/src/parquet/encryption/remote_kms_client.cc
##########
@@ -0,0 +1,127 @@
+// 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.
+
+#include "arrow/json/object_parser.h"
+#include "arrow/json/object_writer.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/remote_kms_client.h"
+#include "parquet/exception.h"
+
+using arrow::json::ObjectParser;
+using arrow::json::ObjectWriter;
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char RemoteKmsClient::kLocalWrapNoKeyVersion[];
+
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapKeyVersionField[];
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapEncryptedKeyField[];
+
+RemoteKmsClient::LocalKeyWrap::LocalKeyWrap(const std::string& master_key_version,
+                                            const std::string& encrypted_encoded_key)
+    : encrypted_encoded_key_(encrypted_encoded_key),
+      master_key_version_(master_key_version) {}
+
+std::string RemoteKmsClient::LocalKeyWrap::CreateSerialized(
+    const std::string& encrypted_encoded_key) {
+  ObjectWriter json_writer;
+
+  json_writer.SetString(kLocalWrapKeyVersionField, kLocalWrapNoKeyVersion);
+  json_writer.SetString(kLocalWrapEncryptedKeyField, encrypted_encoded_key);
+
+  return json_writer.Serialize();
+}
+
+RemoteKmsClient::LocalKeyWrap RemoteKmsClient::LocalKeyWrap::Parse(
+    const std::string& wrapped_key) {
+  ObjectParser json_parser;
+  if (!json_parser.Parse(wrapped_key)) {
+    throw ParquetException("Failed to parse local key wrap json " + wrapped_key);
+  }
+  std::string master_key_version;
+  PARQUET_ASSIGN_OR_THROW(master_key_version,
+                          json_parser.GetString(kLocalWrapKeyVersionField));
+
+  std::string encrypted_encoded_key;
+  PARQUET_ASSIGN_OR_THROW(encrypted_encoded_key,
+                          json_parser.GetString(kLocalWrapEncryptedKeyField));
+
+  return RemoteKmsClient::LocalKeyWrap(master_key_version, encrypted_encoded_key);
+}
+
+void RemoteKmsClient::Initialize(const KmsConnectionConfig& kms_connection_config,
+                                 bool is_wrap_locally) {
+  kms_connection_config_ = kms_connection_config;
+  is_wrap_locally_ = is_wrap_locally;
+  if (is_wrap_locally_) {
+    master_key_cache_.Clear();
+  }
+
+  is_default_token_ =
+      kms_connection_config_.key_access_token() == KmsClient::kKeyAccessTokenDefault;
+
+  InitializeInternal();
+}
+
+std::string RemoteKmsClient::WrapKey(const std::string& key_bytes,
+                                     const std::string& master_key_identifier) {
+  if (is_wrap_locally_) {

Review comment:
       Right @pitrou. I was over-thought.
   @ggershinsky  Actually, as a C++ developer, when looking at `RemoteKmsClient` class implementation:
   * If I'm going to write in-server wrapping kms client, I can see that `RemoteKmsClient` doesn't help much (and also has a lot of local wrapping stuff), I may choose to write `MyRemoteKmsClient` which inherits from `KmsClient`, not `RemoteKmsClient`. There are 2 advantages I can see: reducing one level of inheritance and a lighter-weight class.
   * If I'm going to write local wrapping kms client, I still need to implement `WrapKeyInServer` and `UnwrapKeyInServer` (even I can let them empty methods).
   As you said to me, users are going to use either local wrapping or in-server wrapping in their system, not both at the same time. So current implementation of `RemoteKmsClient` doesn't really make sense to me. Will you reconsider at this point?




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



[GitHub] [arrow] emkornfield commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r482714047



##########
File path: cpp/src/parquet/kms_client.h
##########
@@ -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.
+
+#pragma once
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;
+  std::unordered_map<std::string, std::string> custom_kms_conf;
+
+  const std::string& key_access_token() const {
+    if (refreshable_key_access_token == NULL ||
+        refreshable_key_access_token->value().empty()) {
+      throw ParquetException("key access token is not set!");
+    }
+    return refreshable_key_access_token->value();
+  }
+
+  void SetDefaultIfEmpty();
+};
+
+class PARQUET_EXPORT KmsClient {
+ public:
+  static constexpr char KMS_INSTANCE_ID_DEFAULT[] = "DEFAULT";
+  static constexpr char KMS_INSTANCE_URL_DEFAULT[] = "DEFAULT";
+  static constexpr char KEY_ACCESS_TOKEN_DEFAULT[] = "DEFAULT";
+
+  // Wraps a key - encrypts it with the master key, encodes the result
+  // and potentially adds a KMS-specific metadata.
+  virtual std::string WrapKey(const std::string& key_bytes,
+                              const std::string& master_key_identifier) = 0;

Review comment:
       I haven't reviewed the other encryption code (or the java portion of this code but I would have the same feedback on the Java API if almost all of the API was passing around strings that have different semantic meanings.  It is easy to confuse order of parameters, etc.  
   
   I've seen not great results cargo-culting between Java and C++.  Unless this is for the express purposes of JNI don't think should necessarily be a goal.
   
   in this case with only two parameters it might be OK.  But it seems like it is potentially easy to confuse.
   
   
   
   
   
   




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486420973



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,

Review comment:
       > Why isn't this a method on KmsClientFactory?
   
   @pitrou because here we get from the cache, we create a new client from factory only if the item is not in the cache (or the corresponding token expires).




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497980986



##########
File path: cpp/src/parquet/encryption/key_encryption_key.h
##########
@@ -0,0 +1,62 @@
+// 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 <cstdint>
+#include <vector>
+
+#include "arrow/util/base64.h"
+
+namespace parquet {
+namespace encryption {
+
+// In the double wrapping mode, each "data encryption key" (DEK) is encrypted with a “key
+// encryption key” (KEK), that in turn is encrypted with a "master encryption key" (MEK).
+// In a writer process, a random KEK is generated for each MEK ID, and cached in a <MEK-ID
+// : KEK> map. This allows to perform an interaction with a KMS server only once for each
+// MEK, in order to wrap its KEK. "Data encryption key" (DEK) wrapping is performed

Review comment:
       Yes.




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



[GitHub] [arrow] roee88 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
roee88 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r490958753



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,223 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be

Review comment:
       Doesn't the low level API allow encrypting only the footer? I think this is might be useful even for codebases that use PME without actually encrypting any column. For example, if the columns to encrypt are decided according to policies.
   
   If this is by design then probably a better exception should be thrown on empty parameter to column_keys (currently DCHECK 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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;

Review comment:
       >  if the cache is a member of `PropertiesDrivenCryptoFactory`, users of API must keep the life time of `PropertiesDrivenCryptoFactory` object after getting `FileEncryptionProperties`/`FileDecryptionProperties`. If they forget that, it will be easy to get a crash in their programs.
   
   Nope, the lifetime is for an internal use; the entries (cached by the token) are removed after that time, and automatically re-created upon the next call (the token is checked again via the KMS interaction). Users don't need to handle this.
   
   
   > let users manage the life time of the cache themselves and pass it into PropertiesDrivenCryptoFactory.
   > user have to keep an object alive along the time of their program
   
   I'm afraid this virtually guarantees most of the users will run with a default behavior - without caches, and will suffer from performance hits. On the other hand, if the caches are automatically created in the PropertiesDrivenCryptoFactory, users will get performance optimization out of box. Also, this doesn't involve singletons.
   
   > needs to be well documented
   
   Certainly




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498147171



##########
File path: cpp/src/parquet/encryption/kms_client.h
##########
@@ -0,0 +1,84 @@
+// 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 <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class wraps the key access token of a KMS server. If your token changes over time,
+// you should keep the reference to the KeyAccessToken object and call Refresh() method
+// every time you have a new token.
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();

Review comment:
       After checking again, in both case of wrapping and unwrapping, key access token is always set to "DEFAULT" if empty, so I will remove this method and set it from `KmsConnectionConfig` constructor. Thanks!




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r479624632



##########
File path: cpp/src/parquet/file_key_unwrapper.h
##########
@@ -0,0 +1,65 @@
+// 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 "parquet/encryption.h"
+#include "parquet/key_material.h"
+#include "parquet/key_toolkit.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  FileKeyUnwrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                   const KmsConnectionConfig& kms_connection_config,
+                   uint64_t cache_lifetime, bool is_wrap_locally);
+
+  std::string GetKey(const std::string& key_metadata) override;
+
+ private:
+  class KeyWithMasterID {
+   public:
+    KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+        : key_bytes_(key_bytes), master_id_(master_id) {}
+
+    const std::string& data_key() const { return key_bytes_; }

Review comment:
       Removed it and use `KeyWithMasterId` defined in `key_toolkit.h`




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



[GitHub] [arrow] emkornfield commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r485051463



##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       "current version"  is ambiguous and will go stale.  Please say something like in version 1.0 of the encryption specification ...




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



[GitHub] [arrow] ggershinsky commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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


   Thanks. To clarify - the security concerns I've mentioned above, relate to low level encryption, not to this pull request.


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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,106 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KMS systems wrap keys by encrypting them by master keys, and attaching additional
+// information (such as the version number of the masker key) to the result of encryption.
+// The master key version is required in  key rotation. Currently, the local wrapping mode
+// does not support key rotation (because not all KMS systems allow to fetch a master key
+// by its ID and version number). Still, the local wrapping mode adds a placeholder for
+// the master key version, that will enable support for key rotation in this mode in the
+// future, with appropriate KMS systems. This will also enable backward compatibility,
+// where future readers will be able to extract master key version in the files written by
+// the current code.
+//
+// LocalKeyWrap class writes (and reads) the "key wrap" as a flat json with the following
+// fields:
+// 1. "masterKeyVersion" - a String, with the master key version. In the current version,
+// only one value is allowed - "NO_VERSION".
+// 2. "encryptedKey" - a String, with the key encrypted by the master key
+// (base64-encoded).
+class PARQUET_EXPORT RemoteKmsClient : public KmsClient {
+ public:
+  static constexpr const char kLocalWrapNoKeyVersion[] = "NO_VERSION";
+
+  void Initialize(const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally);
+
+  std::string WrapKey(const std::string& key_bytes,
+                      const std::string& master_key_identifier) override;
+
+  std::string UnwrapKey(const std::string& wrapped_key,
+                        const std::string& master_key_identifier) override;
+
+ protected:
+  // Wrap a key with the master key in the remote KMS server.
+  virtual std::string WrapKeyInServer(const std::string& key_bytes,
+                                      const std::string& master_key_identifier) = 0;
+
+  // Unwrap a key with the master key in the remote KMS server.
+  virtual std::string UnwrapKeyInServer(const std::string& wrapped_key,
+                                        const std::string& master_key_identifier) = 0;
+
+  // Get master key from the remote KMS server.
+  // Required only for local wrapping. No need to implement if KMS supports in-server
+  // wrapping/unwrapping.

Review comment:
       "Note: this function might be called by multiple threads"

##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,106 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KMS systems wrap keys by encrypting them by master keys, and attaching additional
+// information (such as the version number of the masker key) to the result of encryption.
+// The master key version is required in  key rotation. Currently, the local wrapping mode
+// does not support key rotation (because not all KMS systems allow to fetch a master key
+// by its ID and version number). Still, the local wrapping mode adds a placeholder for
+// the master key version, that will enable support for key rotation in this mode in the
+// future, with appropriate KMS systems. This will also enable backward compatibility,
+// where future readers will be able to extract master key version in the files written by
+// the current code.
+//
+// LocalKeyWrap class writes (and reads) the "key wrap" as a flat json with the following
+// fields:
+// 1. "masterKeyVersion" - a String, with the master key version. In the current version,
+// only one value is allowed - "NO_VERSION".
+// 2. "encryptedKey" - a String, with the key encrypted by the master key
+// (base64-encoded).
+class PARQUET_EXPORT RemoteKmsClient : public KmsClient {
+ public:
+  static constexpr const char kLocalWrapNoKeyVersion[] = "NO_VERSION";
+
+  void Initialize(const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally);
+
+  std::string WrapKey(const std::string& key_bytes,
+                      const std::string& master_key_identifier) override;
+
+  std::string UnwrapKey(const std::string& wrapped_key,
+                        const std::string& master_key_identifier) override;
+
+ protected:
+  // Wrap a key with the master key in the remote KMS server.

Review comment:
       Suggest adding this to the comment: "Note: this function might be called by multiple threads"

##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,106 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KMS systems wrap keys by encrypting them by master keys, and attaching additional
+// information (such as the version number of the masker key) to the result of encryption.
+// The master key version is required in  key rotation. Currently, the local wrapping mode
+// does not support key rotation (because not all KMS systems allow to fetch a master key
+// by its ID and version number). Still, the local wrapping mode adds a placeholder for
+// the master key version, that will enable support for key rotation in this mode in the
+// future, with appropriate KMS systems. This will also enable backward compatibility,
+// where future readers will be able to extract master key version in the files written by
+// the current code.
+//
+// LocalKeyWrap class writes (and reads) the "key wrap" as a flat json with the following
+// fields:
+// 1. "masterKeyVersion" - a String, with the master key version. In the current version,
+// only one value is allowed - "NO_VERSION".
+// 2. "encryptedKey" - a String, with the key encrypted by the master key
+// (base64-encoded).
+class PARQUET_EXPORT RemoteKmsClient : public KmsClient {
+ public:
+  static constexpr const char kLocalWrapNoKeyVersion[] = "NO_VERSION";
+
+  void Initialize(const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally);
+
+  std::string WrapKey(const std::string& key_bytes,
+                      const std::string& master_key_identifier) override;
+
+  std::string UnwrapKey(const std::string& wrapped_key,
+                        const std::string& master_key_identifier) override;
+
+ protected:
+  // Wrap a key with the master key in the remote KMS server.
+  virtual std::string WrapKeyInServer(const std::string& key_bytes,
+                                      const std::string& master_key_identifier) = 0;
+
+  // Unwrap a key with the master key in the remote KMS server.

Review comment:
       "Note: this function might be called by multiple threads"

##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,106 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KMS systems wrap keys by encrypting them by master keys, and attaching additional
+// information (such as the version number of the masker key) to the result of encryption.
+// The master key version is required in  key rotation. Currently, the local wrapping mode
+// does not support key rotation (because not all KMS systems allow to fetch a master key
+// by its ID and version number). Still, the local wrapping mode adds a placeholder for
+// the master key version, that will enable support for key rotation in this mode in the
+// future, with appropriate KMS systems. This will also enable backward compatibility,
+// where future readers will be able to extract master key version in the files written by
+// the current code.
+//
+// LocalKeyWrap class writes (and reads) the "key wrap" as a flat json with the following
+// fields:
+// 1. "masterKeyVersion" - a String, with the master key version. In the current version,
+// only one value is allowed - "NO_VERSION".
+// 2. "encryptedKey" - a String, with the key encrypted by the master key
+// (base64-encoded).

Review comment:
       Should this comment (lines 29-44) be moved to the LocalWrapKey class at the line 75?

##########
File path: cpp/src/parquet/encryption/kms_client.h
##########
@@ -0,0 +1,84 @@
+// 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 <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class wraps the key access token of a KMS server. If your token changes over time,
+// you should keep the reference to the KeyAccessToken object and call Refresh() method
+// every time you have a new token.
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;
+  std::unordered_map<std::string, std::string> custom_kms_conf;
+
+  const std::string& key_access_token() const {
+    if (refreshable_key_access_token == NULL ||
+        refreshable_key_access_token->value().empty()) {
+      throw ParquetException("key access token is not set!");
+    }
+    return refreshable_key_access_token->value();
+  }
+
+  void SetDefaultIfEmpty();
+};
+
+class PARQUET_EXPORT KmsClient {
+ public:
+  static constexpr const char kKmsInstanceIdDefault[] = "DEFAULT";
+  static constexpr const char kKmsInstanceUrlDefault[] = "DEFAULT";
+  static constexpr const char kKeyAccessTokenDefault[] = "DEFAULT";
+
+  // Wraps a key - encrypts it with the master key, encodes the result
+  // and potentially adds a KMS-specific metadata.
+  virtual std::string WrapKey(const std::string& key_bytes,
+                              const std::string& master_key_identifier) = 0;
+
+  // Decrypts (unwraps) a key with the master key.

Review comment:
       "Note: this function might be called by multiple threads"

##########
File path: cpp/src/parquet/encryption/kms_client.h
##########
@@ -0,0 +1,84 @@
+// 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 <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class wraps the key access token of a KMS server. If your token changes over time,
+// you should keep the reference to the KeyAccessToken object and call Refresh() method
+// every time you have a new token.
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;
+  std::unordered_map<std::string, std::string> custom_kms_conf;
+
+  const std::string& key_access_token() const {
+    if (refreshable_key_access_token == NULL ||
+        refreshable_key_access_token->value().empty()) {
+      throw ParquetException("key access token is not set!");
+    }
+    return refreshable_key_access_token->value();
+  }
+
+  void SetDefaultIfEmpty();
+};
+
+class PARQUET_EXPORT KmsClient {
+ public:
+  static constexpr const char kKmsInstanceIdDefault[] = "DEFAULT";
+  static constexpr const char kKmsInstanceUrlDefault[] = "DEFAULT";
+  static constexpr const char kKeyAccessTokenDefault[] = "DEFAULT";
+
+  // Wraps a key - encrypts it with the master key, encodes the result
+  // and potentially adds a KMS-specific metadata.

Review comment:
       Suggest adding this to the comment: "Note: this function might be called by multiple threads"




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498151648



##########
File path: cpp/src/parquet/encryption/key_toolkit_internal.h
##########
@@ -0,0 +1,52 @@
+// 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 <chrono>
+#include <string>
+
+namespace parquet {
+namespace encryption {
+namespace internal {
+
+// "data encryption key" and "master key identifier" are paired together as output when
+// parsing from "key material"
+class KeyWithMasterId {
+ public:
+  KeyWithMasterId(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  const std::string key_bytes_;
+  const std::string master_id_;
+};
+
+/// Encrypts "key" with "master_key", using AES-GCM and the "aad"

Review comment:
       Sure @ggershinsky.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486434226



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {

Review comment:
       > Can you put internal-only stuff in separate header files?
   
   `KeyWithMasterId` is an internal thing, however, since it appears in public header file `FileKeyUnwrapper.h" (i.e. `KeyWithMasterId FileKeyUnwrapper::GetDataEncryptionKey(const KeyMaterial& key_material) const` - which will be used in KeyToolkit key rotation later). So it must be in a public `key_toolkit.h`.




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



[GitHub] [arrow] GPSnoopy commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
GPSnoopy commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r485097443



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;

Review comment:
       Yes, I wholly agree with @emkornfield. Don't force a singleton upon your users. Big no no. It doesn't play well with unit tests, multi-threading, and other cases where you may wish to have more than one singleton. By asking the user to give you the reference, you leave full control of the design of the user application to the user.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r482993016



##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key, std::string key_id_in_file);
+
+ private:
+  KeyEncryptionKey CreateKeyEncryptionKey(const std::string& master_key_id);
+
+  // A map of MEK_ID -> KeyEncryptionKey, for the current token
+  std::map<std::string, KeyEncryptionKey> kek_per_master_key_id_;
+
+  std::shared_ptr<KmsClient> kms_client_;
+  KmsConnectionConfig kms_connection_config_;
+  std::shared_ptr<FileKeyMaterialStore> key_material_store_;
+  uint64_t cache_entry_lifetime_;
+  bool double_wrapping_;
+  uint16_t key_counter_;

Review comment:
       Can you help with this question @ggershinsky ?




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



[GitHub] [arrow] bkietz commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497661719



##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {

Review comment:
       In all, I think this doesn't add sufficient value over an inlined `container, mutex` pair. I think this class should be removed, maybe extracting a helper for single lookup insertion:
   ```c++
   template <typename K, typename V, typename Hash, typename Eq, typename Gen>
   auto GetOrInsert(std::unordered_map<K, V, Hash, Eq>* map, const K& key, Gen&& gen, V placeholder = V{})
     -> decltype(map->begin()) {
     auto it_success = map->emplace(key, placeholder);
     if (!it_success.second) {
       // insertion of placeholder was blocked by an existing entry, return that
       return it_success.first;
     }
     // overwrite placeholder with computed value
     it_success.first->second = gen();
     return it_success.first;
   }
   ```




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498074809



##########
File path: cpp/src/parquet/encryption/remote_kms_client.cc
##########
@@ -0,0 +1,127 @@
+// 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.
+
+#include "arrow/json/object_parser.h"
+#include "arrow/json/object_writer.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/remote_kms_client.h"
+#include "parquet/exception.h"
+
+using arrow::json::ObjectParser;
+using arrow::json::ObjectWriter;
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char RemoteKmsClient::kLocalWrapNoKeyVersion[];
+
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapKeyVersionField[];
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapEncryptedKeyField[];
+
+RemoteKmsClient::LocalKeyWrap::LocalKeyWrap(const std::string& master_key_version,
+                                            const std::string& encrypted_encoded_key)
+    : encrypted_encoded_key_(encrypted_encoded_key),
+      master_key_version_(master_key_version) {}
+
+std::string RemoteKmsClient::LocalKeyWrap::CreateSerialized(
+    const std::string& encrypted_encoded_key) {
+  ObjectWriter json_writer;
+
+  json_writer.SetString(kLocalWrapKeyVersionField, kLocalWrapNoKeyVersion);
+  json_writer.SetString(kLocalWrapEncryptedKeyField, encrypted_encoded_key);
+
+  return json_writer.Serialize();
+}
+
+RemoteKmsClient::LocalKeyWrap RemoteKmsClient::LocalKeyWrap::Parse(
+    const std::string& wrapped_key) {
+  ObjectParser json_parser;
+  if (!json_parser.Parse(wrapped_key)) {
+    throw ParquetException("Failed to parse local key wrap json " + wrapped_key);
+  }
+  std::string master_key_version;
+  PARQUET_ASSIGN_OR_THROW(master_key_version,
+                          json_parser.GetString(kLocalWrapKeyVersionField));
+
+  std::string encrypted_encoded_key;
+  PARQUET_ASSIGN_OR_THROW(encrypted_encoded_key,
+                          json_parser.GetString(kLocalWrapEncryptedKeyField));
+
+  return RemoteKmsClient::LocalKeyWrap(master_key_version, encrypted_encoded_key);
+}
+
+void RemoteKmsClient::Initialize(const KmsConnectionConfig& kms_connection_config,
+                                 bool is_wrap_locally) {
+  kms_connection_config_ = kms_connection_config;
+  is_wrap_locally_ = is_wrap_locally;
+  if (is_wrap_locally_) {
+    master_key_cache_.Clear();
+  }
+
+  is_default_token_ =
+      kms_connection_config_.key_access_token() == KmsClient::kKeyAccessTokenDefault;
+
+  InitializeInternal();
+}
+
+std::string RemoteKmsClient::WrapKey(const std::string& key_bytes,
+                                     const std::string& master_key_identifier) {
+  if (is_wrap_locally_) {

Review comment:
       As I said above, I'm skeptical following the Java API is a good idea. You seem to assume that people using the C++ library also use the Java library, which is quite arbitrary (for example, I know absolutely nothing about `parquet-mr`).




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497978500



##########
File path: cpp/src/parquet/encryption/kms_client.h
##########
@@ -0,0 +1,84 @@
+// 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 <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class wraps the key access token of a KMS server. If your token changes over time,
+// you should keep the reference to the KeyAccessToken object and call Refresh() method
+// every time you have a new token.
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();

Review comment:
       This function will set `value_` to `"DEFAULT"` if `value_` is 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



[GitHub] [arrow] emkornfield commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r485038390



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;

Review comment:
       I'm generally skeptical for forcing singletons into the API.  For instance in Java, I would assume that applications would maintain the an instance of KeyToolkit as singleton (not enforced at the language level).  
   
   Its not clear if this would be an issue, but if this would the use of singleton here work well if the class was being used in a multi-tenant system?




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497982290



##########
File path: cpp/src/parquet/encryption/file_key_unwrapper.h
##########
@@ -0,0 +1,66 @@
+// 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 "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/key_material.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class will retrieve the key from "key metadata", following these steps:
+// 1. Parse "key metadata" (see structure in KeyMetadata class).
+// 2. Retrieve "key material" which can be stored inside or outside "key metadata"
+//    Currently we don't support the case "key material" stores outside "key metadata"
+//    yet.
+// 3. Unwrap the "data encryption key" from "key material". There are 2 modes:
+// 3.1. single wrapping: decrypt the wrapped "data encryption key" directly with "master
+// encryption key" 3.2. double wrapping: 2 steps: 3.2.1. "key encryption key" is decrypted
+// with "master encryption key" 3.2.2. "data encryption key" is decrypted with the above
+// "key encryption key"
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  /// 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.
+  FileKeyUnwrapper(KeyToolkit* key_toolkit,
+                   const KmsConnectionConfig& kms_connection_config,
+                   uint64_t cache_lifetime_seconds);
+
+  std::string GetKey(const std::string& key_metadata) const override;
+
+ private:
+  internal::KeyWithMasterId GetDataEncryptionKey(const KeyMaterial& key_material) const;
+  std::shared_ptr<KmsClient> GetKmsClientFromConfigOrKeyMaterial(
+      const KeyMaterial& key_material) const;
+
+  /// A map of Key Encryption Key (KEK) ID -> KEK bytes, for the current token
+  mutable std::shared_ptr<arrow::util::ConcurrentMap<std::string>> kek_per_kek_id_;
+  KeyToolkit* key_toolkit_;
+  mutable KmsConnectionConfig kms_connection_config_;

Review comment:
       Because it is modified in `std::string GetKey(const std::string& key_metadata) const override;`




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498005234



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {
+ public:
+  /// KmsClientFactory must be registered before calling

Review comment:
       "registered" means "set" (i.e. RegisterKmsClientFactory() is called)




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498076584



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {
+ public:
+  /// KmsClientFactory must be registered before calling
+  /// GetFileEncryptionProperties()/GetFileDecryptionProperties() methods.
+  void RegisterKmsClientFactory(std::shared_ptr<KmsClientFactory> kms_client_factory);
+
+  std::shared_ptr<FileEncryptionProperties> GetFileEncryptionProperties(
+      const KmsConnectionConfig& kms_connection_config,
+      std::shared_ptr<EncryptionConfiguration> encryption_config);
+
+  /// The returned FileDecryptionProperties object will use the cache inside this
+  /// PropertiesDrivenCryptoFactory object, so please keep this
+  /// PropertiesDrivenCryptoFactory object alive along with the returned
+  /// FileDecryptionProperties object.
+  std::shared_ptr<FileDecryptionProperties> GetFileDecryptionProperties(
+      const KmsConnectionConfig& kms_connection_config,
+      std::shared_ptr<DecryptionConfiguration> decryption_config);
+
+  void RemoveCacheEntriesForToken(const std::string& access_token) {
+    key_toolkit_.RemoveCacheEntriesForToken(access_token);
+  }
+
+  void RemoveCacheEntriesForAllTokens() { key_toolkit_.RemoveCacheEntriesForAllTokens(); }
+
+ private:
+  /// Acceptable data key lengths in number of bits
+  static constexpr const int32_t kAcceptableDataKeyLengths[] = {128, 192, 256};

Review comment:
       I mean: is the order significant? Is it just a set of constants that you lookup to validate an input?




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



[GitHub] [arrow] roee88 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
roee88 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r490958753



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,223 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          wrap_locally_(kDefaultWrapLocally),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be

Review comment:
       Doesn't the low level API allow encrypting only the footer? I think this is might be useful even for codebases that use PME without actually encrypting any column. For example, if the columns to encrypt are decided according to policies.
   
   If this is by design then probably a better exception should be thrown on empty parameter to column_keys (currently DCHECK 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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498075668



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {

Review comment:
       As said above, I don't think keeping the API the same has much value. Different languages have different favored conventions. There is much to be said about Java's love of long names...




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497973376



##########
File path: cpp/src/parquet/encryption/remote_kms_client.cc
##########
@@ -0,0 +1,127 @@
+// 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.
+
+#include "arrow/json/object_parser.h"
+#include "arrow/json/object_writer.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/remote_kms_client.h"
+#include "parquet/exception.h"
+
+using arrow::json::ObjectParser;
+using arrow::json::ObjectWriter;
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char RemoteKmsClient::kLocalWrapNoKeyVersion[];
+
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapKeyVersionField[];
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapEncryptedKeyField[];
+
+RemoteKmsClient::LocalKeyWrap::LocalKeyWrap(const std::string& master_key_version,
+                                            const std::string& encrypted_encoded_key)
+    : encrypted_encoded_key_(encrypted_encoded_key),
+      master_key_version_(master_key_version) {}
+
+std::string RemoteKmsClient::LocalKeyWrap::CreateSerialized(
+    const std::string& encrypted_encoded_key) {
+  ObjectWriter json_writer;
+
+  json_writer.SetString(kLocalWrapKeyVersionField, kLocalWrapNoKeyVersion);
+  json_writer.SetString(kLocalWrapEncryptedKeyField, encrypted_encoded_key);
+
+  return json_writer.Serialize();
+}
+
+RemoteKmsClient::LocalKeyWrap RemoteKmsClient::LocalKeyWrap::Parse(
+    const std::string& wrapped_key) {
+  ObjectParser json_parser;
+  if (!json_parser.Parse(wrapped_key)) {
+    throw ParquetException("Failed to parse local key wrap json " + wrapped_key);
+  }
+  std::string master_key_version;
+  PARQUET_ASSIGN_OR_THROW(master_key_version,
+                          json_parser.GetString(kLocalWrapKeyVersionField));
+
+  std::string encrypted_encoded_key;
+  PARQUET_ASSIGN_OR_THROW(encrypted_encoded_key,
+                          json_parser.GetString(kLocalWrapEncryptedKeyField));
+
+  return RemoteKmsClient::LocalKeyWrap(master_key_version, encrypted_encoded_key);
+}
+
+void RemoteKmsClient::Initialize(const KmsConnectionConfig& kms_connection_config,
+                                 bool is_wrap_locally) {
+  kms_connection_config_ = kms_connection_config;
+  is_wrap_locally_ = is_wrap_locally;
+  if (is_wrap_locally_) {
+    master_key_cache_.Clear();
+  }
+
+  is_default_token_ =
+      kms_connection_config_.key_access_token() == KmsClient::kKeyAccessTokenDefault;
+
+  InitializeInternal();
+}
+
+std::string RemoteKmsClient::WrapKey(const std::string& key_bytes,
+                                     const std::string& master_key_identifier) {
+  if (is_wrap_locally_) {

Review comment:
       I think it's a good idea. However, needs @ggershinsky to confirm if I can create 2 separated classes, for example: `LocalWrappingRemoteKmsClient` and `InServerWrappingRemoteKmsClient`?




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r485557493



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {

Review comment:
       A general style concern: we tend to avoid nested classes (they aren't super popular in C++ AFAIK), and the Arrow codebase has almost none of them). They tend to make things a bit more akward and don't bring anything in terms of abstraction.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497971996



##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"

Review comment:
       It should be removed. Thanks!




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



[GitHub] [arrow] github-actions[bot] commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-678611290


   https://issues.apache.org/jira/browse/ARROW-9318


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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r496755148



##########
File path: cpp/src/arrow/json/object_parser.h
##########
@@ -0,0 +1,45 @@
+// 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 "arrow/json/rapidjson_defs.h"  // IWYU pragma: keep
+
+#include <rapidjson/document.h>
+
+#include "arrow/result.h"
+#include "arrow/util/string_view.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace json {
+
+namespace rj = arrow::rapidjson;
+
+class ARROW_EXPORT ObjectParser {

Review comment:
       Can you add a docstring?

##########
File path: cpp/src/arrow/json/object_writer.h
##########
@@ -0,0 +1,47 @@
+// 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 "arrow/json/rapidjson_defs.h"  // IWYU pragma: keep
+
+#include <rapidjson/document.h>
+
+#include "arrow/util/string_view.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace json {
+
+namespace rj = arrow::rapidjson;
+
+class ARROW_EXPORT ObjectWriter {
+ public:
+  ObjectWriter();
+
+  void SetString(arrow::util::string_view key, arrow::util::string_view value);
+  void SetBool(arrow::util::string_view key, bool value);
+
+  std::string Serialize();
+
+ private:
+  rj::Document _document;

Review comment:
       Same comment as for `ObjectParser`.

##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>

Review comment:
       I'm surprised that this is only templated on value, btw.

##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {
+ public:
+  void Insert(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_.insert({key, value});
+  }
+
+  void Assign(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_[key] = value;
+  }
+
+  V GetOrAssignIfNotExist(const std::string& key, std::function<V()> compute_value_func) {
+    auto lock = mutex_.Lock();
+    auto it = map_.find(key);
+    if (it == map_.end()) {
+      map_.insert({key, compute_value_func()});
+    }
+    return map_.at(key);
+  }
+
+  void Erase(const std::string& key) {
+    auto lock = mutex_.Lock();
+    map_.erase(key);
+  }
+
+  void Clear() {
+    auto lock = mutex_.Lock();
+    map_.clear();
+  }
+
+  std::pair<bool, V> Find(const std::string& key) {

Review comment:
       Return `util::optional<V>` instead.

##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)
+      : cached_item_(cached_item) {
+    expiration_timestamp_ =
+        CurrentTimePoint() + std::chrono::milliseconds(expiration_interval_millis);
+  }
+
+  bool IsExpired() {
+    auto now = CurrentTimePoint();
+    return (now > expiration_timestamp_);
+  }
+
+  E cached_item() { return cached_item_; }
+
+ private:
+  TimePoint expiration_timestamp_;
+  E cached_item_;
+};
+
+// This class is to avoid the below warning when compiling KeyToolkit class with VS2015
+// warning C4503: decorated name length exceeded, name was truncated
+template <typename V>
+class ExpiringCacheMapEntry {
+ public:
+  ExpiringCacheMapEntry() = default;
+
+  explicit ExpiringCacheMapEntry(std::shared_ptr<ConcurrentMap<V>> cached_item,
+                                 uint64_t expiration_interval_millis)
+      : map_cache_(cached_item, expiration_interval_millis) {}
+
+  bool IsExpired() { return map_cache_.IsExpired(); }
+
+  std::shared_ptr<ConcurrentMap<V>> cached_item() { return map_cache_.cached_item(); }
+
+ private:
+  ExpiringCacheEntry<std::shared_ptr<ConcurrentMap<V>>> map_cache_;
+};
+
+}  // namespace internal
+
+// Two-level cache with expiration of internal caches according to token lifetime.
+// External cache is per token, internal is per string key.
+// Wrapper class around:
+//    std::unordered_map<std::string,
+//    internal::ExpiringCacheEntry<std::unordered_map<std::string, V>>>
+// This cache is safe to be shared between threads.
+template <typename V>
+class TwoLevelCacheWithExpiration {
+ public:
+  TwoLevelCacheWithExpiration() {
+    last_cache_cleanup_timestamp_ = internal::CurrentTimePoint();
+  }
+
+  std::shared_ptr<ConcurrentMap<V>> GetOrCreateInternalCache(
+      const std::string& access_token, uint64_t cache_entry_lifetime_ms) {
+    auto lock = mutex_.Lock();
+
+    auto external_cache_entry = cache_.find(access_token);
+    if (external_cache_entry == cache_.end() ||
+        external_cache_entry->second.IsExpired()) {
+      cache_.insert(
+          {access_token, internal::ExpiringCacheMapEntry<V>(
+                             std::shared_ptr<ConcurrentMap<V>>(new ConcurrentMap<V>()),
+                             cache_entry_lifetime_ms)});
+    }
+
+    return cache_[access_token].cached_item();
+  }
+
+  void RemoveCacheEntriesForToken(const std::string& access_token) {
+    auto lock = mutex_.Lock();
+    cache_.erase(access_token);
+  }
+
+  void RemoveCacheEntriesForAllTokens() {
+    auto lock = mutex_.Lock();
+    cache_.clear();
+  }
+
+  void CheckCacheForExpiredTokens(uint64_t cache_cleanup_period_ms) {
+    auto lock = mutex_.Lock();
+
+    internal::TimePoint now = internal::CurrentTimePoint();
+
+    if (now > (last_cache_cleanup_timestamp_ +
+               std::chrono::milliseconds(cache_cleanup_period_ms))) {
+      RemoveExpiredEntriesNoMutex();
+      last_cache_cleanup_timestamp_ =
+          now + std::chrono::milliseconds(cache_cleanup_period_ms);
+    }
+  }
+
+  void RemoveExpiredEntriesFromCache() {
+    auto lock = mutex_.Lock();
+
+    RemoveExpiredEntriesNoMutex();
+  }
+
+  void Remove(const std::string& access_token) {

Review comment:
       There's already `RemoveCacheEntriesForToken`. Keep one or the other.

##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration_test.cc
##########
@@ -0,0 +1,207 @@
+// 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.
+
+#include <chrono>
+#include <thread>
+
+#include <gtest/gtest.h>
+
+#include "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+class TwoLevelCacheWithExpirationTest : public ::testing::Test {
+ public:
+  void SetUp() {
+    // lifetime is 1s
+    std::shared_ptr<ConcurrentMap<int>> lifetime_1s =
+        cache_.GetOrCreateInternalCache("lifetime1s", 1000);
+    lifetime_1s->Insert("item1", 1);
+    lifetime_1s->Insert("item2", 2);
+
+    // lifetime is 3s
+    std::shared_ptr<ConcurrentMap<int>> lifetime_3s =
+        cache_.GetOrCreateInternalCache("lifetime3s", 3000);
+    lifetime_3s->Insert("item21", 21);
+    lifetime_3s->Insert("item22", 22);
+  }
+
+ protected:
+  void TaskInsert(int thread_no) {
+    for (int i = 0; i < 20; i++) {
+      std::string token = i % 2 == 0 ? "lifetime1s" : "lifetime3s";
+      uint64_t lifetime_ms = i % 2 == 0 ? 1000 : 3000;
+      std::shared_ptr<ConcurrentMap<int>> internal_cache =
+          cache_.GetOrCreateInternalCache(token, lifetime_ms);
+      std::stringstream ss;
+      ss << "item_" << thread_no << "_" << i;
+      internal_cache->Insert(ss.str(), i);
+      std::this_thread::sleep_for(std::chrono::milliseconds(10));
+    }
+  }
+
+  void TaskClean() {
+    for (int i = 0; i < 20; i++) {
+      cache_.Clear();
+      std::this_thread::sleep_for(std::chrono::milliseconds(20));
+    }
+  }
+
+  TwoLevelCacheWithExpiration<int> cache_;
+};
+
+TEST_F(TwoLevelCacheWithExpirationTest, RemoveExpiration) {
+  std::shared_ptr<ConcurrentMap<int>> lifetime_1s_before_expiration =
+      cache_.GetOrCreateInternalCache("lifetime1s", 1000);
+  ASSERT_EQ(lifetime_1s_before_expiration->size(), 2);
+
+  // wait for 2s, we expect:
+  // lifetime_1s will be expired
+  // lifetime_3s will not be expired
+  std::this_thread::sleep_for(std::chrono::milliseconds(2000));
+  // now clear expired items from the cache
+  cache_.RemoveExpiredEntriesFromCache();
+
+  // lifetime_1s (with 2 items) is expired and has been removed from the cache.
+  // Now the cache create a new object which has no item.
+  std::shared_ptr<ConcurrentMap<int>> lifetime_1s =
+      cache_.GetOrCreateInternalCache("lifetime1s", 1000);
+  ASSERT_EQ(lifetime_1s->size(), 0);
+
+  // However, lifetime_1s_before_expiration can still access normally and independently
+  // from the one in cache
+  lifetime_1s_before_expiration->Insert("item3", 3);
+  ASSERT_EQ(lifetime_1s_before_expiration->size(), 3);
+  ASSERT_EQ(lifetime_1s->size(), 0);
+
+  // lifetime_3s is not expired and still contains 2 items.
+  std::shared_ptr<ConcurrentMap<int>> lifetime_3s =
+      cache_.GetOrCreateInternalCache("lifetime3s", 3000);
+  ASSERT_EQ(lifetime_3s->size(), 2);
+}
+
+TEST_F(TwoLevelCacheWithExpirationTest, CleanupPeriodTooBig) {
+  // wait for 2s, now:
+  // lifetime_1s is expired
+  // lifetime_3s isn't expired
+  std::this_thread::sleep_for(std::chrono::milliseconds(2000));
+
+  // if cleanup_period is too big (10s), the expired items may not be removed from cache.
+  cache_.CheckCacheForExpiredTokens(10000);
+
+  // lifetime_1s (with 2 items) is expired but not removed from the cache, still contains

Review comment:
       Why "not removed"? I'm not sure if what's being tested is useful here.

##########
File path: cpp/src/parquet/encryption/test_encryption_util.h
##########
@@ -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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#pragma once
+
+#include <algorithm>
+#include <chrono>
+#include <limits>
+#include <memory>
+#include <random>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/status.h"
+#include "arrow/testing/util.h"
+#include "arrow/util/io_util.h"
+
+#include "parquet/column_page.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
+#include "parquet/encoding.h"
+#include "parquet/encryption/encryption.h"
+#include "parquet/platform.h"
+#include "parquet/test_util.h"
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+using arrow::internal::TemporaryDir;
+
+using parquet::ConvertedType;
+using parquet::Repetition;
+using parquet::Type;
+using schema::GroupNode;
+using schema::NodePtr;
+using schema::PrimitiveNode;
+
+constexpr int kFixedLength = 10;
+
+const char kFooterEncryptionKey[] = "0123456789012345";  // 128bit/16
+const char kColumnEncryptionKey1[] = "1234567890123450";
+const char kColumnEncryptionKey2[] = "1234567890123451";
+const char kFileName[] = "tester";
+
+inline std::string data_file(const char* file) {
+  std::string dir_string(parquet::test::get_data_dir());
+  std::stringstream ss;
+  ss << dir_string << "/" << file;
+  return ss.str();
+}
+
+// A temporary directory that contains the encrypted files generated in the tests.
+extern std::unique_ptr<TemporaryDir> temp_dir;
+
+inline arrow::Result<std::unique_ptr<TemporaryDir>> temp_data_dir() {
+  arrow::Result<std::unique_ptr<TemporaryDir>> dir;
+  ARROW_ASSIGN_OR_RAISE(dir, TemporaryDir::Make("parquet-encryption-test-"));
+  return dir;
+}
+
+static constexpr const char kDoubleFieldName[] = "double_field";
+static constexpr const char kFloatFieldName[] = "float_field";
+static constexpr const char kBooleanFieldName[] = "boolean_field";
+static constexpr const char kInt32FieldName[] = "int32_field";
+static constexpr const char kInt64FieldName[] = "int64_field";
+static constexpr const char kInt96FieldName[] = "int96_field";
+static constexpr const char kByteArrayFieldName[] = "ba_field";
+static constexpr const char kFixedLenByteArrayFieldName[] = "flba_field";
+
+const char kFooterMasterKey[] = "0123456789112345";

Review comment:
       I'm curious why some strings constants are `static constexpr` and some not?

##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED
+  fields.push_back(PrimitiveNode::Make(kBooleanFieldName, Repetition::REQUIRED,
+                                       Type::BOOLEAN, ConvertedType::NONE));
+
+  // Create a primitive node named 'int32_field' with type:INT32, repetition:REQUIRED,
+  // logical type:TIME_MILLIS
+  fields.push_back(PrimitiveNode::Make(kInt32FieldName, Repetition::REQUIRED, Type::INT32,
+                                       ConvertedType::TIME_MILLIS));
+
+  // Create a primitive node named 'int64_field' with type:INT64, repetition:REPEATED
+  fields.push_back(PrimitiveNode::Make(kInt64FieldName, Repetition::REPEATED, Type::INT64,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kInt96FieldName, Repetition::REQUIRED, Type::INT96,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kFloatFieldName, Repetition::REQUIRED, Type::FLOAT,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kDoubleFieldName, Repetition::REQUIRED,
+                                       Type::DOUBLE, ConvertedType::NONE));
+
+  // Create a primitive node named 'ba_field' with type:BYTE_ARRAY, repetition:OPTIONAL
+  fields.push_back(PrimitiveNode::Make(kByteArrayFieldName, Repetition::OPTIONAL,
+                                       Type::BYTE_ARRAY, ConvertedType::NONE));
+
+  // Create a primitive node named 'flba_field' with type:FIXED_LEN_BYTE_ARRAY,
+  // repetition:REQUIRED, field_length = kFixedLength
+  fields.push_back(PrimitiveNode::Make(kFixedLenByteArrayFieldName, Repetition::REQUIRED,
+                                       Type::FIXED_LEN_BYTE_ARRAY, ConvertedType::NONE,
+                                       kFixedLength));
+
+  // Create a GroupNode named 'schema' using the primitive nodes defined above
+  // This GroupNode is the root node of the schema tree
+  return std::static_pointer_cast<GroupNode>(
+      GroupNode::Make("schema", Repetition::REQUIRED, fields));
+}
+
+void FileEncryptor::EncryptFile(
+    std::string file,
+    std::shared_ptr<parquet::FileEncryptionProperties> encryption_configurations) {
+  WriterProperties::Builder prop_builder;
+  prop_builder.compression(parquet::Compression::SNAPPY);
+  prop_builder.encryption(encryption_configurations);
+  std::shared_ptr<WriterProperties> writer_properties = prop_builder.build();
+
+  PARQUET_ASSIGN_OR_THROW(auto out_file, FileClass::Open(file));
+  // Create a ParquetFileWriter instance
+  std::shared_ptr<parquet::ParquetFileWriter> file_writer =
+      parquet::ParquetFileWriter::Open(out_file, schema_, writer_properties);
+
+  for (int r = 0; r < num_rgs; r++) {
+    bool buffered_mode = r % 2 == 0;
+    auto row_group_writer = buffered_mode ? file_writer->AppendBufferedRowGroup()
+                                          : file_writer->AppendRowGroup();
+
+    int column_index = 0;
+    // Captures i by reference; increments it by one
+    auto get_next_column = [&]() {
+      return buffered_mode ? row_group_writer->column(column_index++)
+                           : row_group_writer->NextColumn();
+    };
+
+    // Write the Bool column
+    parquet::BoolWriter* bool_writer =
+        static_cast<parquet::BoolWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      bool value = ((i % 2) == 0) ? true : false;
+      bool_writer->WriteBatch(1, nullptr, nullptr, &value);

Review comment:
       It would be much better to write several values at a time...

##########
File path: cpp/src/parquet/encryption/key_toolkit.cc
##########
@@ -0,0 +1,53 @@
+// 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.
+
+#include "parquet/encryption/key_toolkit.h"
+
+namespace parquet {
+namespace encryption {
+
+std::shared_ptr<KmsClient> KeyToolkit::GetKmsClient(
+    const KmsConnectionConfig& kms_connection_config, uint64_t cache_entry_lifetime_ms) {
+  if (kms_client_factory_ == NULL) {
+    throw ParquetException("No KmsClientFactory is registered.");
+  }
+  std::shared_ptr<ConcurrentMap<std::shared_ptr<parquet::encryption::KmsClient>>>
+      kms_client_per_kms_instance_cache =

Review comment:
       I'm saying this once, but instead of explicitly declaring variables like this, you can (and should, when the type name is long like this) just use `auto`:
   ```c++
   auto kms_client_per_kms_instance_cache = ... 
   ```

##########
File path: cpp/src/arrow/json/object_parser.h
##########
@@ -0,0 +1,45 @@
+// 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 "arrow/json/rapidjson_defs.h"  // IWYU pragma: keep
+
+#include <rapidjson/document.h>
+
+#include "arrow/result.h"
+#include "arrow/util/string_view.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace json {
+
+namespace rj = arrow::rapidjson;
+
+class ARROW_EXPORT ObjectParser {
+ public:
+  bool Parse(arrow::util::string_view json);

Review comment:
       It would be better to return a `Status`.

##########
File path: cpp/src/arrow/json/object_writer.h
##########
@@ -0,0 +1,47 @@
+// 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 "arrow/json/rapidjson_defs.h"  // IWYU pragma: keep
+
+#include <rapidjson/document.h>
+
+#include "arrow/util/string_view.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace json {
+
+namespace rj = arrow::rapidjson;
+
+class ARROW_EXPORT ObjectWriter {

Review comment:
       Docstring?

##########
File path: cpp/src/arrow/json/object_parser.h
##########
@@ -0,0 +1,45 @@
+// 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 "arrow/json/rapidjson_defs.h"  // IWYU pragma: keep
+
+#include <rapidjson/document.h>
+
+#include "arrow/result.h"
+#include "arrow/util/string_view.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace json {
+
+namespace rj = arrow::rapidjson;
+
+class ARROW_EXPORT ObjectParser {
+ public:
+  bool Parse(arrow::util::string_view json);
+
+  Result<std::string> GetString(const char* key) const;
+  Result<bool> GetBool(const char* key) const;
+
+ private:
+  rj::Document _document;

Review comment:
       We'd rather avoid exposing implementation details, especially when it includes heavy headers such as RapidJSON. Instead we would rather use the [pimpl idiom](https://bitboom.github.io/pimpl-idiom), e.g.:
   ```c++
    private:
     class Impl;
     std::unique_ptr<Impl> impl_;
   ```

##########
File path: cpp/src/parquet/encryption/file_key_unwrapper.h
##########
@@ -0,0 +1,66 @@
+// 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 "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/key_material.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class will retrieve the key from "key metadata", following these steps:
+// 1. Parse "key metadata" (see structure in KeyMetadata class).
+// 2. Retrieve "key material" which can be stored inside or outside "key metadata"
+//    Currently we don't support the case "key material" stores outside "key metadata"
+//    yet.
+// 3. Unwrap the "data encryption key" from "key material". There are 2 modes:
+// 3.1. single wrapping: decrypt the wrapped "data encryption key" directly with "master
+// encryption key" 3.2. double wrapping: 2 steps: 3.2.1. "key encryption key" is decrypted
+// with "master encryption key" 3.2.2. "data encryption key" is decrypted with the above
+// "key encryption key"
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  /// 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.
+  FileKeyUnwrapper(KeyToolkit* key_toolkit,
+                   const KmsConnectionConfig& kms_connection_config,
+                   uint64_t cache_lifetime_seconds);
+
+  std::string GetKey(const std::string& key_metadata) const override;
+
+ private:
+  internal::KeyWithMasterId GetDataEncryptionKey(const KeyMaterial& key_material) const;
+  std::shared_ptr<KmsClient> GetKmsClientFromConfigOrKeyMaterial(
+      const KeyMaterial& key_material) const;
+
+  /// A map of Key Encryption Key (KEK) ID -> KEK bytes, for the current token
+  mutable std::shared_ptr<arrow::util::ConcurrentMap<std::string>> kek_per_kek_id_;

Review comment:
       Why `shared_ptr`?

##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {

Review comment:
       I'm not sure about exposing such a primitive. @emkornfield @bkietz What do you think?

##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {
+ public:
+  void Insert(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_.insert({key, value});
+  }
+
+  void Assign(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_[key] = value;
+  }
+
+  V GetOrAssignIfNotExist(const std::string& key, std::function<V()> compute_value_func) {
+    auto lock = mutex_.Lock();
+    auto it = map_.find(key);
+    if (it == map_.end()) {
+      map_.insert({key, compute_value_func()});
+    }
+    return map_.at(key);
+  }
+
+  void Erase(const std::string& key) {
+    auto lock = mutex_.Lock();
+    map_.erase(key);
+  }
+
+  void Clear() {
+    auto lock = mutex_.Lock();
+    map_.clear();
+  }
+
+  std::pair<bool, V> Find(const std::string& key) {

Review comment:
       Also, can mark this `const`.

##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {
+ public:
+  void Insert(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_.insert({key, value});
+  }
+
+  void Assign(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_[key] = value;
+  }
+
+  V GetOrAssignIfNotExist(const std::string& key, std::function<V()> compute_value_func) {
+    auto lock = mutex_.Lock();
+    auto it = map_.find(key);
+    if (it == map_.end()) {
+      map_.insert({key, compute_value_func()});
+    }
+    return map_.at(key);
+  }
+
+  void Erase(const std::string& key) {
+    auto lock = mutex_.Lock();
+    map_.erase(key);
+  }
+
+  void Clear() {
+    auto lock = mutex_.Lock();
+    map_.clear();
+  }
+
+  std::pair<bool, V> Find(const std::string& key) {
+    auto lock = mutex_.Lock();
+    auto it = map_.find(key);
+    if (it != map_.end()) {
+      return std::make_pair(true, it->second);
+    }
+    return std::make_pair(false, V());
+  }
+
+  size_t size() {
+    auto lock = mutex_.Lock();
+    return map_.size();
+  }
+
+ private:
+  std::unordered_map<std::string, V> map_;
+  arrow::util::Mutex mutex_;

Review comment:
       You may want to mark this `mutable`.

##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {
+ public:
+  void Insert(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_.insert({key, value});
+  }
+
+  void Assign(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_[key] = value;
+  }
+
+  V GetOrAssignIfNotExist(const std::string& key, std::function<V()> compute_value_func) {

Review comment:
       `GetOrInsert` is enough.
   
   Also, how about passing a template callable instead?
   ```
   template <typename ValueFunc>
   V GetOrAssignIfNotExist(const std::string& key, ValueFunc&& compute_value_func) {
   ```

##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {
+ public:
+  void Insert(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_.insert({key, value});
+  }
+
+  void Assign(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_[key] = value;
+  }
+
+  V GetOrAssignIfNotExist(const std::string& key, std::function<V()> compute_value_func) {
+    auto lock = mutex_.Lock();
+    auto it = map_.find(key);
+    if (it == map_.end()) {
+      map_.insert({key, compute_value_func()});
+    }
+    return map_.at(key);

Review comment:
       This is doing three hash lookups in a row. You should be able to get by with less. Example (untested):
   ```c++
   auto pair = map_.find(key);
   if (it == map_.end()) {
     it = map_.emplace(key, compute_value_func());
   }
   return it->second;
   ```

##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {
+ public:
+  void Insert(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_.insert({key, value});
+  }
+
+  void Assign(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_[key] = value;
+  }
+
+  V GetOrAssignIfNotExist(const std::string& key, std::function<V()> compute_value_func) {
+    auto lock = mutex_.Lock();
+    auto it = map_.find(key);
+    if (it == map_.end()) {
+      map_.insert({key, compute_value_func()});
+    }
+    return map_.at(key);
+  }
+
+  void Erase(const std::string& key) {
+    auto lock = mutex_.Lock();
+    map_.erase(key);
+  }
+
+  void Clear() {
+    auto lock = mutex_.Lock();
+    map_.clear();
+  }
+
+  std::pair<bool, V> Find(const std::string& key) {
+    auto lock = mutex_.Lock();
+    auto it = map_.find(key);
+    if (it != map_.end()) {
+      return std::make_pair(true, it->second);
+    }
+    return std::make_pair(false, V());
+  }
+
+  size_t size() {

Review comment:
       Can mark this `const`.

##########
File path: cpp/src/parquet/encryption/test_encryption_util.h
##########
@@ -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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#pragma once
+
+#include <algorithm>
+#include <chrono>
+#include <limits>
+#include <memory>
+#include <random>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/status.h"
+#include "arrow/testing/util.h"
+#include "arrow/util/io_util.h"
+
+#include "parquet/column_page.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
+#include "parquet/encoding.h"
+#include "parquet/encryption/encryption.h"
+#include "parquet/platform.h"
+#include "parquet/test_util.h"
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+using arrow::internal::TemporaryDir;
+
+using parquet::ConvertedType;
+using parquet::Repetition;
+using parquet::Type;
+using schema::GroupNode;
+using schema::NodePtr;
+using schema::PrimitiveNode;
+
+constexpr int kFixedLength = 10;
+
+const char kFooterEncryptionKey[] = "0123456789012345";  // 128bit/16
+const char kColumnEncryptionKey1[] = "1234567890123450";
+const char kColumnEncryptionKey2[] = "1234567890123451";
+const char kFileName[] = "tester";
+
+inline std::string data_file(const char* file) {
+  std::string dir_string(parquet::test::get_data_dir());
+  std::stringstream ss;
+  ss << dir_string << "/" << file;
+  return ss.str();
+}
+
+// A temporary directory that contains the encrypted files generated in the tests.
+extern std::unique_ptr<TemporaryDir> temp_dir;
+
+inline arrow::Result<std::unique_ptr<TemporaryDir>> temp_data_dir() {
+  arrow::Result<std::unique_ptr<TemporaryDir>> dir;
+  ARROW_ASSIGN_OR_RAISE(dir, TemporaryDir::Make("parquet-encryption-test-"));
+  return dir;
+}
+
+static constexpr const char kDoubleFieldName[] = "double_field";
+static constexpr const char kFloatFieldName[] = "float_field";
+static constexpr const char kBooleanFieldName[] = "boolean_field";
+static constexpr const char kInt32FieldName[] = "int32_field";
+static constexpr const char kInt64FieldName[] = "int64_field";
+static constexpr const char kInt96FieldName[] = "int96_field";
+static constexpr const char kByteArrayFieldName[] = "ba_field";
+static constexpr const char kFixedLenByteArrayFieldName[] = "flba_field";
+
+const char kFooterMasterKey[] = "0123456789112345";
+const char kFooterMasterKeyId[] = "kf";
+const char* const kColumnMasterKeys[] = {"1234567890123450", "1234567890123451",
+                                         "1234567890123452", "1234567890123453",
+                                         "1234567890123454", "1234567890123455"};
+const char* const kColumnMasterKeyIds[] = {"kc1", "kc2", "kc3", "kc4", "kc5", "kc6"};
+
+inline std::unordered_map<std::string, std::string> BuildKeyMap(
+    const char* const* column_ids, const char* const* column_keys, const char* footer_id,
+    const char* footer_key) {
+  std::unordered_map<std::string, std::string> key_map;
+  // add column keys
+  for (int i = 0; i < 6; i++) {
+    key_map.insert({column_ids[i], column_keys[i]});
+  }
+  // add footer key
+  key_map.insert({footer_id, footer_key});
+
+  return key_map;
+}
+
+inline std::string BuildColumnKeyMapping() {
+  std::ostringstream stream;
+  stream << kColumnMasterKeyIds[0] << ":" << kDoubleFieldName << ";"
+         << kColumnMasterKeyIds[1] << ":" << kFloatFieldName << ";"
+         << kColumnMasterKeyIds[2] << ":" << kBooleanFieldName << ";"
+         << kColumnMasterKeyIds[3] << ":" << kInt32FieldName << ";"
+         << kColumnMasterKeyIds[4] << ":" << kByteArrayFieldName << ";"
+         << kColumnMasterKeyIds[5] << ":" << kFixedLenByteArrayFieldName << ";";
+  return stream.str();
+}
+
+// FileEncryptor and FileDecryptor are helper classes to write/read an encrypted parquet
+// file corresponding to each pair of FileEncryptionProperties/FileDecryptionProperties.
+// FileEncryptor writes the file with fixed data values and FileDecryptor reads the file
+// and verify the correctness of data values.
+class FileEncryptor {
+ public:
+  FileEncryptor();
+
+  void EncryptFile(
+      std::string file,
+      std::shared_ptr<parquet::FileEncryptionProperties> encryption_configurations);
+
+ private:
+  std::shared_ptr<GroupNode> SetupEncryptionSchema();
+
+  int num_rgs = 5;

Review comment:
       "num_rowgroups_" perhaps?

##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)
+      : cached_item_(cached_item) {
+    expiration_timestamp_ =
+        CurrentTimePoint() + std::chrono::milliseconds(expiration_interval_millis);
+  }
+
+  bool IsExpired() {
+    auto now = CurrentTimePoint();
+    return (now > expiration_timestamp_);
+  }
+
+  E cached_item() { return cached_item_; }
+
+ private:
+  TimePoint expiration_timestamp_;
+  E cached_item_;
+};
+
+// This class is to avoid the below warning when compiling KeyToolkit class with VS2015
+// warning C4503: decorated name length exceeded, name was truncated
+template <typename V>
+class ExpiringCacheMapEntry {
+ public:
+  ExpiringCacheMapEntry() = default;
+
+  explicit ExpiringCacheMapEntry(std::shared_ptr<ConcurrentMap<V>> cached_item,
+                                 uint64_t expiration_interval_millis)
+      : map_cache_(cached_item, expiration_interval_millis) {}
+
+  bool IsExpired() { return map_cache_.IsExpired(); }
+
+  std::shared_ptr<ConcurrentMap<V>> cached_item() { return map_cache_.cached_item(); }
+
+ private:
+  ExpiringCacheEntry<std::shared_ptr<ConcurrentMap<V>>> map_cache_;

Review comment:
       Why is this using a `ConcurrentMap` if you already have a lock in the top-level two-level cache?

##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)

Review comment:
       I'd much rather if you passed a number of seconds, e.g. `double expiration_delay`

##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)
+      : cached_item_(cached_item) {
+    expiration_timestamp_ =
+        CurrentTimePoint() + std::chrono::milliseconds(expiration_interval_millis);
+  }
+
+  bool IsExpired() {
+    auto now = CurrentTimePoint();
+    return (now > expiration_timestamp_);
+  }
+
+  E cached_item() { return cached_item_; }
+
+ private:
+  TimePoint expiration_timestamp_;
+  E cached_item_;
+};
+
+// This class is to avoid the below warning when compiling KeyToolkit class with VS2015
+// warning C4503: decorated name length exceeded, name was truncated
+template <typename V>
+class ExpiringCacheMapEntry {
+ public:
+  ExpiringCacheMapEntry() = default;
+
+  explicit ExpiringCacheMapEntry(std::shared_ptr<ConcurrentMap<V>> cached_item,
+                                 uint64_t expiration_interval_millis)
+      : map_cache_(cached_item, expiration_interval_millis) {}
+
+  bool IsExpired() { return map_cache_.IsExpired(); }
+
+  std::shared_ptr<ConcurrentMap<V>> cached_item() { return map_cache_.cached_item(); }
+
+ private:
+  ExpiringCacheEntry<std::shared_ptr<ConcurrentMap<V>>> map_cache_;
+};
+
+}  // namespace internal
+
+// Two-level cache with expiration of internal caches according to token lifetime.
+// External cache is per token, internal is per string key.
+// Wrapper class around:
+//    std::unordered_map<std::string,
+//    internal::ExpiringCacheEntry<std::unordered_map<std::string, V>>>
+// This cache is safe to be shared between threads.
+template <typename V>
+class TwoLevelCacheWithExpiration {
+ public:
+  TwoLevelCacheWithExpiration() {
+    last_cache_cleanup_timestamp_ = internal::CurrentTimePoint();
+  }
+
+  std::shared_ptr<ConcurrentMap<V>> GetOrCreateInternalCache(
+      const std::string& access_token, uint64_t cache_entry_lifetime_ms) {
+    auto lock = mutex_.Lock();
+
+    auto external_cache_entry = cache_.find(access_token);
+    if (external_cache_entry == cache_.end() ||
+        external_cache_entry->second.IsExpired()) {
+      cache_.insert(
+          {access_token, internal::ExpiringCacheMapEntry<V>(
+                             std::shared_ptr<ConcurrentMap<V>>(new ConcurrentMap<V>()),
+                             cache_entry_lifetime_ms)});
+    }
+
+    return cache_[access_token].cached_item();
+  }
+
+  void RemoveCacheEntriesForToken(const std::string& access_token) {
+    auto lock = mutex_.Lock();
+    cache_.erase(access_token);
+  }
+
+  void RemoveCacheEntriesForAllTokens() {
+    auto lock = mutex_.Lock();
+    cache_.clear();
+  }
+
+  void CheckCacheForExpiredTokens(uint64_t cache_cleanup_period_ms) {
+    auto lock = mutex_.Lock();
+
+    internal::TimePoint now = internal::CurrentTimePoint();
+
+    if (now > (last_cache_cleanup_timestamp_ +
+               std::chrono::milliseconds(cache_cleanup_period_ms))) {
+      RemoveExpiredEntriesNoMutex();
+      last_cache_cleanup_timestamp_ =
+          now + std::chrono::milliseconds(cache_cleanup_period_ms);
+    }
+  }
+
+  void RemoveExpiredEntriesFromCache() {
+    auto lock = mutex_.Lock();
+
+    RemoveExpiredEntriesNoMutex();
+  }
+
+  void Remove(const std::string& access_token) {
+    auto lock = mutex_.Lock();
+    cache_.erase(access_token);
+  }
+
+  void Clear() {

Review comment:
       There's already `RemoveCacheEntriesForAllTokens`. Keep one or the other.

##########
File path: cpp/src/parquet/CMakeLists.txt
##########
@@ -388,9 +403,20 @@ add_parquet_test(arrow-internals-test
 if(PARQUET_REQUIRE_ENCRYPTION)
   add_parquet_test(encryption-test
                    SOURCES
-                   encryption_write_configurations_test.cc
-                   encryption_read_configurations_test.cc
-                   encryption_properties_test.cc
+                   encryption/encryption_write_configurations_test.cc

Review comment:
       No need to prefix file names with `encryption_` if they are already in an `encryption` subdirectory.

##########
File path: cpp/src/parquet/encryption/test_encryption_util.h
##########
@@ -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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#pragma once
+
+#include <algorithm>
+#include <chrono>
+#include <limits>
+#include <memory>
+#include <random>
+#include <string>
+#include <utility>
+#include <vector>

Review comment:
       Once you have removed the `inline` functions, you can also reduce the number of headers here.

##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)
+      : cached_item_(cached_item) {
+    expiration_timestamp_ =
+        CurrentTimePoint() + std::chrono::milliseconds(expiration_interval_millis);
+  }
+
+  bool IsExpired() {
+    auto now = CurrentTimePoint();
+    return (now > expiration_timestamp_);
+  }
+
+  E cached_item() { return cached_item_; }
+
+ private:
+  TimePoint expiration_timestamp_;
+  E cached_item_;
+};
+
+// This class is to avoid the below warning when compiling KeyToolkit class with VS2015
+// warning C4503: decorated name length exceeded, name was truncated
+template <typename V>
+class ExpiringCacheMapEntry {
+ public:
+  ExpiringCacheMapEntry() = default;
+
+  explicit ExpiringCacheMapEntry(std::shared_ptr<ConcurrentMap<V>> cached_item,
+                                 uint64_t expiration_interval_millis)
+      : map_cache_(cached_item, expiration_interval_millis) {}
+
+  bool IsExpired() { return map_cache_.IsExpired(); }
+
+  std::shared_ptr<ConcurrentMap<V>> cached_item() { return map_cache_.cached_item(); }
+
+ private:
+  ExpiringCacheEntry<std::shared_ptr<ConcurrentMap<V>>> map_cache_;
+};
+
+}  // namespace internal
+
+// Two-level cache with expiration of internal caches according to token lifetime.
+// External cache is per token, internal is per string key.
+// Wrapper class around:
+//    std::unordered_map<std::string,
+//    internal::ExpiringCacheEntry<std::unordered_map<std::string, V>>>
+// This cache is safe to be shared between threads.
+template <typename V>
+class TwoLevelCacheWithExpiration {
+ public:
+  TwoLevelCacheWithExpiration() {
+    last_cache_cleanup_timestamp_ = internal::CurrentTimePoint();
+  }
+
+  std::shared_ptr<ConcurrentMap<V>> GetOrCreateInternalCache(
+      const std::string& access_token, uint64_t cache_entry_lifetime_ms) {

Review comment:
       I'd rather not have durations in milliseconds everywhere. This is a recipe for confusion.

##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)
+      : cached_item_(cached_item) {
+    expiration_timestamp_ =
+        CurrentTimePoint() + std::chrono::milliseconds(expiration_interval_millis);
+  }
+
+  bool IsExpired() {
+    auto now = CurrentTimePoint();
+    return (now > expiration_timestamp_);
+  }
+
+  E cached_item() { return cached_item_; }
+
+ private:
+  TimePoint expiration_timestamp_;
+  E cached_item_;
+};
+
+// This class is to avoid the below warning when compiling KeyToolkit class with VS2015
+// warning C4503: decorated name length exceeded, name was truncated
+template <typename V>
+class ExpiringCacheMapEntry {
+ public:
+  ExpiringCacheMapEntry() = default;
+
+  explicit ExpiringCacheMapEntry(std::shared_ptr<ConcurrentMap<V>> cached_item,
+                                 uint64_t expiration_interval_millis)
+      : map_cache_(cached_item, expiration_interval_millis) {}
+
+  bool IsExpired() { return map_cache_.IsExpired(); }
+
+  std::shared_ptr<ConcurrentMap<V>> cached_item() { return map_cache_.cached_item(); }
+
+ private:
+  ExpiringCacheEntry<std::shared_ptr<ConcurrentMap<V>>> map_cache_;
+};
+
+}  // namespace internal
+
+// Two-level cache with expiration of internal caches according to token lifetime.
+// External cache is per token, internal is per string key.
+// Wrapper class around:
+//    std::unordered_map<std::string,
+//    internal::ExpiringCacheEntry<std::unordered_map<std::string, V>>>
+// This cache is safe to be shared between threads.
+template <typename V>
+class TwoLevelCacheWithExpiration {
+ public:
+  TwoLevelCacheWithExpiration() {
+    last_cache_cleanup_timestamp_ = internal::CurrentTimePoint();
+  }
+
+  std::shared_ptr<ConcurrentMap<V>> GetOrCreateInternalCache(
+      const std::string& access_token, uint64_t cache_entry_lifetime_ms) {
+    auto lock = mutex_.Lock();
+
+    auto external_cache_entry = cache_.find(access_token);
+    if (external_cache_entry == cache_.end() ||
+        external_cache_entry->second.IsExpired()) {
+      cache_.insert(
+          {access_token, internal::ExpiringCacheMapEntry<V>(
+                             std::shared_ptr<ConcurrentMap<V>>(new ConcurrentMap<V>()),
+                             cache_entry_lifetime_ms)});
+    }
+
+    return cache_[access_token].cached_item();
+  }
+
+  void RemoveCacheEntriesForToken(const std::string& access_token) {
+    auto lock = mutex_.Lock();
+    cache_.erase(access_token);
+  }
+
+  void RemoveCacheEntriesForAllTokens() {
+    auto lock = mutex_.Lock();
+    cache_.clear();
+  }
+
+  void CheckCacheForExpiredTokens(uint64_t cache_cleanup_period_ms) {

Review comment:
       Pass this as seconds?

##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"

Review comment:
       Why is this included?

##########
File path: cpp/src/arrow/util/string.cc
##########
@@ -92,6 +92,20 @@ Status ParseHexValue(const char* data, uint8_t* out) {
 
 namespace internal {
 
+std::vector<std::string> SplitString(util::string_view v, char delimiter) {

Review comment:
       Does this function work if I pass e.g. `"a:b:c:", ':'`?

##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;

Review comment:
       Please don't put `using` directives at the top-level in include files.

##########
File path: cpp/src/parquet/encryption/test_encryption_util.h
##########
@@ -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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#pragma once
+
+#include <algorithm>
+#include <chrono>
+#include <limits>
+#include <memory>
+#include <random>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/status.h"
+#include "arrow/testing/util.h"
+#include "arrow/util/io_util.h"
+
+#include "parquet/column_page.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
+#include "parquet/encoding.h"
+#include "parquet/encryption/encryption.h"
+#include "parquet/platform.h"
+#include "parquet/test_util.h"
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+using arrow::internal::TemporaryDir;
+
+using parquet::ConvertedType;
+using parquet::Repetition;
+using parquet::Type;
+using schema::GroupNode;
+using schema::NodePtr;
+using schema::PrimitiveNode;
+
+constexpr int kFixedLength = 10;
+
+const char kFooterEncryptionKey[] = "0123456789012345";  // 128bit/16
+const char kColumnEncryptionKey1[] = "1234567890123450";
+const char kColumnEncryptionKey2[] = "1234567890123451";
+const char kFileName[] = "tester";
+
+inline std::string data_file(const char* file) {

Review comment:
       Instead of putting these functions `inline`, can you define them to `test_encryption_util.cc`?

##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,115 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT RemoteKmsClient : public KmsClient {
+ public:
+  static constexpr const char kLocalWrapNoKeyVersion[] = "NO_VERSION";
+
+  /// If is_wrap_locally is true, master keys will be fetched from the KMS server and used
+  /// to encrypt other keys (DEKs or KEKs).
+  /// If is_wrap_locally is false, keys will be encrypted inside KMS server. This is
+  /// the preferred mode, since it is safer and also allows for key rotation.
+  void Initialize(const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally);
+
+  std::string WrapKey(const std::string& key_bytes,
+                      const std::string& master_key_identifier) override;
+
+  std::string UnwrapKey(const std::string& wrapped_key,
+                        const std::string& master_key_identifier) override;
+
+ protected:
+  /// Wrap a key with the master key in the remote KMS server.
+  /// Note: this function might be called by multiple threads
+  virtual std::string WrapKeyInServer(const std::string& key_bytes,
+                                      const std::string& master_key_identifier) = 0;
+
+  /// Unwrap a key with the master key in the remote KMS server.
+  /// Note: this function might be called by multiple threads
+  virtual std::string UnwrapKeyInServer(const std::string& wrapped_key,
+                                        const std::string& master_key_identifier) = 0;
+
+  /// Get master key from the remote KMS server.
+  /// Required only for local wrapping. No need to implement if KMS supports in-server
+  /// wrapping/unwrapping.
+  /// Note: this function might be called by multiple threads
+  virtual std::string GetMasterKeyFromServer(
+      const std::string& master_key_identifier) = 0;
+
+  virtual void InitializeInternal() = 0;
+
+ private:
+  /// KMS systems wrap keys by encrypting them by master keys, and attaching additional
+  /// information (such as the version number of the masker key) to the result of
+  /// encryption. The master key version is required in  key rotation. Currently, the
+  /// local wrapping mode does not support key rotation (because not all KMS systems allow
+  /// to fetch a master key by its ID and version number). Still, the local wrapping mode
+  /// adds a placeholder for the master key version, that will enable support for key
+  /// rotation in this mode in the future, with appropriate KMS systems. This will also
+  /// enable backward compatibility, where future readers will be able to extract master
+  /// key version in the files written by the current code.
+  ///
+  /// LocalKeyWrap class writes (and reads) the "key wrap" as a flat json with the
+  /// following fields:
+  /// 1. "masterKeyVersion" - a String, with the master key version. In the current
+  /// version, only one value is allowed - "NO_VERSION".
+  /// 2. "encryptedKey" - a String, with the key encrypted by the master key
+  /// (base64-encoded).
+  class LocalKeyWrap {
+   public:
+    static constexpr const char kLocalWrapKeyVersionField[] = "masterKeyVersion";
+    static constexpr const char kLocalWrapEncryptedKeyField[] = "encryptedKey";
+
+    LocalKeyWrap(const std::string& master_key_version,
+                 const std::string& encrypted_encoded_key);
+
+    static std::string CreateSerialized(const std::string& encrypted_encoded_key);
+
+    static LocalKeyWrap Parse(const std::string& wrapped_key);
+
+    const std::string& master_key_version() const { return master_key_version_; }
+
+    const std::string& encrypted_encoded_key() const { return encrypted_encoded_key_; }
+
+   private:
+    std::string encrypted_encoded_key_;
+    std::string master_key_version_;
+  };
+
+  std::string GetKeyFromServer(const std::string& key_identifier);
+
+ protected:
+  KmsConnectionConfig kms_connection_config_;
+  std::atomic<bool> is_wrap_locally_;
+  std::atomic<bool> is_default_token_;

Review comment:
       Why are these `atomic`?

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {

Review comment:
       Hm, just a style thing, but can we use shorter names? Why do you need to mention that's it's "properties-driven"? Is there another factory that's not properties-driven (if so, why)?

##########
File path: cpp/src/parquet/encryption/remote_kms_client.h
##########
@@ -0,0 +1,115 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "arrow/util/concurrent_map.h"

Review comment:
       Ideally, maps and caches should be hidden behind a `pimpl`.

##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;

Review comment:
       It would be less confusing to keep `FileOutputStream`.

##########
File path: cpp/src/parquet/encryption/remote_kms_client.cc
##########
@@ -0,0 +1,127 @@
+// 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.
+
+#include "arrow/json/object_parser.h"
+#include "arrow/json/object_writer.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/remote_kms_client.h"
+#include "parquet/exception.h"
+
+using arrow::json::ObjectParser;
+using arrow::json::ObjectWriter;
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char RemoteKmsClient::kLocalWrapNoKeyVersion[];
+
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapKeyVersionField[];
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapEncryptedKeyField[];
+
+RemoteKmsClient::LocalKeyWrap::LocalKeyWrap(const std::string& master_key_version,
+                                            const std::string& encrypted_encoded_key)
+    : encrypted_encoded_key_(encrypted_encoded_key),
+      master_key_version_(master_key_version) {}
+
+std::string RemoteKmsClient::LocalKeyWrap::CreateSerialized(
+    const std::string& encrypted_encoded_key) {
+  ObjectWriter json_writer;
+
+  json_writer.SetString(kLocalWrapKeyVersionField, kLocalWrapNoKeyVersion);
+  json_writer.SetString(kLocalWrapEncryptedKeyField, encrypted_encoded_key);
+
+  return json_writer.Serialize();
+}
+
+RemoteKmsClient::LocalKeyWrap RemoteKmsClient::LocalKeyWrap::Parse(
+    const std::string& wrapped_key) {
+  ObjectParser json_parser;
+  if (!json_parser.Parse(wrapped_key)) {
+    throw ParquetException("Failed to parse local key wrap json " + wrapped_key);
+  }
+  std::string master_key_version;
+  PARQUET_ASSIGN_OR_THROW(master_key_version,
+                          json_parser.GetString(kLocalWrapKeyVersionField));
+
+  std::string encrypted_encoded_key;
+  PARQUET_ASSIGN_OR_THROW(encrypted_encoded_key,
+                          json_parser.GetString(kLocalWrapEncryptedKeyField));
+
+  return RemoteKmsClient::LocalKeyWrap(master_key_version, encrypted_encoded_key);
+}
+
+void RemoteKmsClient::Initialize(const KmsConnectionConfig& kms_connection_config,
+                                 bool is_wrap_locally) {
+  kms_connection_config_ = kms_connection_config;
+  is_wrap_locally_ = is_wrap_locally;
+  if (is_wrap_locally_) {
+    master_key_cache_.Clear();
+  }
+
+  is_default_token_ =
+      kms_connection_config_.key_access_token() == KmsClient::kKeyAccessTokenDefault;
+
+  InitializeInternal();
+}
+
+std::string RemoteKmsClient::WrapKey(const std::string& key_bytes,
+                                     const std::string& master_key_identifier) {
+  if (is_wrap_locally_) {

Review comment:
       I don't understand this. It looks like you have basically two distinct implementations of `RemoteKmsClient`?
   Why don't you have two separate concrete classes for them, instead of writing all those switches by hand?

##########
File path: cpp/src/parquet/encryption/test_encryption_util.h
##########
@@ -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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#pragma once
+
+#include <algorithm>
+#include <chrono>
+#include <limits>
+#include <memory>
+#include <random>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/status.h"
+#include "arrow/testing/util.h"
+#include "arrow/util/io_util.h"
+
+#include "parquet/column_page.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
+#include "parquet/encoding.h"
+#include "parquet/encryption/encryption.h"
+#include "parquet/platform.h"
+#include "parquet/test_util.h"
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+using arrow::internal::TemporaryDir;
+
+using parquet::ConvertedType;
+using parquet::Repetition;
+using parquet::Type;
+using schema::GroupNode;
+using schema::NodePtr;
+using schema::PrimitiveNode;
+
+constexpr int kFixedLength = 10;
+
+const char kFooterEncryptionKey[] = "0123456789012345";  // 128bit/16
+const char kColumnEncryptionKey1[] = "1234567890123450";
+const char kColumnEncryptionKey2[] = "1234567890123451";
+const char kFileName[] = "tester";
+
+inline std::string data_file(const char* file) {
+  std::string dir_string(parquet::test::get_data_dir());
+  std::stringstream ss;
+  ss << dir_string << "/" << file;
+  return ss.str();
+}
+
+// A temporary directory that contains the encrypted files generated in the tests.
+extern std::unique_ptr<TemporaryDir> temp_dir;
+
+inline arrow::Result<std::unique_ptr<TemporaryDir>> temp_data_dir() {
+  arrow::Result<std::unique_ptr<TemporaryDir>> dir;
+  ARROW_ASSIGN_OR_RAISE(dir, TemporaryDir::Make("parquet-encryption-test-"));
+  return dir;

Review comment:
       This could be simplified to (untested):
   ```c++
   arrow::Result<std::unique_ptr<TemporaryDir>> temp_data_dir() {
     return TemporaryDir::Make("parquet-encryption-test-");
   }
   ```
   

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {
+ public:
+  /// KmsClientFactory must be registered before calling
+  /// GetFileEncryptionProperties()/GetFileDecryptionProperties() methods.
+  void RegisterKmsClientFactory(std::shared_ptr<KmsClientFactory> kms_client_factory);
+
+  std::shared_ptr<FileEncryptionProperties> GetFileEncryptionProperties(
+      const KmsConnectionConfig& kms_connection_config,
+      std::shared_ptr<EncryptionConfiguration> encryption_config);
+
+  /// The returned FileDecryptionProperties object will use the cache inside this
+  /// PropertiesDrivenCryptoFactory object, so please keep this
+  /// PropertiesDrivenCryptoFactory object alive along with the returned
+  /// FileDecryptionProperties object.
+  std::shared_ptr<FileDecryptionProperties> GetFileDecryptionProperties(
+      const KmsConnectionConfig& kms_connection_config,
+      std::shared_ptr<DecryptionConfiguration> decryption_config);
+
+  void RemoveCacheEntriesForToken(const std::string& access_token) {
+    key_toolkit_.RemoveCacheEntriesForToken(access_token);
+  }
+
+  void RemoveCacheEntriesForAllTokens() { key_toolkit_.RemoveCacheEntriesForAllTokens(); }
+
+ private:
+  /// Acceptable data key lengths in number of bits
+  static constexpr const int32_t kAcceptableDataKeyLengths[] = {128, 192, 256};

Review comment:
       Hmm... what is it indexed with? Why is `128` the first entry, for example?

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {
+ public:
+  /// KmsClientFactory must be registered before calling

Review comment:
       Hmm... what does "registered" mean here?

##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED

Review comment:
       Are these comments required? They basically paraphrase what's being done in the code :-)

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;

Review comment:
       So you have ten trivial accessors, plus a constructor with ten explicit arguments, plus a `Builder` subclass with ten setter methods... I don't understand the point of this. `EncryptionConfiguration` can be a simple struct with public attributes...

##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);

Review comment:
       I would question the usefulness of all these method indirections. Instead, you could also make it a plain struct with public fields.

##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED
+  fields.push_back(PrimitiveNode::Make(kBooleanFieldName, Repetition::REQUIRED,
+                                       Type::BOOLEAN, ConvertedType::NONE));
+
+  // Create a primitive node named 'int32_field' with type:INT32, repetition:REQUIRED,
+  // logical type:TIME_MILLIS
+  fields.push_back(PrimitiveNode::Make(kInt32FieldName, Repetition::REQUIRED, Type::INT32,
+                                       ConvertedType::TIME_MILLIS));
+
+  // Create a primitive node named 'int64_field' with type:INT64, repetition:REPEATED
+  fields.push_back(PrimitiveNode::Make(kInt64FieldName, Repetition::REPEATED, Type::INT64,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kInt96FieldName, Repetition::REQUIRED, Type::INT96,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kFloatFieldName, Repetition::REQUIRED, Type::FLOAT,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kDoubleFieldName, Repetition::REQUIRED,
+                                       Type::DOUBLE, ConvertedType::NONE));
+
+  // Create a primitive node named 'ba_field' with type:BYTE_ARRAY, repetition:OPTIONAL
+  fields.push_back(PrimitiveNode::Make(kByteArrayFieldName, Repetition::OPTIONAL,
+                                       Type::BYTE_ARRAY, ConvertedType::NONE));
+
+  // Create a primitive node named 'flba_field' with type:FIXED_LEN_BYTE_ARRAY,
+  // repetition:REQUIRED, field_length = kFixedLength
+  fields.push_back(PrimitiveNode::Make(kFixedLenByteArrayFieldName, Repetition::REQUIRED,
+                                       Type::FIXED_LEN_BYTE_ARRAY, ConvertedType::NONE,
+                                       kFixedLength));

Review comment:
       I'm curious, is it useful to test all these different types? I would expect all primitive types (bool, int, float...) to behave the same.

##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED
+  fields.push_back(PrimitiveNode::Make(kBooleanFieldName, Repetition::REQUIRED,
+                                       Type::BOOLEAN, ConvertedType::NONE));
+
+  // Create a primitive node named 'int32_field' with type:INT32, repetition:REQUIRED,
+  // logical type:TIME_MILLIS
+  fields.push_back(PrimitiveNode::Make(kInt32FieldName, Repetition::REQUIRED, Type::INT32,
+                                       ConvertedType::TIME_MILLIS));
+
+  // Create a primitive node named 'int64_field' with type:INT64, repetition:REPEATED
+  fields.push_back(PrimitiveNode::Make(kInt64FieldName, Repetition::REPEATED, Type::INT64,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kInt96FieldName, Repetition::REQUIRED, Type::INT96,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kFloatFieldName, Repetition::REQUIRED, Type::FLOAT,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kDoubleFieldName, Repetition::REQUIRED,
+                                       Type::DOUBLE, ConvertedType::NONE));
+
+  // Create a primitive node named 'ba_field' with type:BYTE_ARRAY, repetition:OPTIONAL
+  fields.push_back(PrimitiveNode::Make(kByteArrayFieldName, Repetition::OPTIONAL,
+                                       Type::BYTE_ARRAY, ConvertedType::NONE));
+
+  // Create a primitive node named 'flba_field' with type:FIXED_LEN_BYTE_ARRAY,
+  // repetition:REQUIRED, field_length = kFixedLength
+  fields.push_back(PrimitiveNode::Make(kFixedLenByteArrayFieldName, Repetition::REQUIRED,
+                                       Type::FIXED_LEN_BYTE_ARRAY, ConvertedType::NONE,
+                                       kFixedLength));
+
+  // Create a GroupNode named 'schema' using the primitive nodes defined above
+  // This GroupNode is the root node of the schema tree
+  return std::static_pointer_cast<GroupNode>(
+      GroupNode::Make("schema", Repetition::REQUIRED, fields));
+}
+
+void FileEncryptor::EncryptFile(
+    std::string file,
+    std::shared_ptr<parquet::FileEncryptionProperties> encryption_configurations) {
+  WriterProperties::Builder prop_builder;
+  prop_builder.compression(parquet::Compression::SNAPPY);
+  prop_builder.encryption(encryption_configurations);
+  std::shared_ptr<WriterProperties> writer_properties = prop_builder.build();
+
+  PARQUET_ASSIGN_OR_THROW(auto out_file, FileClass::Open(file));
+  // Create a ParquetFileWriter instance
+  std::shared_ptr<parquet::ParquetFileWriter> file_writer =
+      parquet::ParquetFileWriter::Open(out_file, schema_, writer_properties);
+
+  for (int r = 0; r < num_rgs; r++) {
+    bool buffered_mode = r % 2 == 0;
+    auto row_group_writer = buffered_mode ? file_writer->AppendBufferedRowGroup()
+                                          : file_writer->AppendRowGroup();
+
+    int column_index = 0;
+    // Captures i by reference; increments it by one
+    auto get_next_column = [&]() {
+      return buffered_mode ? row_group_writer->column(column_index++)
+                           : row_group_writer->NextColumn();
+    };
+
+    // Write the Bool column
+    parquet::BoolWriter* bool_writer =
+        static_cast<parquet::BoolWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      bool value = ((i % 2) == 0) ? true : false;
+      bool_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Int32 column
+    parquet::Int32Writer* int32_writer =
+        static_cast<parquet::Int32Writer*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      int32_t value = i;
+      int32_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Int64 column. Each row has repeats twice.
+    parquet::Int64Writer* int64_writer =
+        static_cast<parquet::Int64Writer*>(get_next_column());
+    for (int i = 0; i < 2 * rows_per_rowgroup_; i++) {
+      int64_t value = i * 1000 * 1000;
+      value *= 1000 * 1000;
+      int16_t definition_level = 1;
+      int16_t repetition_level = 0;
+      if ((i % 2) == 0) {
+        repetition_level = 1;  // start of a new record
+      }
+      int64_writer->WriteBatch(1, &definition_level, &repetition_level, &value);
+    }
+
+    // Write the INT96 column.
+    parquet::Int96Writer* int96_writer =
+        static_cast<parquet::Int96Writer*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::Int96 value;
+      value.value[0] = i;
+      value.value[1] = i + 1;
+      value.value[2] = i + 2;
+      int96_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Float column
+    parquet::FloatWriter* float_writer =
+        static_cast<parquet::FloatWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      float value = static_cast<float>(i) * 1.1f;
+      float_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Double column
+    parquet::DoubleWriter* double_writer =
+        static_cast<parquet::DoubleWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      double value = i * 1.1111111;
+      double_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the ByteArray column. Make every alternate values NULL
+    parquet::ByteArrayWriter* ba_writer =
+        static_cast<parquet::ByteArrayWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::ByteArray value;
+      char hello[kFixedLength] = "parquet";
+      hello[7] = static_cast<char>(static_cast<int>('0') + i / 100);
+      hello[8] = static_cast<char>(static_cast<int>('0') + (i / 10) % 10);
+      hello[9] = static_cast<char>(static_cast<int>('0') + i % 10);
+      if (i % 2 == 0) {
+        int16_t definition_level = 1;
+        value.ptr = reinterpret_cast<const uint8_t*>(&hello[0]);
+        value.len = kFixedLength;
+        ba_writer->WriteBatch(1, &definition_level, nullptr, &value);
+      } else {
+        int16_t definition_level = 0;
+        ba_writer->WriteBatch(1, &definition_level, nullptr, nullptr);
+      }
+    }
+
+    // Write the FixedLengthByteArray column
+    parquet::FixedLenByteArrayWriter* flba_writer =
+        static_cast<parquet::FixedLenByteArrayWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::FixedLenByteArray value;
+      char v = static_cast<char>(i);
+      char flba[kFixedLength] = {v, v, v, v, v, v, v, v, v, v};
+      value.ptr = reinterpret_cast<const uint8_t*>(&flba[0]);
+      flba_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+  }
+
+  // Close the ParquetFileWriter
+  file_writer->Close();
+
+  return;
+}
+
+void FileDecryptor::DecryptFile(
+    std::string file,
+    std::shared_ptr<FileDecryptionProperties> file_decryption_properties) {
+  std::string exception_msg;
+  parquet::ReaderProperties reader_properties = parquet::default_reader_properties();
+  if (file_decryption_properties) {
+    reader_properties.file_decryption_properties(file_decryption_properties->DeepClone());
+  }
+
+  auto file_reader = parquet::ParquetFileReader::OpenFile(file, false, reader_properties);
+
+  // Get the File MetaData
+  std::shared_ptr<parquet::FileMetaData> file_metadata = file_reader->metadata();
+
+  // Get the number of RowGroups
+  int num_row_groups = file_metadata->num_row_groups();
+
+  // Get the number of Columns
+  int num_columns = file_metadata->num_columns();
+  ASSERT_EQ(num_columns, 8);
+
+  // Iterate over all the RowGroups in the file
+  for (int r = 0; r < num_row_groups; ++r) {
+    // Get the RowGroup Reader
+    std::shared_ptr<parquet::RowGroupReader> row_group_reader = file_reader->RowGroup(r);
+
+    // Get the RowGroupMetaData
+    std::unique_ptr<RowGroupMetaData> rg_metadata = file_metadata->RowGroup(r);
+
+    int64_t values_read = 0;
+    int64_t rows_read = 0;
+    int16_t definition_level;
+    int16_t repetition_level;
+    int i;
+    std::shared_ptr<parquet::ColumnReader> column_reader;
+
+    // Get the Column Reader for the boolean column
+    column_reader = row_group_reader->Column(0);
+    parquet::BoolReader* bool_reader =
+        static_cast<parquet::BoolReader*>(column_reader.get());
+
+    // Get the ColumnChunkMetaData for the boolean column
+    std::unique_ptr<ColumnChunkMetaData> boolean_md = rg_metadata->ColumnChunk(0);
+
+    // Read all the rows in the column
+    i = 0;
+    while (bool_reader->HasNext()) {
+      bool value;
+      // Read one value at a time. The number of rows read is returned. values_read
+      // contains the number of non-null rows
+      rows_read = bool_reader->ReadBatch(1, nullptr, nullptr, &value, &values_read);

Review comment:
       It would be much better to read several values at a time...

##########
File path: cpp/src/parquet/encryption/key_toolkit_internal.h
##########
@@ -0,0 +1,52 @@
+// 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 <chrono>
+#include <string>
+
+namespace parquet {
+namespace encryption {
+namespace internal {
+
+// "data encryption key" and "master key identifier" are paired together as output when
+// parsing from "key material"
+class KeyWithMasterId {
+ public:
+  KeyWithMasterId(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  const std::string key_bytes_;
+  const std::string master_id_;
+};
+
+/// Encrypts "key" with "master_key", using AES-GCM and the "aad"

Review comment:
       What's the rationale for AES-GCM? Also, no mention of the key size?

##########
File path: cpp/src/parquet/encryption/kms_client.cc
##########
@@ -0,0 +1,47 @@
+// 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.
+
+#include "parquet/encryption/kms_client.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char KmsClient::kKmsInstanceIdDefault[];
+constexpr const char KmsClient::kKmsInstanceUrlDefault[];
+constexpr const char KmsClient::kKeyAccessTokenDefault[];

Review comment:
       So these are all empty strings, right? I'm trying to make sure I don't miss anything :-)

##########
File path: cpp/src/parquet/encryption/key_toolkit_internal.h
##########
@@ -0,0 +1,52 @@
+// 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 <chrono>

Review comment:
       Hmm, can you clean up any unneeded headers? In C++, each additional header adds compilation time.

##########
File path: cpp/src/parquet/encryption/file_key_unwrapper.h
##########
@@ -0,0 +1,66 @@
+// 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 "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/key_material.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class will retrieve the key from "key metadata", following these steps:
+// 1. Parse "key metadata" (see structure in KeyMetadata class).
+// 2. Retrieve "key material" which can be stored inside or outside "key metadata"
+//    Currently we don't support the case "key material" stores outside "key metadata"
+//    yet.
+// 3. Unwrap the "data encryption key" from "key material". There are 2 modes:
+// 3.1. single wrapping: decrypt the wrapped "data encryption key" directly with "master
+// encryption key" 3.2. double wrapping: 2 steps: 3.2.1. "key encryption key" is decrypted
+// with "master encryption key" 3.2.2. "data encryption key" is decrypted with the above
+// "key encryption key"
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  /// 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.
+  FileKeyUnwrapper(KeyToolkit* key_toolkit,

Review comment:
       Is it normal to pass a raw pointer here?

##########
File path: cpp/src/parquet/encryption/file_key_unwrapper.h
##########
@@ -0,0 +1,66 @@
+// 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 "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/key_material.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class will retrieve the key from "key metadata", following these steps:
+// 1. Parse "key metadata" (see structure in KeyMetadata class).
+// 2. Retrieve "key material" which can be stored inside or outside "key metadata"
+//    Currently we don't support the case "key material" stores outside "key metadata"
+//    yet.
+// 3. Unwrap the "data encryption key" from "key material". There are 2 modes:
+// 3.1. single wrapping: decrypt the wrapped "data encryption key" directly with "master
+// encryption key" 3.2. double wrapping: 2 steps: 3.2.1. "key encryption key" is decrypted
+// with "master encryption key" 3.2.2. "data encryption key" is decrypted with the above
+// "key encryption key"
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  /// 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.
+  FileKeyUnwrapper(KeyToolkit* key_toolkit,
+                   const KmsConnectionConfig& kms_connection_config,
+                   uint64_t cache_lifetime_seconds);
+
+  std::string GetKey(const std::string& key_metadata) const override;
+
+ private:
+  internal::KeyWithMasterId GetDataEncryptionKey(const KeyMaterial& key_material) const;
+  std::shared_ptr<KmsClient> GetKmsClientFromConfigOrKeyMaterial(
+      const KeyMaterial& key_material) const;
+
+  /// A map of Key Encryption Key (KEK) ID -> KEK bytes, for the current token
+  mutable std::shared_ptr<arrow::util::ConcurrentMap<std::string>> kek_per_kek_id_;
+  KeyToolkit* key_toolkit_;
+  mutable KmsConnectionConfig kms_connection_config_;

Review comment:
       Why is the configuration mutable?

##########
File path: cpp/src/parquet/encryption/kms_client.h
##########
@@ -0,0 +1,84 @@
+// 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 <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class wraps the key access token of a KMS server. If your token changes over time,
+// you should keep the reference to the KeyAccessToken object and call Refresh() method
+// every time you have a new token.
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();

Review comment:
       And the default itself is empty? Is it planned to change to something else, or is this method just irrelevant? :-)

##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED
+  fields.push_back(PrimitiveNode::Make(kBooleanFieldName, Repetition::REQUIRED,
+                                       Type::BOOLEAN, ConvertedType::NONE));
+
+  // Create a primitive node named 'int32_field' with type:INT32, repetition:REQUIRED,
+  // logical type:TIME_MILLIS
+  fields.push_back(PrimitiveNode::Make(kInt32FieldName, Repetition::REQUIRED, Type::INT32,
+                                       ConvertedType::TIME_MILLIS));
+
+  // Create a primitive node named 'int64_field' with type:INT64, repetition:REPEATED
+  fields.push_back(PrimitiveNode::Make(kInt64FieldName, Repetition::REPEATED, Type::INT64,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kInt96FieldName, Repetition::REQUIRED, Type::INT96,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kFloatFieldName, Repetition::REQUIRED, Type::FLOAT,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kDoubleFieldName, Repetition::REQUIRED,
+                                       Type::DOUBLE, ConvertedType::NONE));
+
+  // Create a primitive node named 'ba_field' with type:BYTE_ARRAY, repetition:OPTIONAL
+  fields.push_back(PrimitiveNode::Make(kByteArrayFieldName, Repetition::OPTIONAL,
+                                       Type::BYTE_ARRAY, ConvertedType::NONE));
+
+  // Create a primitive node named 'flba_field' with type:FIXED_LEN_BYTE_ARRAY,
+  // repetition:REQUIRED, field_length = kFixedLength
+  fields.push_back(PrimitiveNode::Make(kFixedLenByteArrayFieldName, Repetition::REQUIRED,
+                                       Type::FIXED_LEN_BYTE_ARRAY, ConvertedType::NONE,
+                                       kFixedLength));
+
+  // Create a GroupNode named 'schema' using the primitive nodes defined above
+  // This GroupNode is the root node of the schema tree
+  return std::static_pointer_cast<GroupNode>(
+      GroupNode::Make("schema", Repetition::REQUIRED, fields));
+}
+
+void FileEncryptor::EncryptFile(
+    std::string file,
+    std::shared_ptr<parquet::FileEncryptionProperties> encryption_configurations) {
+  WriterProperties::Builder prop_builder;
+  prop_builder.compression(parquet::Compression::SNAPPY);
+  prop_builder.encryption(encryption_configurations);
+  std::shared_ptr<WriterProperties> writer_properties = prop_builder.build();
+
+  PARQUET_ASSIGN_OR_THROW(auto out_file, FileClass::Open(file));
+  // Create a ParquetFileWriter instance
+  std::shared_ptr<parquet::ParquetFileWriter> file_writer =
+      parquet::ParquetFileWriter::Open(out_file, schema_, writer_properties);
+
+  for (int r = 0; r < num_rgs; r++) {
+    bool buffered_mode = r % 2 == 0;
+    auto row_group_writer = buffered_mode ? file_writer->AppendBufferedRowGroup()
+                                          : file_writer->AppendRowGroup();
+
+    int column_index = 0;
+    // Captures i by reference; increments it by one
+    auto get_next_column = [&]() {
+      return buffered_mode ? row_group_writer->column(column_index++)
+                           : row_group_writer->NextColumn();
+    };
+
+    // Write the Bool column
+    parquet::BoolWriter* bool_writer =
+        static_cast<parquet::BoolWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      bool value = ((i % 2) == 0) ? true : false;
+      bool_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Int32 column
+    parquet::Int32Writer* int32_writer =
+        static_cast<parquet::Int32Writer*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      int32_t value = i;
+      int32_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Int64 column. Each row has repeats twice.
+    parquet::Int64Writer* int64_writer =
+        static_cast<parquet::Int64Writer*>(get_next_column());
+    for (int i = 0; i < 2 * rows_per_rowgroup_; i++) {
+      int64_t value = i * 1000 * 1000;
+      value *= 1000 * 1000;
+      int16_t definition_level = 1;
+      int16_t repetition_level = 0;
+      if ((i % 2) == 0) {
+        repetition_level = 1;  // start of a new record
+      }
+      int64_writer->WriteBatch(1, &definition_level, &repetition_level, &value);
+    }
+
+    // Write the INT96 column.
+    parquet::Int96Writer* int96_writer =
+        static_cast<parquet::Int96Writer*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::Int96 value;
+      value.value[0] = i;
+      value.value[1] = i + 1;
+      value.value[2] = i + 2;
+      int96_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Float column
+    parquet::FloatWriter* float_writer =
+        static_cast<parquet::FloatWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      float value = static_cast<float>(i) * 1.1f;
+      float_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the Double column
+    parquet::DoubleWriter* double_writer =
+        static_cast<parquet::DoubleWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      double value = i * 1.1111111;
+      double_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+
+    // Write the ByteArray column. Make every alternate values NULL
+    parquet::ByteArrayWriter* ba_writer =
+        static_cast<parquet::ByteArrayWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::ByteArray value;
+      char hello[kFixedLength] = "parquet";
+      hello[7] = static_cast<char>(static_cast<int>('0') + i / 100);
+      hello[8] = static_cast<char>(static_cast<int>('0') + (i / 10) % 10);
+      hello[9] = static_cast<char>(static_cast<int>('0') + i % 10);
+      if (i % 2 == 0) {
+        int16_t definition_level = 1;
+        value.ptr = reinterpret_cast<const uint8_t*>(&hello[0]);
+        value.len = kFixedLength;
+        ba_writer->WriteBatch(1, &definition_level, nullptr, &value);
+      } else {
+        int16_t definition_level = 0;
+        ba_writer->WriteBatch(1, &definition_level, nullptr, nullptr);
+      }
+    }
+
+    // Write the FixedLengthByteArray column
+    parquet::FixedLenByteArrayWriter* flba_writer =
+        static_cast<parquet::FixedLenByteArrayWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      parquet::FixedLenByteArray value;
+      char v = static_cast<char>(i);
+      char flba[kFixedLength] = {v, v, v, v, v, v, v, v, v, v};
+      value.ptr = reinterpret_cast<const uint8_t*>(&flba[0]);
+      flba_writer->WriteBatch(1, nullptr, nullptr, &value);
+    }
+  }
+
+  // Close the ParquetFileWriter
+  file_writer->Close();
+
+  return;
+}
+
+void FileDecryptor::DecryptFile(
+    std::string file,
+    std::shared_ptr<FileDecryptionProperties> file_decryption_properties) {
+  std::string exception_msg;
+  parquet::ReaderProperties reader_properties = parquet::default_reader_properties();
+  if (file_decryption_properties) {
+    reader_properties.file_decryption_properties(file_decryption_properties->DeepClone());
+  }
+
+  auto file_reader = parquet::ParquetFileReader::OpenFile(file, false, reader_properties);
+
+  // Get the File MetaData
+  std::shared_ptr<parquet::FileMetaData> file_metadata = file_reader->metadata();
+
+  // Get the number of RowGroups
+  int num_row_groups = file_metadata->num_row_groups();
+
+  // Get the number of Columns
+  int num_columns = file_metadata->num_columns();
+  ASSERT_EQ(num_columns, 8);
+
+  // Iterate over all the RowGroups in the file
+  for (int r = 0; r < num_row_groups; ++r) {
+    // Get the RowGroup Reader
+    std::shared_ptr<parquet::RowGroupReader> row_group_reader = file_reader->RowGroup(r);
+
+    // Get the RowGroupMetaData
+    std::unique_ptr<RowGroupMetaData> rg_metadata = file_metadata->RowGroup(r);
+
+    int64_t values_read = 0;
+    int64_t rows_read = 0;
+    int16_t definition_level;
+    int16_t repetition_level;
+    int i;
+    std::shared_ptr<parquet::ColumnReader> column_reader;
+
+    // Get the Column Reader for the boolean column
+    column_reader = row_group_reader->Column(0);
+    parquet::BoolReader* bool_reader =
+        static_cast<parquet::BoolReader*>(column_reader.get());
+
+    // Get the ColumnChunkMetaData for the boolean column
+    std::unique_ptr<ColumnChunkMetaData> boolean_md = rg_metadata->ColumnChunk(0);
+
+    // Read all the rows in the column
+    i = 0;
+    while (bool_reader->HasNext()) {
+      bool value;
+      // Read one value at a time. The number of rows read is returned. values_read
+      // contains the number of non-null rows
+      rows_read = bool_reader->ReadBatch(1, nullptr, nullptr, &value, &values_read);
+      // Ensure only one value is read
+      ASSERT_EQ(rows_read, 1);
+      // There are no NULL values in the rows written
+      ASSERT_EQ(values_read, 1);
+      // Verify the value written
+      bool expected_value = ((i % 2) == 0) ? true : false;
+      ASSERT_EQ(value, expected_value);
+      i++;
+    }
+    // make sure we got the same number of values the metadata says
+    ASSERT_EQ(boolean_md->num_values(), i);
+
+    // Get the Column Reader for the Int32 column
+    column_reader = row_group_reader->Column(1);
+    parquet::Int32Reader* int32_reader =
+        static_cast<parquet::Int32Reader*>(column_reader.get());
+
+    // Get the ColumnChunkMetaData for the Int32 column
+    std::unique_ptr<ColumnChunkMetaData> int32_md = rg_metadata->ColumnChunk(1);
+
+    // Read all the rows in the column

Review comment:
       There's really a lot of repetition below. I think we could go two ways:
   1) you reduce the number of tested types (I don't understand if there's anything different about them: I don't think so, since encryption seems to be a page-level thing)
   2) or you find a way to factor out all the writing/reading of example values, such that we don't have so much code to read and maintain when going through the encryption tests

##########
File path: cpp/src/parquet/encryption/key_wrapping_test.cc
##########
@@ -0,0 +1,105 @@
+// 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.
+
+#include <gtest/gtest.h>
+
+#include "parquet/encryption/file_key_unwrapper.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/encryption/test_in_memory_kms.h"
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+class KeyWrappingTest : public ::testing::Test {
+ public:
+  void SetUp() {
+    key_list_ = BuildKeyMap(kColumnMasterKeyIds, kColumnMasterKeys, kFooterMasterKeyId,
+                            kFooterMasterKey);
+    kms_connection_config_.refreshable_key_access_token =
+        std::make_shared<KeyAccessToken>();
+  }
+
+ protected:
+  void WrapThenUnwrap(std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                      bool double_wrapping, bool is_wrap_locally) {
+    uint64_t cache_entry_lifetime_seconds = 600;
+
+    KeyToolkit key_toolkit;
+    key_toolkit.RegisterKmsClientFactory(
+        std::make_shared<TestOnlyInMemoryKmsClientFactory>(is_wrap_locally, key_list_));
+
+    FileKeyWrapper wrapper(&key_toolkit, kms_connection_config_, key_material_store,
+                           cache_entry_lifetime_seconds, double_wrapping);
+
+    std::string key_metadata_json_footer =
+        wrapper.GetEncryptionKeyMetadata(kFooterEncryptionKey, kFooterMasterKeyId, true);
+    std::string key_metadata_json_column = wrapper.GetEncryptionKeyMetadata(
+        kColumnEncryptionKey1, kColumnMasterKeyIds[0], false);
+
+    FileKeyUnwrapper unwrapper(&key_toolkit, kms_connection_config_,
+                               cache_entry_lifetime_seconds);
+    std::string footer_key = unwrapper.GetKey(key_metadata_json_footer);
+    ASSERT_EQ(footer_key, kFooterEncryptionKey);
+
+    std::string column_key = unwrapper.GetKey(key_metadata_json_column);
+    ASSERT_EQ(column_key, kColumnEncryptionKey1);
+  }
+
+  // TODO: this method will be removed when material external storage is supported
+  void WrapThenUnwrapWithUnsupportedExternalStorage(bool double_wrapping,
+                                                    bool is_wrap_locally) {
+    uint64_t cache_entry_lifetime_seconds = 600;
+
+    KeyToolkit key_toolkit;
+    key_toolkit.RegisterKmsClientFactory(
+        std::make_shared<TestOnlyInMemoryKmsClientFactory>(is_wrap_locally, key_list_));
+
+    std::shared_ptr<FileKeyMaterialStore> unsupported_material_store =
+        std::make_shared<FileKeyMaterialStore>();
+
+    FileKeyWrapper wrapper(&key_toolkit, kms_connection_config_,
+                           unsupported_material_store, cache_entry_lifetime_seconds,
+                           double_wrapping);
+
+    EXPECT_THROW(
+        wrapper.GetEncryptionKeyMetadata(kFooterEncryptionKey, kFooterMasterKeyId, true),
+        ParquetException);
+  }
+
+  std::unordered_map<std::string, std::string> key_list_;
+  KmsConnectionConfig kms_connection_config_;
+};
+
+TEST_F(KeyWrappingTest, InternalMaterialStorage) {
+  /// key_material_store = NULL indicates that "key material" is stored inside parquet

Review comment:
       Please double-slashes for comments, and triple-slashes only for docstrings.

##########
File path: cpp/src/parquet/encryption/kms_client.h
##########
@@ -0,0 +1,84 @@
+// 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 <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class wraps the key access token of a KMS server. If your token changes over time,
+// you should keep the reference to the KeyAccessToken object and call Refresh() method
+// every time you have a new token.
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;

Review comment:
       I don't understand why this is a `shared_ptr`. Why not simply embed a `std::string` here? What does `KeyAccessToken` bring?

##########
File path: cpp/src/parquet/encryption/key_metadata.h
##########
@@ -0,0 +1,92 @@
+// 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 <string>
+
+#include "arrow/util/variant.h"
+
+#include "parquet/encryption/key_material.h"
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer. The "key metadata" field is made available to file readers
+// to enable recovery of the key. This interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows integration with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details). This class is implemented to support version 1 of the parquet key
+// management tools specification.
+//
+// KeyMetadata writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material.
+// 2. "internalStorage" - a boolean. If true, means that "key material" is kept inside the
+// "key metadata" field. If false, "key material" is kept externally (outside Parquet
+// files) - in this case, "key metadata" keeps a reference to the external "key material".
+// 3. "keyReference" - a String, with the reference to the external "key material".
+// Written only if internalStorage is false.
+//
+// If internalStorage is true, "key material" is a part of "key metadata", and the json
+// keeps additional fields, described in the KeyMaterial class.
+class PARQUET_EXPORT KeyMetadata {
+ public:
+  static constexpr const char kKeyMaterialInternalStorageField[] = "internalStorage";
+  static constexpr const char kKeyReferenceField[] = "keyReference";
+
+  static KeyMetadata Parse(const std::string& key_metadata_bytes);

Review comment:
       Does this take a serialized JSON object? Comment/docstring perhaps?

##########
File path: cpp/src/parquet/encryption/key_metadata.h
##########
@@ -0,0 +1,92 @@
+// 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 <string>
+
+#include "arrow/util/variant.h"
+
+#include "parquet/encryption/key_material.h"
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer. The "key metadata" field is made available to file readers
+// to enable recovery of the key. This interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows integration with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details). This class is implemented to support version 1 of the parquet key
+// management tools specification.
+//
+// KeyMetadata writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material.
+// 2. "internalStorage" - a boolean. If true, means that "key material" is kept inside the
+// "key metadata" field. If false, "key material" is kept externally (outside Parquet
+// files) - in this case, "key metadata" keeps a reference to the external "key material".
+// 3. "keyReference" - a String, with the reference to the external "key material".
+// Written only if internalStorage is false.
+//
+// If internalStorage is true, "key material" is a part of "key metadata", and the json
+// keeps additional fields, described in the KeyMaterial class.
+class PARQUET_EXPORT KeyMetadata {
+ public:
+  static constexpr const char kKeyMaterialInternalStorageField[] = "internalStorage";
+  static constexpr const char kKeyReferenceField[] = "keyReference";

Review comment:
       Is it useful exposing this two constants as `public`?

##########
File path: cpp/src/parquet/encryption/key_encryption_key.h
##########
@@ -0,0 +1,62 @@
+// 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 <cstdint>
+#include <vector>
+
+#include "arrow/util/base64.h"
+
+namespace parquet {
+namespace encryption {
+
+// In the double wrapping mode, each "data encryption key" (DEK) is encrypted with a “key
+// encryption key” (KEK), that in turn is encrypted with a "master encryption key" (MEK).
+// In a writer process, a random KEK is generated for each MEK ID, and cached in a <MEK-ID
+// : KEK> map. This allows to perform an interaction with a KMS server only once for each
+// MEK, in order to wrap its KEK. "Data encryption key" (DEK) wrapping is performed

Review comment:
       "Wrap" means "encrypt"?

##########
File path: cpp/src/parquet/encryption/file_key_wrapper.h
##########
@@ -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.
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/file_key_material_store.h"
+#include "parquet/encryption/key_encryption_key.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class will generate "key metadata" from "data encryption key" and "master key",
+// following these steps:
+// 1. Wrap "data encryption key". There are 2 modes:
+// 1.1. single wrapping: encrypt "data encryption key" directly with "master encryption
+// key" 1.2. double wrapping: 2 steps: 1.2.1. "key encryption key" is randomized (see
+// structure of KeyEncryptionKey class) 1.2.2. "data encryption key" is encrypted with the
+// above "key encryption key"
+// 2. Create "key material" (see structure in KeyMaterial class)
+// 3. Create "key metadata" with "key material" inside or a reference to outside "key
+// material" (see structure in KeyMetadata class).
+// Currently we don't support the case "key material" stores outside "key metadata"
+// yet.
+class PARQUET_EXPORT FileKeyWrapper {
+ public:
+  static constexpr int kKeyEncryptionKeyLength = 16;
+  static constexpr int kKeyEncryptionKeyIdLength = 16;
+
+  /// key_toolkit and kms_connection_config is to get KmsClient from the cache or create
+  /// KmsClient if it's not in the cache yet. cache_entry_lifetime_seconds is life time of
+  /// KmsClient in the cache. key_material_store is to store "key material" outside
+  /// parquet file, NULL if "key material" is stored inside parquet file.
+  FileKeyWrapper(KeyToolkit* key_toolkit,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime_seconds, bool double_wrapping);
+
+  /// Creates key_metadata field for a given data key, via wrapping the key with the
+  /// master key
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+ private:
+  KeyEncryptionKey CreateKeyEncryptionKey(const std::string& master_key_id);
+
+  /// A map of Master Encryption Key ID -> KeyEncryptionKey, for the current token
+  std::shared_ptr<arrow::util::ConcurrentMap<KeyEncryptionKey>> kek_per_master_key_id_;

Review comment:
       Why `shared_ptr`?

##########
File path: cpp/src/parquet/encryption/key_material.h
##########
@@ -0,0 +1,125 @@
+// 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 <map>
+#include <string>
+
+#include "arrow/json/object_parser.h"

Review comment:
       We should avoid including this in a `.h` file.

##########
File path: cpp/src/parquet/encryption/key_material.h
##########
@@ -0,0 +1,125 @@
+// 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 <map>
+#include <string>
+
+#include "arrow/json/object_parser.h"
+
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KeyMaterial class represents the "key material", keeping the information that allows
+// readers to recover an encryption key (see description of the KeyMetadata class). The
+// keytools package (PARQUET-1373) implements the "envelope encryption" pattern, in a
+// "single wrapping" or "double wrapping" mode. In the single wrapping mode, the key
+// material is generated by encrypting the "data encryption key" (DEK) by a "master key".
+// In the double wrapping mode, the key material is generated by encrypting the DEK by a
+// "key encryption key" (KEK), that in turn is encrypted by a "master key".
+//
+// Key material is kept in a flat json object, with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current
+// version, only one value is allowed - "PKMT1" (stands
+//     for "parquet key management tools, version 1"). For external key material storage,
+//     this field is written in both "key metadata" and "key material" jsons. For internal
+//     key material storage, this field is written only once in the common json.
+// 2. "isFooterKey" - a boolean. If true, means that the material belongs to a file footer
+// key, and keeps additional information (such as
+//     KMS instance ID and URL). If false, means that the material belongs to a column
+//     key.
+// 3. "kmsInstanceID" - a String, with the KMS Instance ID. Written only in footer key
+// material.
+// 4. "kmsInstanceURL" - a String, with the KMS Instance URL. Written only in footer key
+// material.
+// 5. "masterKeyID" - a String, with the ID of the master key used to generate the
+// material.
+// 6. "wrappedDEK" - a String, with the wrapped DEK (base64 encoding).
+// 7. "doubleWrapping" - a boolean. If true, means that the material was generated in
+// double wrapping mode.
+//     If false - in single wrapping mode.
+// 8. "keyEncryptionKeyID" - a String, with the ID of the KEK used to generate the
+// material. Written only in double wrapping mode.
+// 9. "wrappedKEK" - a String, with the wrapped KEK (base64 encoding). Written only in
+// double wrapping mode.
+class PARQUET_EXPORT KeyMaterial {
+ public:
+  // these fields are defined in a specification and should never be changed
+  static constexpr const char kKeyMaterialTypeField[] = "keyMaterialType";
+  static constexpr const char kKeyMaterialType1[] = "PKMT1";
+
+  static constexpr const char kFooterKeyIdInFile[] = "footerKey";
+  static constexpr const char kColumnKeyIdInFilePrefix[] = "columnKey";
+
+  static constexpr const char kIsFooterKeyField[] = "isFooterKey";
+  static constexpr const char kDoubleWrappingField[] = "doubleWrapping";
+  static constexpr const char kKmsInstanceIdField[] = "kmsInstanceID";
+  static constexpr const char kKmsInstanceUrlField[] = "kmsInstanceURL";
+  static constexpr const char kMasterKeyIdField[] = "masterKeyID";
+  static constexpr const char kWrappedDataEncryptionKeyField[] = "wrappedDEK";
+  static constexpr const char kKeyEncryptionKeyIdField[] = "keyEncryptionKeyID";
+  static constexpr const char kWrappedKeyEncryptionKeyField[] = "wrappedKEK";
+
+ public:
+  KeyMaterial() = default;
+
+  static KeyMaterial Parse(const std::string& key_material_string);
+
+  static KeyMaterial Parse(const arrow::json::ObjectParser& key_material_json);
+
+  /// This method returns a json string that will be stored either inside a parquet file
+  /// or in a key material store outside the parquet file.

Review comment:
       "stored"?

##########
File path: cpp/src/parquet/encryption/key_material.h
##########
@@ -0,0 +1,125 @@
+// 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 <map>

Review comment:
       Doesn't seem required.




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



[GitHub] [arrow] pitrou edited a comment on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou edited a comment on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-796770371


   As for testing, I'd still welcome unit tests with actual pieces of JSON as per the spec, but I won't insist on it here :-)
   
   Last question: do we want to mark those APIs experimental so that we feel free to change them in the future?


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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r483013904



##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,

Review comment:
       Since we already have a struct KeyMetadata, so creating one more struct to contain this string doesn't make sense. We can change it to:
   ```
   KeyMetadata FileKeyWrapper::GetEncryptionKeyMetadata(...)
   ```
   and create a method:
   ```
   std::string KeyMetadata::SerializeToJson()
   ```
   But it seems not a reversed step of `std::string FileKeyWrapper::GetKey(const std::string& key_metadata) const`.
   So I still prefer current approach. cc @ggershinsky.

##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,

Review comment:
       @emkornfield Since we already have a struct KeyMetadata, so creating one more struct to contain this string doesn't make sense. We can change it to:
   ```
   KeyMetadata FileKeyWrapper::GetEncryptionKeyMetadata(...)
   ```
   and create a method:
   ```
   std::string KeyMetadata::SerializeToJson()
   ```
   But it seems not a reversed step of `std::string FileKeyWrapper::GetKey(const std::string& key_metadata) const`.
   So I still prefer current approach. cc @ggershinsky.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486370626



##########
File path: cpp/src/parquet/kms_client_factory.h
##########
@@ -0,0 +1,42 @@
+// 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 "parquet/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KmsClientFactory {
+ public:
+  explicit KmsClientFactory(bool wrap_locally) : wrap_locally_(wrap_locally) {}
+
+  KmsClientFactory() : KmsClientFactory(false) {}
+
+  virtual ~KmsClientFactory() {}
+
+  virtual std::shared_ptr<KmsClient> CreateKmsClient(

Review comment:
       > The user can't create a KmsClientFactory directly?
   
   @pitrou No, he can't. `KmsClientFactory` doesn't know the actual type of `KmsClient` yet. When the user create their own kms client class (inherited from `KmsClient`), he will have to create their own `KmsClientFactory` as well.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {

Review comment:
       sorry about that. This is a core class, that translates the parameters of high level encryption (like the names of encrypted columns, names of master keys, etc), into parameters of low level encryption (like the key metadata, DEK, etc). A factory that produces the low level FileEncryptionProperties and FileDecryptionProperties objects, from the high level parameters. I agree though the name is long and confusing. Moreover, we don't have another crypto factory here, so something simple like `CryptoFactory` would do as the class name from my POV.




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



[GitHub] [arrow] ggershinsky commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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


   @thamht4190 my comments were related to the function documentation (comments); all addressed now, so I'm good with the current version.


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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/key_metadata.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 <string>
+
+#include "parquet/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer . The "key metadata" field is made available to file readers
+// to enable recovery of the key. This simple interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows to work with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details).
+//
+// KeyMetadata class writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current

Review comment:
       this indeed can be ambiguous. I'd suggest removing the second part of the comment entirely, starting from "In the current version..". The main purpose of the comment is served by its first part.

##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       This is kind of borderline, not really internal indeed because it is used in the InMemoryKMS mock for tests etc; but on the other hand most/all users won't have a reason to call them. We might want not to expose these functions in the public 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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r485556981



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {

Review comment:
       Can you put internal-only stuff in separate header files? e.g. `key_toolkit_internal.h`?




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497970111



##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)
+      : cached_item_(cached_item) {
+    expiration_timestamp_ =
+        CurrentTimePoint() + std::chrono::milliseconds(expiration_interval_millis);
+  }
+
+  bool IsExpired() {
+    auto now = CurrentTimePoint();
+    return (now > expiration_timestamp_);
+  }
+
+  E cached_item() { return cached_item_; }
+
+ private:
+  TimePoint expiration_timestamp_;
+  E cached_item_;
+};
+
+// This class is to avoid the below warning when compiling KeyToolkit class with VS2015
+// warning C4503: decorated name length exceeded, name was truncated
+template <typename V>
+class ExpiringCacheMapEntry {
+ public:
+  ExpiringCacheMapEntry() = default;
+
+  explicit ExpiringCacheMapEntry(std::shared_ptr<ConcurrentMap<V>> cached_item,
+                                 uint64_t expiration_interval_millis)
+      : map_cache_(cached_item, expiration_interval_millis) {}
+
+  bool IsExpired() { return map_cache_.IsExpired(); }
+
+  std::shared_ptr<ConcurrentMap<V>> cached_item() { return map_cache_.cached_item(); }
+
+ private:
+  ExpiringCacheEntry<std::shared_ptr<ConcurrentMap<V>>> map_cache_;
+};
+
+}  // namespace internal
+
+// Two-level cache with expiration of internal caches according to token lifetime.
+// External cache is per token, internal is per string key.
+// Wrapper class around:
+//    std::unordered_map<std::string,
+//    internal::ExpiringCacheEntry<std::unordered_map<std::string, V>>>
+// This cache is safe to be shared between threads.
+template <typename V>
+class TwoLevelCacheWithExpiration {
+ public:
+  TwoLevelCacheWithExpiration() {
+    last_cache_cleanup_timestamp_ = internal::CurrentTimePoint();
+  }
+
+  std::shared_ptr<ConcurrentMap<V>> GetOrCreateInternalCache(
+      const std::string& access_token, uint64_t cache_entry_lifetime_ms) {
+    auto lock = mutex_.Lock();
+
+    auto external_cache_entry = cache_.find(access_token);
+    if (external_cache_entry == cache_.end() ||
+        external_cache_entry->second.IsExpired()) {
+      cache_.insert(
+          {access_token, internal::ExpiringCacheMapEntry<V>(
+                             std::shared_ptr<ConcurrentMap<V>>(new ConcurrentMap<V>()),
+                             cache_entry_lifetime_ms)});
+    }
+
+    return cache_[access_token].cached_item();
+  }
+
+  void RemoveCacheEntriesForToken(const std::string& access_token) {
+    auto lock = mutex_.Lock();
+    cache_.erase(access_token);
+  }
+
+  void RemoveCacheEntriesForAllTokens() {
+    auto lock = mutex_.Lock();
+    cache_.clear();
+  }
+
+  void CheckCacheForExpiredTokens(uint64_t cache_cleanup_period_ms) {

Review comment:
       Why?




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r479634599



##########
File path: cpp/src/parquet/key_material.cc
##########
@@ -0,0 +1,161 @@
+// 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.
+
+#include <rapidjson/document.h>
+#include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
+
+#include "parquet/exception.h"
+#include "parquet/key_material.h"
+#include "parquet/key_metadata.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE_FIELD[];
+constexpr char KeyMaterial::KEY_MATERIAL_TYPE1[];
+
+constexpr char KeyMaterial::FOOTER_KEY_ID_IN_FILE[];
+constexpr char KeyMaterial::COLUMN_KEY_ID_IN_FILE_PREFIX[];
+
+constexpr char KeyMaterial::IS_FOOTER_KEY_FIELD[];
+constexpr char KeyMaterial::DOUBLE_WRAPPING_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_ID_FIELD[];
+constexpr char KeyMaterial::KMS_INSTANCE_URL_FIELD[];
+constexpr char KeyMaterial::MASTER_KEY_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_DEK_FIELD[];
+constexpr char KeyMaterial::KEK_ID_FIELD[];
+constexpr char KeyMaterial::WRAPPED_KEK_FIELD[];
+
+KeyMaterial::KeyMaterial(bool is_footer_key, const std::string& kms_instance_id,
+                         const std::string& kms_instance_url,
+                         const std::string& master_key_id, bool is_double_wrapped,
+                         const std::string& kek_id,
+                         const std::string& encoded_wrapped_kek,
+                         const std::string& encoded_wrapped_dek)
+    : is_footer_key_(is_footer_key),
+      kms_instance_id_(kms_instance_id),
+      kms_instance_url_(kms_instance_url),
+      master_key_id_(master_key_id),
+      is_double_wrapped_(is_double_wrapped),
+      kek_id_(kek_id),
+      encoded_wrapped_kek_(encoded_wrapped_kek),
+      encoded_wrapped_dek_(encoded_wrapped_dek) {}
+
+KeyMaterial KeyMaterial::Parse(const std::string& key_material_string) {
+  rapidjson::Document document;
+  document.Parse(key_material_string.c_str());
+
+  if (document.HasParseError() || !document.IsObject()) {
+    throw ParquetException("Failed to parse key metadata " + key_material_string);
+  }
+
+  // External key material - extract "key material type", and make sure it is supported
+  std::string key_material_type = document[KEY_MATERIAL_TYPE_FIELD].GetString();
+  if (KEY_MATERIAL_TYPE1 != key_material_type) {
+    throw ParquetException("Wrong key material type: " + key_material_type + " vs " +
+                           KEY_MATERIAL_TYPE1);
+  }
+  // Parse other fields (common to internal and external key material)
+  return Parse(document);
+}
+
+KeyMaterial KeyMaterial::Parse(const rapidjson::Document& key_material_json) {
+  // 2. Check if "key material" belongs to file footer key
+  bool is_footer_key = key_material_json[IS_FOOTER_KEY_FIELD].GetBool();
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  if (is_footer_key) {
+    // 3.  For footer key, extract KMS Instance ID
+    kms_instance_id = key_material_json[KMS_INSTANCE_ID_FIELD].GetString();
+    // 4.  For footer key, extract KMS Instance URL
+    kms_instance_url = key_material_json[KMS_INSTANCE_URL_FIELD].GetString();
+  }
+  // 5. Extract master key ID
+  std::string master_key_id = key_material_json[MASTER_KEY_ID_FIELD].GetString();
+  // 6. Extract wrapped DEK
+  std::string encoded_wrapped_dek = key_material_json[WRAPPED_DEK_FIELD].GetString();
+  std::string kek_id;
+  std::string encoded_wrapped_kek;
+  // 7. Check if "key material" was generated in double wrapping mode
+  bool is_double_wrapped = key_material_json[DOUBLE_WRAPPING_FIELD].GetBool();
+  if (is_double_wrapped) {
+    // 8. In double wrapping mode, extract KEK ID
+    kek_id = key_material_json[KEK_ID_FIELD].GetString();
+    // 9. In double wrapping mode, extract wrapped KEK
+    encoded_wrapped_kek = key_material_json[WRAPPED_KEK_FIELD].GetString();
+  }
+
+  return KeyMaterial(is_footer_key, kms_instance_id, kms_instance_url, master_key_id,
+                     is_double_wrapped, kek_id, encoded_wrapped_kek, encoded_wrapped_dek);
+}
+
+std::string KeyMaterial::CreateSerialized(
+    bool is_footer_key, const std::string& kms_instance_id,

Review comment:
       I'm not sure it's a good idea to create a new struct, since this struct will have the same structure with `KeyMaterial`. So we can public set methods. However, I'm not sure it will go right with the idea that all these information should be passed. Using a new struct / set methods, some information can be easily missed.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r479625018



##########
File path: cpp/src/parquet/file_key_unwrapper.h
##########
@@ -0,0 +1,65 @@
+// 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 "parquet/encryption.h"
+#include "parquet/key_material.h"
+#include "parquet/key_toolkit.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  FileKeyUnwrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,

Review comment:
       `kms_client_factory` is stored here in `FileKeyUnwrapper` class and `PropertiesDrivenCryptoFactory` class. We are not sure the lifetime of these 2 classes, then I use shared_ptr. One way to use unique_ptr is only storing it at one place as a static variable in `KeyToolkit` class. But because you questioned about `KeyTookit` singleton, I'll wait until we resolve that question.




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



[GitHub] [arrow] thamht4190 commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-702625493


   @bkietz I have the same thought with @ggershinsky. Moreover, I wrote the code based on Java version, so I may not have enough deep understanding to write out the mind & design behind. I will try to list out some points in sub PR's description. Is that ok?
   Anw, I will try to fix the comment already listed on this PR first.


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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/kms_client.h
##########
@@ -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.
+
+#pragma once
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KeyAccessToken {
+ public:
+  KeyAccessToken() = default;
+
+  explicit KeyAccessToken(const std::string value) : value_(value) {}
+
+  void Refresh(const std::string& new_value) { value_ = new_value; }
+
+  const std::string& value() const { return value_; }
+
+  void SetDefaultIfEmpty();
+
+ private:
+  std::string value_;
+};
+
+struct PARQUET_EXPORT KmsConnectionConfig {
+  std::string kms_instance_id;
+  std::string kms_instance_url;
+  std::shared_ptr<KeyAccessToken> refreshable_key_access_token;
+  std::unordered_map<std::string, std::string> custom_kms_conf;
+
+  const std::string& key_access_token() const {
+    if (refreshable_key_access_token == NULL ||
+        refreshable_key_access_token->value().empty()) {
+      throw ParquetException("key access token is not set!");
+    }
+    return refreshable_key_access_token->value();
+  }
+
+  void SetDefaultIfEmpty();
+};
+
+class PARQUET_EXPORT KmsClient {
+ public:
+  static constexpr char KMS_INSTANCE_ID_DEFAULT[] = "DEFAULT";
+  static constexpr char KMS_INSTANCE_URL_DEFAULT[] = "DEFAULT";
+  static constexpr char KEY_ACCESS_TOKEN_DEFAULT[] = "DEFAULT";
+
+  // Wraps a key - encrypts it with the master key, encodes the result
+  // and potentially adds a KMS-specific metadata.
+  virtual std::string WrapKey(const std::string& key_bytes,
+                              const std::string& master_key_identifier) = 0;

Review comment:
       e.g., when data is written by parquet-cpp, and read by parquet-mr, both nodes talk to the same KMS that stores the master keys. Therefore, two versions of a custom KmsClient need to be developed, in C++ and Java. This is easier to design and implement if the KmsClient API is identical/similar in both language bindings.  So far, we've managed to keep all Parquet encryption API to be as similar as possible in Java and C++.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486434226



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {

Review comment:
       > Can you put internal-only stuff in separate header files?
   
   `KeyWithMasterId` is an internal thing, however, since it appears in public header file `FileKeyUnwrapper.h"` (i.e. `KeyWithMasterId FileKeyUnwrapper::GetDataEncryptionKey(const KeyMaterial& key_material) const` - which will be used in KeyToolkit key rotation later). So it must be in a public `key_toolkit.h`.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486371291



##########
File path: cpp/src/parquet/kms_client_factory.h
##########
@@ -0,0 +1,42 @@
+// 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 "parquet/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KmsClientFactory {
+ public:
+  explicit KmsClientFactory(bool wrap_locally) : wrap_locally_(wrap_locally) {}
+
+  KmsClientFactory() : KmsClientFactory(false) {}
+
+  virtual ~KmsClientFactory() {}
+
+  virtual std::shared_ptr<KmsClient> CreateKmsClient(

Review comment:
       > If this is only for internal use
   
   @pitrou it's for public use.




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498339318



##########
File path: cpp/src/parquet/encryption/remote_kms_client.cc
##########
@@ -0,0 +1,127 @@
+// 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.
+
+#include "arrow/json/object_parser.h"
+#include "arrow/json/object_writer.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/remote_kms_client.h"
+#include "parquet/exception.h"
+
+using arrow::json::ObjectParser;
+using arrow::json::ObjectWriter;
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char RemoteKmsClient::kLocalWrapNoKeyVersion[];
+
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapKeyVersionField[];
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapEncryptedKeyField[];
+
+RemoteKmsClient::LocalKeyWrap::LocalKeyWrap(const std::string& master_key_version,
+                                            const std::string& encrypted_encoded_key)
+    : encrypted_encoded_key_(encrypted_encoded_key),
+      master_key_version_(master_key_version) {}
+
+std::string RemoteKmsClient::LocalKeyWrap::CreateSerialized(
+    const std::string& encrypted_encoded_key) {
+  ObjectWriter json_writer;
+
+  json_writer.SetString(kLocalWrapKeyVersionField, kLocalWrapNoKeyVersion);
+  json_writer.SetString(kLocalWrapEncryptedKeyField, encrypted_encoded_key);
+
+  return json_writer.Serialize();
+}
+
+RemoteKmsClient::LocalKeyWrap RemoteKmsClient::LocalKeyWrap::Parse(
+    const std::string& wrapped_key) {
+  ObjectParser json_parser;
+  if (!json_parser.Parse(wrapped_key)) {
+    throw ParquetException("Failed to parse local key wrap json " + wrapped_key);
+  }
+  std::string master_key_version;
+  PARQUET_ASSIGN_OR_THROW(master_key_version,
+                          json_parser.GetString(kLocalWrapKeyVersionField));
+
+  std::string encrypted_encoded_key;
+  PARQUET_ASSIGN_OR_THROW(encrypted_encoded_key,
+                          json_parser.GetString(kLocalWrapEncryptedKeyField));
+
+  return RemoteKmsClient::LocalKeyWrap(master_key_version, encrypted_encoded_key);
+}
+
+void RemoteKmsClient::Initialize(const KmsConnectionConfig& kms_connection_config,
+                                 bool is_wrap_locally) {
+  kms_connection_config_ = kms_connection_config;
+  is_wrap_locally_ = is_wrap_locally;
+  if (is_wrap_locally_) {
+    master_key_cache_.Clear();
+  }
+
+  is_default_token_ =
+      kms_connection_config_.key_access_token() == KmsClient::kKeyAccessTokenDefault;
+
+  InitializeInternal();
+}
+
+std::string RemoteKmsClient::WrapKey(const std::string& key_bytes,
+                                     const std::string& master_key_identifier) {
+  if (is_wrap_locally_) {

Review comment:
       It doesn't really make sense to use a `variant` for such polymorphism, IMHO.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;

Review comment:
       The cache entries are stored/retrieved by the access tokens, which enables multi-tenancy. Each caller is able to access only the objects cached with its token, using the token as the cache key. [btw, the validity of the token is verified vs the KMS, but this doesn't relate directly to the current discussion].  Also, the technical challenges of using a singleton can be surmounted with a reasonable effort. So I think that the current design is sound.
   
   However, I can understand the desire to use an alternative to singletons where possible; and it indeed should be possible in the case of this particular encryption interface.
   I agree it's possible to provide users with an explicit API to create cache instances, and pass them to parquet. But such API would be difficult to explain, and would likely lead to a situation where many users will run without caches, meaning without KMS RPC optimization (each thread will interact with a KMS server, for every key).
   So let me propose the following. This encryption interface has a class called PropertiesDrivenCryptoFactory. It's an anchor; one factory instance would typically be created by a user / tenant, and used to create crypto objects for different files / threads. Therefore, we can make the cache a regular, non-singleton, member in this class. It will be shared then across the threads, and will be limited to this tenant. Naturally, this will need to be well documented for the  PropertiesDrivenCryptoFactory class (already a part of the public API of encryption).
   What do you think?
   
   




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r485558482



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>> cache_;
+  };
+
+  class KEKWriteCache {
+   public:
+    static KEKWriteCache& GetInstance() {
+      static KEKWriteCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<KeyEncryptionKey>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<KeyEncryptionKey> cache_;
+  };
+
+  class KEKReadCache {
+   public:
+    static KEKReadCache& GetInstance() {
+      static KEKReadCache instance;
+      return instance;
+    }
+    TwoLevelCacheWithExpiration<std::string>& cache() { return cache_; }
+
+   private:
+    TwoLevelCacheWithExpiration<std::string> cache_;
+  };
+
+  // KMS client two level cache: token -> KMSInstanceId -> KmsClient
+  static TwoLevelCacheWithExpiration<std::shared_ptr<KmsClient>>&
+  kms_client_cache_per_token() {
+    return KmsClientCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for wrapping: token -> MEK_ID -> KeyEncryptionKey
+  static TwoLevelCacheWithExpiration<KeyEncryptionKey>& kek_write_cache_per_token() {
+    return KEKWriteCache::GetInstance().cache();
+  }
+
+  // KEK two level cache for unwrapping: token -> KEK_ID -> KEK bytes
+  static TwoLevelCacheWithExpiration<std::string>& kek_read_cache_per_token() {
+    return KEKReadCache::GetInstance().cache();
+  }
+
+  static std::shared_ptr<KmsClient> GetKmsClient(
+      std::shared_ptr<KmsClientFactory> kms_client_factory,
+      const KmsConnectionConfig& kms_connection_config, bool is_wrap_locally,
+      uint64_t cache_entry_lifetime);
+
+  // Encrypts "key" with "master_key", using AES-GCM and the "aad"
+  static std::string EncryptKeyLocally(const std::string& key,

Review comment:
       Encrypting a key doesn't have anything to do with a cache, right? If you need some kind of "internal utilities" place, you can create a separate header file for it (e.g. `encryption_util_internal.h`).




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498080392



##########
File path: cpp/src/parquet/encryption/key_material.h
##########
@@ -0,0 +1,125 @@
+// 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 <map>
+#include <string>
+
+#include "arrow/json/object_parser.h"
+
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// KeyMaterial class represents the "key material", keeping the information that allows
+// readers to recover an encryption key (see description of the KeyMetadata class). The
+// keytools package (PARQUET-1373) implements the "envelope encryption" pattern, in a
+// "single wrapping" or "double wrapping" mode. In the single wrapping mode, the key
+// material is generated by encrypting the "data encryption key" (DEK) by a "master key".
+// In the double wrapping mode, the key material is generated by encrypting the DEK by a
+// "key encryption key" (KEK), that in turn is encrypted by a "master key".
+//
+// Key material is kept in a flat json object, with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material. In the current
+// version, only one value is allowed - "PKMT1" (stands
+//     for "parquet key management tools, version 1"). For external key material storage,
+//     this field is written in both "key metadata" and "key material" jsons. For internal
+//     key material storage, this field is written only once in the common json.
+// 2. "isFooterKey" - a boolean. If true, means that the material belongs to a file footer
+// key, and keeps additional information (such as
+//     KMS instance ID and URL). If false, means that the material belongs to a column
+//     key.
+// 3. "kmsInstanceID" - a String, with the KMS Instance ID. Written only in footer key
+// material.
+// 4. "kmsInstanceURL" - a String, with the KMS Instance URL. Written only in footer key
+// material.
+// 5. "masterKeyID" - a String, with the ID of the master key used to generate the
+// material.
+// 6. "wrappedDEK" - a String, with the wrapped DEK (base64 encoding).
+// 7. "doubleWrapping" - a boolean. If true, means that the material was generated in
+// double wrapping mode.
+//     If false - in single wrapping mode.
+// 8. "keyEncryptionKeyID" - a String, with the ID of the KEK used to generate the
+// material. Written only in double wrapping mode.
+// 9. "wrappedKEK" - a String, with the wrapped KEK (base64 encoding). Written only in
+// double wrapping mode.
+class PARQUET_EXPORT KeyMaterial {
+ public:
+  // these fields are defined in a specification and should never be changed
+  static constexpr const char kKeyMaterialTypeField[] = "keyMaterialType";
+  static constexpr const char kKeyMaterialType1[] = "PKMT1";
+
+  static constexpr const char kFooterKeyIdInFile[] = "footerKey";
+  static constexpr const char kColumnKeyIdInFilePrefix[] = "columnKey";
+
+  static constexpr const char kIsFooterKeyField[] = "isFooterKey";
+  static constexpr const char kDoubleWrappingField[] = "doubleWrapping";
+  static constexpr const char kKmsInstanceIdField[] = "kmsInstanceID";
+  static constexpr const char kKmsInstanceUrlField[] = "kmsInstanceURL";
+  static constexpr const char kMasterKeyIdField[] = "masterKeyID";
+  static constexpr const char kWrappedDataEncryptionKeyField[] = "wrappedDEK";
+  static constexpr const char kKeyEncryptionKeyIdField[] = "keyEncryptionKeyID";
+  static constexpr const char kWrappedKeyEncryptionKeyField[] = "wrappedKEK";
+
+ public:
+  KeyMaterial() = default;
+
+  static KeyMaterial Parse(const std::string& key_material_string);
+
+  static KeyMaterial Parse(const arrow::json::ObjectParser& key_material_json);
+
+  /// This method returns a json string that will be stored either inside a parquet file
+  /// or in a key material store outside the parquet file.

Review comment:
       Ah, no, it was just me, sorry.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r579596080



##########
File path: cpp/src/parquet/encryption/test_encryption_util.cc
##########
@@ -0,0 +1,481 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#include <arrow/io/file.h>
+
+#include "parquet/encryption/test_encryption_util.h"
+#include "parquet/file_reader.h"
+#include "parquet/file_writer.h"
+#include "parquet/properties.h"
+
+using FileClass = ::arrow::io::FileOutputStream;
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+FileEncryptor::FileEncryptor() { schema_ = SetupEncryptionSchema(); }
+
+std::shared_ptr<GroupNode> FileEncryptor::SetupEncryptionSchema() {
+  parquet::schema::NodeVector fields;
+  // Create a primitive node named 'boolean_field' with type:BOOLEAN,
+  // repetition:REQUIRED
+  fields.push_back(PrimitiveNode::Make(kBooleanFieldName, Repetition::REQUIRED,
+                                       Type::BOOLEAN, ConvertedType::NONE));
+
+  // Create a primitive node named 'int32_field' with type:INT32, repetition:REQUIRED,
+  // logical type:TIME_MILLIS
+  fields.push_back(PrimitiveNode::Make(kInt32FieldName, Repetition::REQUIRED, Type::INT32,
+                                       ConvertedType::TIME_MILLIS));
+
+  // Create a primitive node named 'int64_field' with type:INT64, repetition:REPEATED
+  fields.push_back(PrimitiveNode::Make(kInt64FieldName, Repetition::REPEATED, Type::INT64,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kInt96FieldName, Repetition::REQUIRED, Type::INT96,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kFloatFieldName, Repetition::REQUIRED, Type::FLOAT,
+                                       ConvertedType::NONE));
+
+  fields.push_back(PrimitiveNode::Make(kDoubleFieldName, Repetition::REQUIRED,
+                                       Type::DOUBLE, ConvertedType::NONE));
+
+  // Create a primitive node named 'ba_field' with type:BYTE_ARRAY, repetition:OPTIONAL
+  fields.push_back(PrimitiveNode::Make(kByteArrayFieldName, Repetition::OPTIONAL,
+                                       Type::BYTE_ARRAY, ConvertedType::NONE));
+
+  // Create a primitive node named 'flba_field' with type:FIXED_LEN_BYTE_ARRAY,
+  // repetition:REQUIRED, field_length = kFixedLength
+  fields.push_back(PrimitiveNode::Make(kFixedLenByteArrayFieldName, Repetition::REQUIRED,
+                                       Type::FIXED_LEN_BYTE_ARRAY, ConvertedType::NONE,
+                                       kFixedLength));
+
+  // Create a GroupNode named 'schema' using the primitive nodes defined above
+  // This GroupNode is the root node of the schema tree
+  return std::static_pointer_cast<GroupNode>(
+      GroupNode::Make("schema", Repetition::REQUIRED, fields));
+}
+
+void FileEncryptor::EncryptFile(
+    std::string file,
+    std::shared_ptr<parquet::FileEncryptionProperties> encryption_configurations) {
+  WriterProperties::Builder prop_builder;
+  prop_builder.compression(parquet::Compression::SNAPPY);
+  prop_builder.encryption(encryption_configurations);
+  std::shared_ptr<WriterProperties> writer_properties = prop_builder.build();
+
+  PARQUET_ASSIGN_OR_THROW(auto out_file, FileClass::Open(file));
+  // Create a ParquetFileWriter instance
+  std::shared_ptr<parquet::ParquetFileWriter> file_writer =
+      parquet::ParquetFileWriter::Open(out_file, schema_, writer_properties);
+
+  for (int r = 0; r < num_rgs; r++) {
+    bool buffered_mode = r % 2 == 0;
+    auto row_group_writer = buffered_mode ? file_writer->AppendBufferedRowGroup()
+                                          : file_writer->AppendRowGroup();
+
+    int column_index = 0;
+    // Captures i by reference; increments it by one
+    auto get_next_column = [&]() {
+      return buffered_mode ? row_group_writer->column(column_index++)
+                           : row_group_writer->NextColumn();
+    };
+
+    // Write the Bool column
+    parquet::BoolWriter* bool_writer =
+        static_cast<parquet::BoolWriter*>(get_next_column());
+    for (int i = 0; i < rows_per_rowgroup_; i++) {
+      bool value = ((i % 2) == 0) ? true : false;
+      bool_writer->WriteBatch(1, nullptr, nullptr, &value);

Review comment:
       I did it for other types, except `bool` and byte array.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r479625144



##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;

Review comment:
       This constant is unused -> removed 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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486370626



##########
File path: cpp/src/parquet/kms_client_factory.h
##########
@@ -0,0 +1,42 @@
+// 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 "parquet/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+class PARQUET_EXPORT KmsClientFactory {
+ public:
+  explicit KmsClientFactory(bool wrap_locally) : wrap_locally_(wrap_locally) {}
+
+  KmsClientFactory() : KmsClientFactory(false) {}
+
+  virtual ~KmsClientFactory() {}
+
+  virtual std::shared_ptr<KmsClient> CreateKmsClient(

Review comment:
       > The user can't create a KmsClientFactory directly?
   No, he can't. `KmsClientFactory` doesn't know the actual type of `KmsClient` yet. When the user create their own kms client class (inherited from `KmsClient`), he will have to create their own `KmsClientFactory` as well.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r479624053



##########
File path: cpp/src/parquet/key_encryption_key.h
##########
@@ -0,0 +1,57 @@
+// 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 <cstdint>
+#include <vector>
+
+#include "arrow/util/base64.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyEncryptionKey {
+ public:
+  KeyEncryptionKey() = default;

Review comment:
       Removed.




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



[GitHub] [arrow] roee88 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
roee88 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r491000329



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.cc
##########
@@ -0,0 +1,274 @@
+// 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.
+
+#include <sstream>
+
+#include "arrow/result.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+#include "parquet/encryption/encryption_internal.h"
+#include "parquet/encryption/file_key_material_store.h"
+#include "parquet/encryption/file_key_unwrapper.h"
+#include "parquet/encryption/properties_driven_crypto_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr const int32_t PropertiesDrivenCryptoFactory::kAcceptableDataKeyLengths[];
+
+EncryptionConfiguration::Builder* EncryptionConfiguration::Builder::column_keys(
+    const std::string& column_keys) {
+  DCHECK(!column_keys.empty());

Review comment:
       Should this be replaced with an exception that explains that at least one column must be specified? To make it explicit that encryption of just the footer is not a valid option. 




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



[GitHub] [arrow] thamht4190 edited a comment on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 edited a comment on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-787455523


   I fixed all the comments in this pull, can you please take a look? @emkornfield @pitrou @bkietz cc @ggershinsky 


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



[GitHub] [arrow] pitrou commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-801144014


   I'll merge then. Thank you @thamht4190 and @ggershinsky for contributing this!


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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/encryption/remote_kms_client.cc
##########
@@ -0,0 +1,127 @@
+// 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.
+
+#include "arrow/json/object_parser.h"
+#include "arrow/json/object_writer.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/remote_kms_client.h"
+#include "parquet/exception.h"
+
+using arrow::json::ObjectParser;
+using arrow::json::ObjectWriter;
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char RemoteKmsClient::kLocalWrapNoKeyVersion[];
+
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapKeyVersionField[];
+constexpr const char RemoteKmsClient::LocalKeyWrap::kLocalWrapEncryptedKeyField[];
+
+RemoteKmsClient::LocalKeyWrap::LocalKeyWrap(const std::string& master_key_version,
+                                            const std::string& encrypted_encoded_key)
+    : encrypted_encoded_key_(encrypted_encoded_key),
+      master_key_version_(master_key_version) {}
+
+std::string RemoteKmsClient::LocalKeyWrap::CreateSerialized(
+    const std::string& encrypted_encoded_key) {
+  ObjectWriter json_writer;
+
+  json_writer.SetString(kLocalWrapKeyVersionField, kLocalWrapNoKeyVersion);
+  json_writer.SetString(kLocalWrapEncryptedKeyField, encrypted_encoded_key);
+
+  return json_writer.Serialize();
+}
+
+RemoteKmsClient::LocalKeyWrap RemoteKmsClient::LocalKeyWrap::Parse(
+    const std::string& wrapped_key) {
+  ObjectParser json_parser;
+  if (!json_parser.Parse(wrapped_key)) {
+    throw ParquetException("Failed to parse local key wrap json " + wrapped_key);
+  }
+  std::string master_key_version;
+  PARQUET_ASSIGN_OR_THROW(master_key_version,
+                          json_parser.GetString(kLocalWrapKeyVersionField));
+
+  std::string encrypted_encoded_key;
+  PARQUET_ASSIGN_OR_THROW(encrypted_encoded_key,
+                          json_parser.GetString(kLocalWrapEncryptedKeyField));
+
+  return RemoteKmsClient::LocalKeyWrap(master_key_version, encrypted_encoded_key);
+}
+
+void RemoteKmsClient::Initialize(const KmsConnectionConfig& kms_connection_config,
+                                 bool is_wrap_locally) {
+  kms_connection_config_ = kms_connection_config;
+  is_wrap_locally_ = is_wrap_locally;
+  if (is_wrap_locally_) {
+    master_key_cache_.Clear();
+  }
+
+  is_default_token_ =
+      kms_connection_config_.key_access_token() == KmsClient::kKeyAccessTokenDefault;
+
+  InitializeInternal();
+}
+
+std::string RemoteKmsClient::WrapKey(const std::string& key_bytes,
+                                     const std::string& master_key_identifier) {
+  if (is_wrap_locally_) {

Review comment:
       @thamht4190 This is a very good point! I see how it can be confusing. Lets then rename (and slightly modify)  `RemoteKmsClient` to something like  `LocalWrapKmsClient` that will be used only for KMS systems that don't support in-server wrapping. All other KMS clients will extend the `KmsClient` interface directly.
   I'll try to make this change also in the Java API (parquet-mr).




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497977436



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;

Review comment:
       I followed the same the way we write classes in https://github.com/apache/arrow/blob/master/cpp/src/parquet/encryption.h. I thought that were the way we should implement configuration class in arrow. No?




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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498073822



##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration.h
##########
@@ -0,0 +1,167 @@
+// 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 <unordered_map>
+
+#include "arrow/util/concurrent_map.h"
+#include "arrow/util/mutex.h"
+
+#include "parquet/encryption/key_toolkit_internal.h"
+
+using arrow::util::ConcurrentMap;
+
+namespace parquet {
+namespace encryption {
+
+namespace internal {
+
+// in miliseconds
+using TimePoint = std::chrono::system_clock::time_point;
+
+static inline TimePoint CurrentTimePoint() { return std::chrono::system_clock::now(); }
+
+template <typename E>
+class ExpiringCacheEntry {
+ public:
+  ExpiringCacheEntry() = default;
+
+  ExpiringCacheEntry(const E& cached_item, uint64_t expiration_interval_millis)
+      : cached_item_(cached_item) {
+    expiration_timestamp_ =
+        CurrentTimePoint() + std::chrono::milliseconds(expiration_interval_millis);
+  }
+
+  bool IsExpired() {
+    auto now = CurrentTimePoint();
+    return (now > expiration_timestamp_);
+  }
+
+  E cached_item() { return cached_item_; }
+
+ private:
+  TimePoint expiration_timestamp_;
+  E cached_item_;
+};
+
+// This class is to avoid the below warning when compiling KeyToolkit class with VS2015
+// warning C4503: decorated name length exceeded, name was truncated
+template <typename V>
+class ExpiringCacheMapEntry {
+ public:
+  ExpiringCacheMapEntry() = default;
+
+  explicit ExpiringCacheMapEntry(std::shared_ptr<ConcurrentMap<V>> cached_item,
+                                 uint64_t expiration_interval_millis)
+      : map_cache_(cached_item, expiration_interval_millis) {}
+
+  bool IsExpired() { return map_cache_.IsExpired(); }
+
+  std::shared_ptr<ConcurrentMap<V>> cached_item() { return map_cache_.cached_item(); }
+
+ private:
+  ExpiringCacheEntry<std::shared_ptr<ConcurrentMap<V>>> map_cache_;
+};
+
+}  // namespace internal
+
+// Two-level cache with expiration of internal caches according to token lifetime.
+// External cache is per token, internal is per string key.
+// Wrapper class around:
+//    std::unordered_map<std::string,
+//    internal::ExpiringCacheEntry<std::unordered_map<std::string, V>>>
+// This cache is safe to be shared between threads.
+template <typename V>
+class TwoLevelCacheWithExpiration {
+ public:
+  TwoLevelCacheWithExpiration() {
+    last_cache_cleanup_timestamp_ = internal::CurrentTimePoint();
+  }
+
+  std::shared_ptr<ConcurrentMap<V>> GetOrCreateInternalCache(
+      const std::string& access_token, uint64_t cache_entry_lifetime_ms) {

Review comment:
       In general I don't think recreating the Java API in C++ is a good idea. Common conventions and expectations are quite different accross languages.




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



[GitHub] [arrow] ggershinsky commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

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



##########
File path: cpp/src/parquet/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,207 @@
+// 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 <memory>
+
+#include "parquet/encryption.h"
+#include "parquet/file_key_wrapper.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr bool kDefaultWrapLocally = false;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    explicit Builder(const std::string& footer_key)

Review comment:
       this is a public API, so a comment above this line would be helpful; something like 
   "// ID of the master key for footer encryption/signing"




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



[GitHub] [arrow] emkornfield commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r477025570



##########
File path: cpp/src/parquet/encryption.h
##########
@@ -47,15 +47,15 @@ using ColumnPathToEncryptionPropertiesMap =
 
 class PARQUET_EXPORT DecryptionKeyRetriever {
  public:
-  virtual std::string GetKey(const std::string& key_metadata) const = 0;

Review comment:
       There are two reasons:
   1. This is breaking API change.  Users that were relying on passing this through as a const object in places can non longer do so.
   2.  Semantically, I would in most cases to have Get operations be const, even if there is caching happening underneath.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497981750



##########
File path: cpp/src/parquet/encryption/file_key_unwrapper.h
##########
@@ -0,0 +1,66 @@
+// 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 "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/key_material.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/key_toolkit_internal.h"
+#include "parquet/encryption/kms_client.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// This class will retrieve the key from "key metadata", following these steps:
+// 1. Parse "key metadata" (see structure in KeyMetadata class).
+// 2. Retrieve "key material" which can be stored inside or outside "key metadata"
+//    Currently we don't support the case "key material" stores outside "key metadata"
+//    yet.
+// 3. Unwrap the "data encryption key" from "key material". There are 2 modes:
+// 3.1. single wrapping: decrypt the wrapped "data encryption key" directly with "master
+// encryption key" 3.2. double wrapping: 2 steps: 3.2.1. "key encryption key" is decrypted
+// with "master encryption key" 3.2.2. "data encryption key" is decrypted with the above
+// "key encryption key"
+class PARQUET_EXPORT FileKeyUnwrapper : public DecryptionKeyRetriever {
+ public:
+  /// 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.
+  FileKeyUnwrapper(KeyToolkit* key_toolkit,
+                   const KmsConnectionConfig& kms_connection_config,
+                   uint64_t cache_lifetime_seconds);
+
+  std::string GetKey(const std::string& key_metadata) const override;
+
+ private:
+  internal::KeyWithMasterId GetDataEncryptionKey(const KeyMaterial& key_material) const;
+  std::shared_ptr<KmsClient> GetKmsClientFromConfigOrKeyMaterial(
+      const KeyMaterial& key_material) const;
+
+  /// A map of Key Encryption Key (KEK) ID -> KEK bytes, for the current token
+  mutable std::shared_ptr<arrow::util::ConcurrentMap<std::string>> kek_per_kek_id_;

Review comment:
       This variable references the item in cache. When the item is expired, a thread can remove it from cache, while this thread can still access it. So I use shared_ptr here to make sure this variable still alive when the item in cache is removed.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497969857



##########
File path: cpp/src/parquet/encryption/two_level_cache_with_expiration_test.cc
##########
@@ -0,0 +1,207 @@
+// 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.
+
+#include <chrono>
+#include <thread>
+
+#include <gtest/gtest.h>
+
+#include "arrow/util/concurrent_map.h"
+
+#include "parquet/encryption/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+namespace test {
+
+class TwoLevelCacheWithExpirationTest : public ::testing::Test {
+ public:
+  void SetUp() {
+    // lifetime is 1s
+    std::shared_ptr<ConcurrentMap<int>> lifetime_1s =
+        cache_.GetOrCreateInternalCache("lifetime1s", 1000);
+    lifetime_1s->Insert("item1", 1);
+    lifetime_1s->Insert("item2", 2);
+
+    // lifetime is 3s
+    std::shared_ptr<ConcurrentMap<int>> lifetime_3s =
+        cache_.GetOrCreateInternalCache("lifetime3s", 3000);
+    lifetime_3s->Insert("item21", 21);
+    lifetime_3s->Insert("item22", 22);
+  }
+
+ protected:
+  void TaskInsert(int thread_no) {
+    for (int i = 0; i < 20; i++) {
+      std::string token = i % 2 == 0 ? "lifetime1s" : "lifetime3s";
+      uint64_t lifetime_ms = i % 2 == 0 ? 1000 : 3000;
+      std::shared_ptr<ConcurrentMap<int>> internal_cache =
+          cache_.GetOrCreateInternalCache(token, lifetime_ms);
+      std::stringstream ss;
+      ss << "item_" << thread_no << "_" << i;
+      internal_cache->Insert(ss.str(), i);
+      std::this_thread::sleep_for(std::chrono::milliseconds(10));
+    }
+  }
+
+  void TaskClean() {
+    for (int i = 0; i < 20; i++) {
+      cache_.Clear();
+      std::this_thread::sleep_for(std::chrono::milliseconds(20));
+    }
+  }
+
+  TwoLevelCacheWithExpiration<int> cache_;
+};
+
+TEST_F(TwoLevelCacheWithExpirationTest, RemoveExpiration) {
+  std::shared_ptr<ConcurrentMap<int>> lifetime_1s_before_expiration =
+      cache_.GetOrCreateInternalCache("lifetime1s", 1000);
+  ASSERT_EQ(lifetime_1s_before_expiration->size(), 2);
+
+  // wait for 2s, we expect:
+  // lifetime_1s will be expired
+  // lifetime_3s will not be expired
+  std::this_thread::sleep_for(std::chrono::milliseconds(2000));
+  // now clear expired items from the cache
+  cache_.RemoveExpiredEntriesFromCache();
+
+  // lifetime_1s (with 2 items) is expired and has been removed from the cache.
+  // Now the cache create a new object which has no item.
+  std::shared_ptr<ConcurrentMap<int>> lifetime_1s =
+      cache_.GetOrCreateInternalCache("lifetime1s", 1000);
+  ASSERT_EQ(lifetime_1s->size(), 0);
+
+  // However, lifetime_1s_before_expiration can still access normally and independently
+  // from the one in cache
+  lifetime_1s_before_expiration->Insert("item3", 3);
+  ASSERT_EQ(lifetime_1s_before_expiration->size(), 3);
+  ASSERT_EQ(lifetime_1s->size(), 0);
+
+  // lifetime_3s is not expired and still contains 2 items.
+  std::shared_ptr<ConcurrentMap<int>> lifetime_3s =
+      cache_.GetOrCreateInternalCache("lifetime3s", 3000);
+  ASSERT_EQ(lifetime_3s->size(), 2);
+}
+
+TEST_F(TwoLevelCacheWithExpirationTest, CleanupPeriodTooBig) {
+  // wait for 2s, now:
+  // lifetime_1s is expired
+  // lifetime_3s isn't expired
+  std::this_thread::sleep_for(std::chrono::milliseconds(2000));
+
+  // if cleanup_period is too big (10s), the expired items may not be removed from cache.
+  cache_.CheckCacheForExpiredTokens(10000);
+
+  // lifetime_1s (with 2 items) is expired but not removed from the cache, still contains

Review comment:
       I think you're right about "what's being tested is useful" which is not. In real use case, developers should pass a small enough value for `cleanup_period` param. I will delete this 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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498116308



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;

Review comment:
       It's easy to make an attribute `const` if you want to avoid mutations.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r475668043



##########
File path: cpp/src/parquet/encryption.h
##########
@@ -47,15 +47,15 @@ using ColumnPathToEncryptionPropertiesMap =
 
 class PARQUET_EXPORT DecryptionKeyRetriever {
  public:
-  virtual std::string GetKey(const std::string& key_metadata) const = 0;

Review comment:
       Yes, I can do it. However, I don't understand why we must let it be `const` method. Can you please explain?




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



[GitHub] [arrow] pitrou commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-800231451


   @emkornfield The JSON changes are expected, the encryption layer needs to parse and generate some JSON.


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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497974822



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;
+};
+
+class PARQUET_EXPORT DecryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    Builder() : cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds) {}
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    std::shared_ptr<DecryptionConfiguration> build();
+
+   private:
+    uint64_t cache_lifetime_seconds_;
+  };
+
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+
+ private:
+  explicit DecryptionConfiguration(uint64_t cache_lifetime_seconds)
+      : cache_lifetime_seconds_(cache_lifetime_seconds) {}
+
+  uint64_t cache_lifetime_seconds_;
+};
+
+class PARQUET_EXPORT PropertiesDrivenCryptoFactory {

Review comment:
       The name is based on Java version: https://github.com/apache/parquet-mr/pull/615/files#diff-b0ae7da2b29e6a52fd3b3a91faeb02e1 and "properties driven", as I know, links with hadoop properties: https://github.com/apache/parquet-mr/pull/615/files#diff-cb88582794e86857f46fa3a3aff7ec40R36. @ggershinsky and I think we should keep the same API as much as possible between Java and C++.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497997865



##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {

Review comment:
       @pitrou  @bkietz I need a map which can be accessed by multiple threads. Please advice which is the preferred way to do 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



[GitHub] [arrow] bkietz commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r497630717



##########
File path: cpp/src/arrow/util/string.h
##########
@@ -41,6 +41,10 @@ ARROW_EXPORT Status ParseHexValue(const char* data, uint8_t* out);
 
 namespace internal {
 
+/// \brief Split a string with a delimiter
+ARROW_EXPORT
+std::vector<std::string> SplitString(util::string_view v, char delim);

Review comment:
       Why doesn't this return `vector<string_view>`?

##########
File path: cpp/src/parquet/encryption/key_encryption_key.h
##########
@@ -0,0 +1,62 @@
+// 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 <cstdint>
+#include <vector>
+
+#include "arrow/util/base64.h"
+
+namespace parquet {
+namespace encryption {
+
+// In the double wrapping mode, each "data encryption key" (DEK) is encrypted with a “key
+// encryption key” (KEK), that in turn is encrypted with a "master encryption key" (MEK).
+// In a writer process, a random KEK is generated for each MEK ID, and cached in a <MEK-ID
+// : KEK> map. This allows to perform an interaction with a KMS server only once for each
+// MEK, in order to wrap its KEK. "Data encryption key" (DEK) wrapping is performed
+// locally, and does not involve an interaction with a KMS server.
+class KeyEncryptionKey {
+ public:
+  KeyEncryptionKey(const std::string& kek_bytes, const std::string& kek_id,
+                   const std::string& encoded_wrapped_kek)
+      : kek_bytes_(kek_bytes),
+        kek_id_(kek_id),
+        encoded_wrapped_kek_(encoded_wrapped_kek) {
+    encoded_kek_id_ =
+        arrow::util::base64_encode(reinterpret_cast<const uint8_t*>(kek_id_.data()),
+                                   static_cast<uint32_t>(kek_id_.size()));
+  }

Review comment:
       Please use `move`:
   ```suggestion
     KeyEncryptionKey(std::string kek_bytes, std::string kek_id,
                      std::string encoded_wrapped_kek)
         : kek_bytes_(std::move(kek_bytes)),
           kek_id_(std::move(kek_id)),
           encoded_kek_id_(arrow::util::base64_encode(reinterpret_cast<const uint8_t*>(kek_id_.data()),
                                                      static_cast<uint32_t>(kek_id_.size()))),
           encoded_wrapped_kek_(std::move(encoded_wrapped_kek)) {}
   ```

##########
File path: cpp/src/parquet/encryption/kms_client.cc
##########
@@ -0,0 +1,47 @@
+// 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.
+
+#include "parquet/encryption/kms_client.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char KmsClient::kKmsInstanceIdDefault[];
+constexpr const char KmsClient::kKmsInstanceUrlDefault[];
+constexpr const char KmsClient::kKeyAccessTokenDefault[];

Review comment:
       No, they're static class members so even though they are constexpr their declaration https://github.com/apache/arrow/pull/8023/files#diff-063d5acd8fa092535ebe261f9f63b6b5R68 is not also a definition; these definitions of the constants indicate that the string `"DEFAULT"` is stored in `kms_client.o`

##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {

Review comment:
       In all, I think this doesn't add sufficient value over an inlined `container, mutex` pair. I think this class should be removed, maybe extracting a helper for single lookup insertion:
   ```c++
   template <typename K, typename V, typename Hash, typename Eq, typename Gen>
   auto GetOrInsert(std::unordered_map<K, V, Hash, Eq>* map, Gen&& gen, V placeholder = V{})
     -> decltype(map->begin()) {
     auto it_success = map->emplace(key, placeholder);
     if (!it_success.second) {
       // insertion of placeholder was blocked by an existing entry, return that
       return it_success.first;
     }
     // overwrite placeholder with computed value
     it_success.first->second = gen();
     return it_success.first;
   }
   ```

##########
File path: cpp/src/arrow/json/object_parser.h
##########
@@ -0,0 +1,45 @@
+// 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 "arrow/json/rapidjson_defs.h"  // IWYU pragma: keep
+
+#include <rapidjson/document.h>
+
+#include "arrow/result.h"
+#include "arrow/util/string_view.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace json {
+
+namespace rj = arrow::rapidjson;
+
+class ARROW_EXPORT ObjectParser {
+ public:
+  bool Parse(arrow::util::string_view json);
+
+  Result<std::string> GetString(const char* key) const;
+  Result<bool> GetBool(const char* key) const;
+
+ private:
+  rj::Document _document;

Review comment:
       Since this ObjectParser and ObjectWriter are fairly thin wrappers around rj::Document they can be removed or at least made `internal`. For example, the only public mention of it ObjectParser is KeyMaterial::Parse where it could be replaced by a string.

##########
File path: cpp/src/arrow/util/concurrent_map.h
##########
@@ -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.
+
+#pragma once
+
+#include <functional>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/mutex.h"
+
+namespace arrow {
+namespace util {
+
+template <typename V>
+class ConcurrentMap {
+ public:
+  void Insert(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_.insert({key, value});
+  }
+
+  void Assign(const std::string& key, const V& value) {
+    auto lock = mutex_.Lock();
+    map_[key] = value;
+  }
+
+  V GetOrAssignIfNotExist(const std::string& key, std::function<V()> compute_value_func) {
+    auto lock = mutex_.Lock();
+    auto it = map_.find(key);
+    if (it == map_.end()) {
+      map_.insert({key, compute_value_func()});
+    }
+    return map_.at(key);

Review comment:
       Can be accomplished with one if you have a lightweight default/placeholder for `V`:
   ```c++
   auto it_success = map_.emplace(key, V{});
   V* value_or_placeholder = &it_success->first->second;
   if (!it_success.second) {
     // insert was blocked by an existing entry, return that
     return *value_or_placeholder;
   }
   // overwrite placeholder with computed value
   *value_or_placeholder = compute_value_func();
   return *value_or_placeholder;
   ```




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498153010



##########
File path: cpp/src/parquet/encryption/key_metadata.h
##########
@@ -0,0 +1,92 @@
+// 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 <string>
+
+#include "arrow/util/variant.h"
+
+#include "parquet/encryption/key_material.h"
+#include "parquet/exception.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+// Parquet encryption specification defines "key metadata" as an arbitrary byte array,
+// generated by file writers for each encryption key, and passed to the low level API for
+// storage in the file footer. The "key metadata" field is made available to file readers
+// to enable recovery of the key. This interface can be utilized for implementation
+// of any key management scheme.
+//
+// The keytools package (PARQUET-1373) implements one approach, of many possible, to key
+// management and to generation of the "key metadata" fields. This approach, based on the
+// "envelope encryption" pattern, allows integration with KMS servers. It keeps the actual
+// material, required to recover a key, in a "key material" object (see the KeyMaterial
+// class for details). This class is implemented to support version 1 of the parquet key
+// management tools specification.
+//
+// KeyMetadata writes (and reads) the "key metadata" field as a flat json object,
+// with the following fields:
+// 1. "keyMaterialType" - a String, with the type of  key material.
+// 2. "internalStorage" - a boolean. If true, means that "key material" is kept inside the
+// "key metadata" field. If false, "key material" is kept externally (outside Parquet
+// files) - in this case, "key metadata" keeps a reference to the external "key material".
+// 3. "keyReference" - a String, with the reference to the external "key material".
+// Written only if internalStorage is false.
+//
+// If internalStorage is true, "key material" is a part of "key metadata", and the json
+// keeps additional fields, described in the KeyMaterial class.
+class PARQUET_EXPORT KeyMetadata {
+ public:
+  static constexpr const char kKeyMaterialInternalStorageField[] = "internalStorage";
+  static constexpr const char kKeyReferenceField[] = "keyReference";
+
+  static KeyMetadata Parse(const std::string& key_metadata_bytes);

Review comment:
       yes, `key_metadata_bytes` is expected to be a serialized json object.




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



[GitHub] [arrow] thamht4190 commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-699915124


   I've fixed all comments in the first review round. @pitrou @emkornfield @ggershinsky Can you please take a look? I guess round 2 will take time to review as well since the code change is huge. Thanks!
   


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



[GitHub] [arrow] pitrou commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r498078212



##########
File path: cpp/src/parquet/encryption/properties_driven_crypto_factory.h
##########
@@ -0,0 +1,210 @@
+// 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 <memory>
+
+#include "parquet/encryption/encryption.h"
+#include "parquet/encryption/file_key_wrapper.h"
+#include "parquet/encryption/key_toolkit.h"
+#include "parquet/encryption/kms_client_factory.h"
+#include "parquet/platform.h"
+
+namespace parquet {
+namespace encryption {
+
+static constexpr ParquetCipher::type kDefaultEncryptionAlgorithm =
+    ParquetCipher::AES_GCM_V1;
+static constexpr bool kDefaultPlaintextFooter = false;
+static constexpr bool kDefaultDoubleWrapping = true;
+static constexpr uint64_t kDefaultCacheLifetimeSeconds = 600;  // 10 minutes
+static constexpr bool kDefaultInternalKeyMaterial = true;
+static constexpr bool kDefaultUniformEncryption = false;
+static constexpr int32_t kDefaultDataKeyLengthBits = 128;
+
+class PARQUET_EXPORT EncryptionConfiguration {
+ public:
+  class PARQUET_EXPORT Builder {
+   public:
+    /// footer_key: ID of the master key for footer encryption/signing
+    explicit Builder(const std::string& footer_key)
+        : footer_key_(footer_key),
+          encryption_algorithm_(kDefaultEncryptionAlgorithm),
+          plaintext_footer_(kDefaultPlaintextFooter),
+          double_wrapping_(kDefaultDoubleWrapping),
+          cache_lifetime_seconds_(kDefaultCacheLifetimeSeconds),
+          internal_key_material_(kDefaultInternalKeyMaterial),
+          uniform_encryption_(kDefaultUniformEncryption),
+          data_key_length_bits_(kDefaultDataKeyLengthBits) {}
+
+    /// List of columns to encrypt, with master key IDs (see HIVE-21848).
+    /// Format: "masterKeyID:colName,colName;masterKeyID:colName..."
+    /// Either
+    /// column_keys(const std::string&)
+    /// or
+    /// uniform_encryption()
+    /// must be called. If none are called, or if both are called, an exception will be
+    /// thrown.
+    Builder* column_keys(const std::string& column_keys);
+
+    /// encrypt footer and all columns with the same encryption key.
+    Builder* uniform_encryption();
+
+    /// Parquet encryption algorithm. Can be "AES_GCM_V1" (default), or "AES_GCM_CTR_V1".
+    Builder* encryption_algorithm(ParquetCipher::type algo);
+
+    /// Write files with plaintext footer.
+    /// The default is false - files are written with encrypted footer.
+    Builder* plaintext_footer(bool plaintext_footer);
+
+    /// Use double wrapping - where data encryption keys (DEKs) are encrypted with key
+    /// encryption keys (KEKs), which in turn are encrypted with master keys.
+    /// The default is true. If set to false, use single wrapping - where DEKs are
+    /// encrypted directly with master keys.
+    Builder* double_wrapping(bool double_wrapping);
+
+    /// Lifetime of cached entities (key encryption keys, local wrapping keys, KMS client
+    /// objects).
+    /// The default is 600 (10 minutes).
+    Builder* cache_lifetime_seconds(uint64_t cache_lifetime_seconds);
+
+    /// Store key material inside Parquet file footers; this mode doesn’t produce
+    /// additional files. By default, true. If set to false, key material is stored in
+    /// separate files in the same folder, which enables key rotation for immutable
+    /// Parquet files.
+    Builder* internal_key_material(bool internal_key_material);
+
+    /// Length of data encryption keys (DEKs), randomly generated by parquet key
+    /// management tools. Can be 128, 192 or 256 bits.
+    /// The default is 128 bits.
+    Builder* data_key_length_bits(int32_t data_key_length_bits);
+
+    std::shared_ptr<EncryptionConfiguration> build();
+
+   private:
+    std::string footer_key_;
+    std::string column_keys_;
+    ParquetCipher::type encryption_algorithm_;
+    bool plaintext_footer_;
+    bool double_wrapping_;
+    uint64_t cache_lifetime_seconds_;
+    bool internal_key_material_;
+    bool uniform_encryption_;
+    int32_t data_key_length_bits_;
+  };
+
+  const std::string& footer_key() const { return footer_key_; }
+  const std::string& column_keys() const { return column_keys_; }
+  ParquetCipher::type encryption_algorithm() const { return encryption_algorithm_; }
+  bool plaintext_footer() const { return plaintext_footer_; }
+  bool double_wrapping() const { return double_wrapping_; }
+  uint64_t cache_lifetime_seconds() const { return cache_lifetime_seconds_; }
+  bool internal_key_material() const { return internal_key_material_; }
+  bool uniform_encryption() const { return uniform_encryption_; }
+  int32_t data_key_length_bits() const { return data_key_length_bits_; }
+
+ private:
+  EncryptionConfiguration(const std::string& footer_key, const std::string& column_keys,
+                          ParquetCipher::type encryption_algorithm, bool plaintext_footer,
+                          bool double_wrapping, uint64_t cache_lifetime_seconds,
+                          bool internal_key_material, bool uniform_encryption,
+                          int32_t data_key_length_bits)
+      : footer_key_(footer_key),
+        column_keys_(column_keys),
+        encryption_algorithm_(encryption_algorithm),
+        plaintext_footer_(plaintext_footer),
+        double_wrapping_(double_wrapping),
+        cache_lifetime_seconds_(cache_lifetime_seconds),
+        internal_key_material_(internal_key_material),
+        uniform_encryption_(uniform_encryption),
+        data_key_length_bits_(data_key_length_bits) {}
+
+  std::string footer_key_;
+  std::string column_keys_;
+  ParquetCipher::type encryption_algorithm_;
+  bool plaintext_footer_;
+  bool double_wrapping_;
+  uint64_t cache_lifetime_seconds_;
+  bool internal_key_material_;
+  bool uniform_encryption_;
+  int32_t data_key_length_bits_;

Review comment:
       Hmm... unfortunately, we don't have a uniform convention (especially as the Parquet C++ codebase used to live its own life), but in the future I think we'd rather use simple structs such as those:
   https://github.com/apache/arrow/blob/master/cpp/src/arrow/csv/options.h
   
   I'll turn the question on its head: the Builder pattern involves a lot of boilerplate and maintenance, so what does it bring compared to a simple struct?




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r486382035



##########
File path: cpp/src/parquet/file_key_wrapper.h
##########
@@ -0,0 +1,68 @@
+// 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 <map>
+#include <memory>
+#include <string>
+
+#include "parquet/file_key_material_store.h"
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+
+namespace parquet {
+namespace encryption {
+
+class FileKeyWrapper {
+ public:
+  static constexpr int KEK_LENGTH = 16;
+  static constexpr int KEK_ID_LENGTH = 16;
+
+  static constexpr int RND_MAX_BYTES = 32;
+
+  FileKeyWrapper(std::shared_ptr<KmsClientFactory> kms_client_factory,
+                 const KmsConnectionConfig& kms_connection_config,
+                 std::shared_ptr<FileKeyMaterialStore> key_material_store,
+                 uint64_t cache_entry_lifetime, bool double_wrapping,
+                 bool is_wrap_locally);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key);
+
+  std::string GetEncryptionKeyMetadata(const std::string& data_key,
+                                       const std::string& master_key_id,
+                                       bool is_footer_key, std::string key_id_in_file);
+
+ private:
+  KeyEncryptionKey CreateKeyEncryptionKey(const std::string& master_key_id);
+
+  // A map of MEK_ID -> KeyEncryptionKey, for the current token
+  std::map<std::string, KeyEncryptionKey> kek_per_master_key_id_;
+
+  std::shared_ptr<KmsClient> kms_client_;
+  KmsConnectionConfig kms_connection_config_;
+  std::shared_ptr<FileKeyMaterialStore> key_material_store_;
+  uint64_t cache_entry_lifetime_;
+  bool double_wrapping_;
+  uint16_t key_counter_;

Review comment:
       I removed it anw, since it's for external storage.




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r482995943



##########
File path: cpp/src/parquet/key_toolkit.h
##########
@@ -0,0 +1,123 @@
+// 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 <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/key_encryption_key.h"
+#include "parquet/kms_client.h"
+#include "parquet/kms_client_factory.h"
+#include "parquet/platform.h"
+#include "parquet/two_level_cache_with_expiration.h"
+
+namespace parquet {
+namespace encryption {
+
+class KeyWithMasterID {
+ public:
+  KeyWithMasterID(const std::string& key_bytes, const std::string& master_id)
+      : key_bytes_(key_bytes), master_id_(master_id) {}
+
+  const std::string& data_key() const { return key_bytes_; }
+  const std::string& master_id() const { return master_id_; }
+
+ private:
+  std::string key_bytes_;
+  std::string master_id_;
+};
+
+class PARQUET_EXPORT KeyToolkit {
+ public:
+  class KmsClientCache {
+   public:
+    static KmsClientCache& GetInstance() {
+      static KmsClientCache instance;

Review comment:
       to @ggershinsky 




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



[GitHub] [arrow] thamht4190 commented on a change in pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on a change in pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#discussion_r475368729



##########
File path: cpp/src/parquet/encryption.h
##########
@@ -47,15 +47,15 @@ using ColumnPathToEncryptionPropertiesMap =
 
 class PARQUET_EXPORT DecryptionKeyRetriever {
  public:
-  virtual std::string GetKey(const std::string& key_metadata) const = 0;

Review comment:
       @emkornfield because inherited classes (i.e. `FileKeyUnwrapper` class in this pull) may need to modify its members inside this method.




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



[GitHub] [arrow] pitrou closed pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
pitrou closed pull request #8023:
URL: https://github.com/apache/arrow/pull/8023


   


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



[GitHub] [arrow] thamht4190 commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
thamht4190 commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-787455523


   I fixed all the comments in this pull, can you please take a look? @emkornfield @pitrou @bkietz 


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



[GitHub] [arrow] emkornfield commented on pull request #8023: ARROW-9318: [C++] Parquet encryption key management

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #8023:
URL: https://github.com/apache/arrow/pull/8023#issuecomment-800400946


   > @emkornfield The JSON changes are expected, the encryption layer needs to parse and generate some JSON.
   
   Ah right,  I forgot, will start looking again.


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