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

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

adamreeve commented on code in PR #10491:
URL: https://github.com/apache/arrow/pull/10491#discussion_r1093763415


##########
cpp/src/parquet/encryption/key_toolkit.cc:
##########
@@ -35,6 +46,89 @@ std::shared_ptr<KmsClient> KeyToolkit::GetKmsClient(
       });
 }
 
+// Filter out files that are not suitable for key rotation.
+inline bool filter_out_file(std::string child) {
+  // Filters out hidden files. A file is considered to
+  // be hidden, and should not be considered for processing, when the file name
+  // starts with a period ('.') or an underscore ('_').
+  if (child.at(0) == kPeriod || child.at(0) == kUnderscore) return true;
+  return false;
+}
+
+void KeyToolkit::RotateMasterKeys(const KmsConnectionConfig& kms_connection_config,
+                                  const std::shared_ptr<FilePath>& folder_path,
+                                  bool double_wrapping, double cache_lifetime_seconds) {
+  // If process wrote files with double-wrapped keys, clean KEK cache (since master keys
+  // are changing). Only once for each key rotation cycle; not for every folder
+  const auto now = internal::CurrentTimePoint();
+  auto lock = last_cache_clean_for_key_rotation_time_mutex_.Lock();
+  if (now > last_cache_clean_for_key_rotation_time_ +
+                std::chrono::duration<double>(kCacheCleanPeriodForKeyRotation)) {
+    kek_write_cache_per_token().Clear();
+    last_cache_clean_for_key_rotation_time_ = now;
+  }
+  lock.Unlock();
+  std::vector<::arrow::fs::FileInfo> parquet_files_in_folder;
+  ::arrow::fs::FileSelector s;
+  s.base_dir = folder_path->dir_name();
+  parquet_files_in_folder = folder_path->filesystem()->GetFileInfo(s).ValueOrDie();
+
+  if (parquet_files_in_folder.size() == 0) {
+    throw ParquetException("Couldn't rotate keys - no parquet files in folder " +
+                           folder_path->dir_name());
+  }
+
+  for (auto const& parquet_file : parquet_files_in_folder) {
+    if (parquet_file.type() != ::arrow::fs::FileType::File)
+      throw ParquetException("Expecting file type in " + folder_path->dir_name());
+    std::string child = parquet_file.base_name();
+    if (filter_out_file(child)) continue;

Review Comment:
     Would it make more sense to only look for files that begin with the `_KEY_MATERIAL_FOR_` prefix, rather than assuming that any file that doesn't begin with "." or "_" is a Parquet file with an associated external key material file? Otherwise this will fail for any directories that contain any other type of file.



##########
cpp/src/parquet/encryption/crypto_factory.cc:
##########
@@ -48,8 +51,20 @@ std::shared_ptr<FileEncryptionProperties> CryptoFactory::GetFileEncryptionProper
 
   std::shared_ptr<FileKeyMaterialStore> key_material_store = NULL;
   if (!encryption_config.internal_key_material) {
-    // TODO: using external key material store with Hadoop file system
-    throw ParquetException("External key material store is not supported yet.");
+    if (parquet_file_path == nullptr) {
+      std::stringstream ss;
+      ss << "Output file path cannot be null"

Review Comment:
   It would be good to clarify this a bit:
   ```suggestion
         ss << "Output file path cannot be null when using external key material"
   ```



##########
cpp/src/parquet/encryption/file_system_key_material_store.cc:
##########
@@ -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.
+
+#include "arrow/filesystem/filesystem.h"
+#include "arrow/testing/json_integration.h"
+#include "arrow/testing/json_internal.h"
+#include "arrow/util/io_util.h"
+#include "arrow/util/make_unique.h"
+
+#include "parquet/encryption/file_system_key_material_store.h"
+#include "parquet/encryption/key_material.h"
+
+namespace parquet {
+namespace encryption {
+
+constexpr const char FileSystemKeyMaterialStore::kKetMaterialFilePrefix[];

Review Comment:
   Typo, should be `kKeyMaterialFilePrefix`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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