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/24 05:32:12 UTC

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

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