You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/10/01 08:24:31 UTC

[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1178: MINIFICPP-1643 Add Managed Identity support for Azure processors

adamdebreceni commented on a change in pull request #1178:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1178#discussion_r720047763



##########
File path: extensions/azure/processors/PutAzureBlobStorage.cpp
##########
@@ -138,38 +149,75 @@ void PutAzureBlobStorage::onSchedule(const std::shared_ptr<core::ProcessContext>
   logger_->log_info("Using storage account name and SAS token for authentication");
 }
 
-std::string PutAzureBlobStorage::getAzureConnectionStringFromProperties(
+storage::AzureStorageCredentials PutAzureBlobStorage::getAzureCredentialsFromProperties(
     const std::shared_ptr<core::ProcessContext> &context,
-    const std::shared_ptr<core::FlowFile> &flow_file) {
+    const std::shared_ptr<core::FlowFile> &flow_file) const {
   storage::AzureStorageCredentials credentials;
-  context->getProperty(StorageAccountName, credentials.storage_account_name, flow_file);
-  context->getProperty(StorageAccountKey, credentials.storage_account_key, flow_file);
-  context->getProperty(SASToken, credentials.sas_token, flow_file);
-  context->getProperty(CommonStorageAccountEndpointSuffix, credentials.endpoint_suffix, flow_file);
-  context->getProperty(ConnectionString, credentials.connection_string, flow_file);
-  return credentials.getConnectionString();
+  std::string value;
+  if (context->getProperty(StorageAccountName, value, flow_file)) {
+    credentials.setStorageAccountName(value);
+  }
+  if (context->getProperty(StorageAccountKey, value, flow_file)) {
+    credentials.setStorageAccountKey(value);
+  }
+  if (context->getProperty(SASToken, value, flow_file)) {
+    credentials.setSasToken(value);
+  }
+  if (context->getProperty(CommonStorageAccountEndpointSuffix, value, flow_file)) {
+    credentials.setEndpontSuffix(value);
+  }
+  if (context->getProperty(ConnectionString, value, flow_file)) {
+    credentials.setConnectionString(value);
+  }
+  credentials.setUseManagedIdentityCredentials(use_managed_identity_credentials_);
+  return credentials;
 }
 
-void PutAzureBlobStorage::createAzureStorageClient(const std::string &connection_string, const std::string &container_name) {
-  // When used in multithreaded environment make sure to use the azure_storage_mutex_ to lock the wrapper so the
-  // client is not reset with different configuration while another thread is using it.
-  if (blob_storage_wrapper_ == nullptr) {
-    blob_storage_wrapper_ = std::make_unique<storage::AzureBlobStorage>(connection_string, container_name);
-    return;
+std::optional<storage::PutAzureBlobStorageParameters> PutAzureBlobStorage::buildAzureBlobStorageParameters(
+    const std::shared_ptr<core::ProcessContext> &context,
+    const std::shared_ptr<core::FlowFile> &flow_file) {
+  storage::PutAzureBlobStorageParameters params;
+  auto credentials = getCredentials(context, flow_file);
+  if (!credentials) {
+    logger_->log_error("No valid credentials are set!");
+    return std::nullopt;
+  }
+
+  params.credentials = *credentials;
+
+  if (!context->getProperty(ContainerName, params.container_name, flow_file) || params.container_name.empty()) {
+    logger_->log_error("Container Name is invalid or empty!");
+    return std::nullopt;
+  }
+
+  context->getProperty(Blob, params.blob_name, flow_file);
+  if (params.blob_name.empty() && (!flow_file->getAttribute("filename", params.blob_name) || params.blob_name.empty())) {
+    logger_->log_error("Blob is not set and default 'filename' attribute could not be found!");
+    return std::nullopt;
   }
 
-  blob_storage_wrapper_->resetClientIfNeeded(connection_string, container_name);
+  return params;
 }
 
-std::string PutAzureBlobStorage::getConnectionString(
+std::optional<storage::AzureStorageCredentials> PutAzureBlobStorage::getCredentials(
     const std::shared_ptr<core::ProcessContext> &context,
     const std::shared_ptr<core::FlowFile> &flow_file) const {
-  auto connection_string = getAzureConnectionStringFromProperties(context, flow_file);
-  if (!connection_string.empty()) {
-    return connection_string;
+  auto controller_service_creds = getCredentialsFromControllerService(context);
+  if (controller_service_creds && controller_service_creds->isValid()) {
+    logger_->log_debug("Azure credentials read from credentials controller service!");
+    return controller_service_creds;
+  }
+
+  logger_->log_debug("No valid Azure credentials are set in credentials controller service, checking properties...");
+
+  auto property_creds = getAzureCredentialsFromProperties(context, flow_file);
+  if (property_creds.isValid()) {
+    logger_->log_debug("Azure credentials read from properties!");
+    return property_creds;
   }
 
-  return getConnectionStringFromControllerService(context);
+  logger_->log_error("No valid Azure credentials are set in credentials controller service nor in properties!");

Review comment:
       we already log an error after calling this method and it returns a nullopt at line 182




-- 
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: issues-unsubscribe@nifi.apache.org

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