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/09/15 10:38:02 UTC

[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #1158: MINIFICPP-1616 Create PutAzureDatalakeStorage processor

fgerlits commented on a change in pull request #1158:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1158#discussion_r709000677



##########
File path: extensions/azure/processors/AzureStorageProcessor.h
##########
@@ -0,0 +1,51 @@
+/**
+ * @file AzureStorageProcessor.h
+ * AzureStorageProcessor class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "core/Property.h"
+#include "core/Processor.h"
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+class AzureStorageProcessor : public core::Processor {
+ public:
+  // Supported Properties
+  static const core::Property AzureStorageCredentialsService;
+
+  explicit AzureStorageProcessor(const std::string& name, const minifi::utils::Identifier& uuid, const std::shared_ptr<logging::Logger>& logger)

Review comment:
       this is not a single-argument constructor, so it doesn't need to be explicit

##########
File path: .github/workflows/ci.yml
##########
@@ -35,7 +35,7 @@ jobs:
           export LDFLAGS="-L/usr/local/opt/flex/lib"
           export CPPFLAGS="-I/usr/local/opt/flex/include"
           # CPPFLAGS are not recognized by cmake, so we have to force them to CFLAGS and CXXFLAGS to have flex 2.6 working
-          ./bootstrap.sh -e -t && cd build  && cmake -DCMAKE_BUILD_TYPE=Release -DCMAKE_C_FLAGS="${CPPFLAGS} ${CFLAGS}" -DCMAKE_CXX_FLAGS="${CPPFLAGS} ${CXXFLAGS}" -DENABLE_LUA_SCRIPTING=ON -DENABLE_SQL=ON -DUSE_REAL_ODBC_TEST_DRIVER=ON -DCMAKE_VERBOSE_MAKEFILE=ON -DCMAKE_RULE_MESSAGES=OFF -DSTRICT_GSL_CHECKS=AUDIT -DFAIL_ON_WARNINGS=ON .. && cmake --build . --parallel 4
+          ./bootstrap.sh -e -t && cd build  && cmake -DCMAKE_BUILD_TYPE=Release -DCMAKE_C_FLAGS="${CPPFLAGS} ${CFLAGS}" -DCMAKE_CXX_FLAGS="${CPPFLAGS} ${CXXFLAGS}" -DENABLE_LUA_SCRIPTING=ON -DENABLE_SQL=ON -DUSE_REAL_ODBC_TEST_DRIVER=ON -DENABLE_AZURE=ON -DCMAKE_VERBOSE_MAKEFILE=ON -DCMAKE_RULE_MESSAGES=OFF -DSTRICT_GSL_CHECKS=AUDIT -DFAIL_ON_WARNINGS=ON .. && cmake --build . --parallel 4

Review comment:
       Why is this in the MacOS run?  Depending on who is going to use it, it might make more sense to add it to either the Windows run or one of the Linux runs.

##########
File path: extensions/azure/processors/PutAzureDataLakeStorage.h
##########
@@ -0,0 +1,106 @@
+/**
+ * @file PutAzureDataLakeStorage.h
+ * PutAzureDataLakeStorage class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <utility>
+#include <string>
+#include <memory>
+#include <optional>
+#include <vector>
+
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "storage/AzureDataLakeStorage.h"
+#include "utils/Enum.h"
+#include "AzureStorageProcessor.h"
+
+class PutAzureDataLakeStorageTestsFixture;
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+class PutAzureDataLakeStorage final : public AzureStorageProcessor {
+ public:
+  // Supported Properties
+  static const core::Property FilesystemName;
+  static const core::Property DirectoryName;
+  static const core::Property FileName;
+  static const core::Property ConflictResolutionStrategy;
+
+  // Supported Relationships
+  static const core::Relationship Failure;
+  static const core::Relationship Success;
+
+  SMART_ENUM(FileExistsResolutionStrategy,
+    (FAIL_FLOW, "fail"),
+    (REPLACE_FILE, "replace"),
+    (IGNORE_REQUEST, "ignore")
+  )
+
+  explicit PutAzureDataLakeStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
+    : PutAzureDataLakeStorage(name, uuid, nullptr) {
+  }
+
+  ~PutAzureDataLakeStorage() override = default;
+
+  void initialize() override;
+  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+
+ private:
+  friend class ::PutAzureDataLakeStorageTestsFixture;
+
+  class ReadCallback : public InputStreamCallback {
+   public:
+    ReadCallback(uint64_t flow_size, storage::AzureDataLakeStorage& azure_data_lake_storage, const storage::PutAzureDataLakeStorageParameters& params, std::shared_ptr<logging::Logger> logger);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream) override;
+
+    azure::storage::UploadDataLakeStorageResult getResult() const {

Review comment:
       `storage` and `azure::storage` are the same namespace, right?  it would be better to always use one or the other (at least in the same class)

##########
File path: extensions/azure/storage/AzureDataLakeStorageClient.cpp
##########
@@ -0,0 +1,55 @@
+/**
+ * @file AzureDataLakeStorageClient.cpp
+ * AzureDataLakeStorageClient class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 "AzureDataLakeStorageClient.h"
+
+namespace org::apache::nifi::minifi::azure::storage {
+
+void AzureDataLakeStorageClient::resetClientIfNeeded(const std::string& connection_string, const std::string& file_system_name) {
+  if (client_ == nullptr || connection_string_ != connection_string || file_system_name_ != file_system_name) {
+    client_ = std::make_unique<Azure::Storage::Files::DataLake::DataLakeFileSystemClient>(
+      Azure::Storage::Files::DataLake::DataLakeFileSystemClient::CreateFromConnectionString(connection_string, file_system_name));
+    file_system_name_ = file_system_name;
+    connection_string_ = connection_string;
+  }
+}
+
+Azure::Storage::Files::DataLake::DataLakeFileClient AzureDataLakeStorageClient::getFileClient(const PutAzureDataLakeStorageParameters& params) {
+  resetClientIfNeeded(params.connection_string, params.file_system_name);
+  auto directory_client = client_->GetDirectoryClient(params.directory_name);
+  if (!params.directory_name.empty()) {
+    directory_client.CreateIfNotExists();
+  }
+  return directory_client.GetFileClient(params.filename);
+}
+
+bool AzureDataLakeStorageClient::createFile(const PutAzureDataLakeStorageParameters& params) {
+  auto file_client = getFileClient(params);
+  auto response = file_client.CreateIfNotExists();
+  return response.Value.Created;

Review comment:
       Is it possible to detect why the file creation failed?  I would expect three different return values here: Success, File Already Exists, and Other Error.  In the last case we don't want to ignore the error, whatever the Conflict Resolution Strategy is.

##########
File path: extensions/azure/processors/AzureStorageProcessor.h
##########
@@ -0,0 +1,51 @@
+/**
+ * @file AzureStorageProcessor.h
+ * AzureStorageProcessor class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "core/Property.h"
+#include "core/Processor.h"
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+class AzureStorageProcessor : public core::Processor {

Review comment:
       As this is not really a processor, just a common base for processors, I would rename it `AzureStorageProcessorBase`, and make its constructor protected.

##########
File path: extensions/azure/processors/PutAzureDataLakeStorage.h
##########
@@ -0,0 +1,106 @@
+/**
+ * @file PutAzureDataLakeStorage.h
+ * PutAzureDataLakeStorage class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <utility>
+#include <string>
+#include <memory>
+#include <optional>
+#include <vector>
+
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "storage/AzureDataLakeStorage.h"
+#include "utils/Enum.h"
+#include "AzureStorageProcessor.h"
+
+class PutAzureDataLakeStorageTestsFixture;
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+class PutAzureDataLakeStorage final : public AzureStorageProcessor {
+ public:
+  // Supported Properties
+  static const core::Property FilesystemName;
+  static const core::Property DirectoryName;
+  static const core::Property FileName;
+  static const core::Property ConflictResolutionStrategy;
+
+  // Supported Relationships
+  static const core::Relationship Failure;
+  static const core::Relationship Success;
+
+  SMART_ENUM(FileExistsResolutionStrategy,
+    (FAIL_FLOW, "fail"),
+    (REPLACE_FILE, "replace"),
+    (IGNORE_REQUEST, "ignore")
+  )
+
+  explicit PutAzureDataLakeStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
+    : PutAzureDataLakeStorage(name, uuid, nullptr) {
+  }
+
+  ~PutAzureDataLakeStorage() override = default;

Review comment:
       this can be removed

##########
File path: libminifi/test/azure-tests/PutAzureDataLakeStorageTests.cpp
##########
@@ -0,0 +1,272 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 "../TestBase.h"
+#include "utils/IntegrationTestUtils.h"
+#include "core/Processor.h"
+#include "processors/PutAzureDataLakeStorage.h"
+#include "processors/GetFile.h"
+#include "processors/PutFile.h"
+#include "processors/LogAttribute.h"
+#include "processors/UpdateAttribute.h"
+#include "storage/DataLakeStorageClient.h"
+#include "utils/file/FileUtils.h"
+#include "controllerservices/AzureStorageCredentialsService.h"
+
+using namespace std::chrono_literals;
+
+const std::string FILESYSTEM_NAME = "testfilesystem";
+const std::string DIRECTORY_NAME = "testdir";
+const std::string FILE_NAME = "testfile.txt";
+const std::string CONNECTION_STRING = "test-connectionstring";
+const std::string TEST_DATA = "data123";
+const std::string GETFILE_FILE_NAME = "input_data.log";
+
+class MockDataLakeStorageClient : public minifi::azure::storage::DataLakeStorageClient {
+ public:
+  const std::string PRIMARY_URI = "http://test-uri/file";
+
+  bool createFile(const minifi::azure::storage::PutAzureDataLakeStorageParameters& /*params*/) override {
+    if (file_creation_error_) {
+      throw std::runtime_error("error");
+    }
+    return create_file_;
+  }
+
+  std::string uploadFile(const minifi::azure::storage::PutAzureDataLakeStorageParameters& params, const uint8_t* buffer, std::size_t buffer_size) override {
+    input_data_ = std::string(buffer, buffer + buffer_size);
+    params_ = params;
+
+    if (upload_fails_) {
+      throw std::runtime_error("error");
+    }
+
+    return RETURNED_PRIMARY_URI;
+  }
+
+  void setFileCreation(bool create_file) {
+    create_file_ = create_file;
+  }
+
+  void setFileCreationError(bool file_creation_error) {
+    file_creation_error_ = file_creation_error;
+  }
+
+  void setUploadFailure(bool upload_fails) {
+    upload_fails_ = upload_fails;
+  }
+
+  minifi::azure::storage::PutAzureDataLakeStorageParameters getPassedParams() const {
+    return params_;
+  }
+
+ private:
+  const std::string RETURNED_PRIMARY_URI = "http://test-uri/file?secret-sas";
+  bool create_file_ = true;
+  bool file_creation_error_ = false;
+  bool upload_fails_ = false;
+  std::string input_data_;
+  minifi::azure::storage::PutAzureDataLakeStorageParameters params_;
+};
+
+class PutAzureDataLakeStorageTestsFixture {
+ public:
+  PutAzureDataLakeStorageTestsFixture() {
+    LogTestController::getInstance().setDebug<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::core::Processor>();
+    LogTestController::getInstance().setTrace<minifi::core::ProcessSession>();
+    LogTestController::getInstance().setTrace<processors::GetFile>();
+    LogTestController::getInstance().setTrace<processors::PutFile>();
+    LogTestController::getInstance().setDebug<processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<processors::LogAttribute>();
+    LogTestController::getInstance().setTrace<minifi::azure::processors::PutAzureDataLakeStorage>();
+
+    // Build MiNiFi processing graph
+    plan_ = test_controller_.createPlan();
+    auto mock_data_lake_storage_client = std::make_unique<MockDataLakeStorageClient>();
+    mock_data_lake_storage_client_ptr_ = mock_data_lake_storage_client.get();
+    put_azure_data_lake_storage_ = std::shared_ptr<minifi::azure::processors::PutAzureDataLakeStorage>(
+      new minifi::azure::processors::PutAzureDataLakeStorage("PutAzureDataLakeStorage", utils::Identifier(), std::move(mock_data_lake_storage_client)));
+    auto input_dir = test_controller_.createTempDirectory();
+    std::ofstream input_file_stream(input_dir + utils::file::FileUtils::get_separator() + GETFILE_FILE_NAME);
+    input_file_stream << TEST_DATA;
+    input_file_stream.close();

Review comment:
       you could use `putFileToDir` (in TestUtils.h) for this

##########
File path: extensions/azure/processors/PutAzureDataLakeStorage.cpp
##########
@@ -0,0 +1,167 @@
+/**
+ * @file PutAzureDataLakeStorage.cpp
+ * PutAzureDataLakeStorage class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 "PutAzureDataLakeStorage.h"
+
+#include "utils/ProcessorConfigUtils.h"
+#include "utils/gsl.h"
+#include "controllerservices/AzureStorageCredentialsService.h"
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+const core::Property PutAzureDataLakeStorage::FilesystemName(
+    core::PropertyBuilder::createProperty("Filesystem Name")
+      ->withDescription("Name of the Azure Storage File System. It is assumed to be already existing.")
+      ->supportsExpressionLanguage(true)
+      ->isRequired(true)
+      ->build());
+const core::Property PutAzureDataLakeStorage::DirectoryName(
+    core::PropertyBuilder::createProperty("Directory Name")
+      ->withDescription("Name of the Azure Storage Directory. The Directory Name cannot contain a leading '/'. "
+                        "If left empty it designates the root directory. The directory will be created if not already existing.")
+      ->supportsExpressionLanguage(true)
+      ->build());
+const core::Property PutAzureDataLakeStorage::FileName(
+    core::PropertyBuilder::createProperty("File Name")
+      ->withDescription("The filename")
+      ->supportsExpressionLanguage(true)
+      ->build());
+const core::Property PutAzureDataLakeStorage::ConflictResolutionStrategy(
+    core::PropertyBuilder::createProperty("Conflict Resolution Strategy")
+      ->withDescription("Indicates what should happen when a file with the same name already exists in the output directory.")
+      ->isRequired(true)
+      ->withDefaultValue<std::string>(toString(FileExistsResolutionStrategy::FAIL_FLOW))
+      ->withAllowableValues<std::string>(FileExistsResolutionStrategy::values())
+      ->build());
+
+const core::Relationship PutAzureDataLakeStorage::Success("success", "Files that have been successfully written to Azure storage are transferred to this relationship");
+const core::Relationship PutAzureDataLakeStorage::Failure("failure", "Files that could not be written to Azure storage for some reason are transferred to this relationship");
+
+void PutAzureDataLakeStorage::initialize() {
+  // Set the supported properties
+  updateSupportedProperties({
+    FilesystemName,
+    DirectoryName,
+    FileName,
+    ConflictResolutionStrategy
+  });
+  // Set the supported relationships
+  setSupportedRelationships({
+    Success,
+    Failure
+  });
+}
+
+void PutAzureDataLakeStorage::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
+  connection_string_ = getConnectionStringFromControllerService(context);
+  if (connection_string_.empty()) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Azure Storage Credentials Service property missing or invalid");
+  }
+
+  conflict_resolution_strategy_ = FileExistsResolutionStrategy::parse(
+    utils::parsePropertyWithAllowableValuesOrThrow(*context, ConflictResolutionStrategy.getName(), FileExistsResolutionStrategy::values()).c_str());
+}
+
+std::optional<storage::PutAzureDataLakeStorageParameters> PutAzureDataLakeStorage::buildUploadParameters(
+    const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::FlowFile>& flow_file) {
+  storage::PutAzureDataLakeStorageParameters params;
+  params.connection_string = connection_string_;
+  params.replace_file = conflict_resolution_strategy_ == FileExistsResolutionStrategy::REPLACE_FILE;
+
+  if (!context->getProperty(FilesystemName, params.file_system_name, flow_file) || params.file_system_name.empty()) {
+    logger_->log_error("Filesystem Name '%s' is invalid or empty!", params.file_system_name);
+    return std::nullopt;
+  }
+
+  context->getProperty(DirectoryName, params.directory_name, flow_file);
+
+  context->getProperty(FileName, params.filename, flow_file);
+  if (params.filename.empty() && (!flow_file->getAttribute("filename", params.filename) || params.filename.empty())) {
+    logger_->log_error("No File Name is set and default object key 'filename' attribute could not be found!");
+    return std::nullopt;
+  }
+
+  return params;
+}
+
+void PutAzureDataLakeStorage::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  logger_->log_debug("PutAzureDataLakeStorage onTrigger");
+  std::shared_ptr<core::FlowFile> flow_file = session->get();
+  if (!flow_file) {
+    context->yield();
+    return;
+  }
+
+  const auto params = buildUploadParameters(context, flow_file);
+  if (!params) {
+    session->transfer(flow_file, Failure);
+    return;
+  }
+
+  PutAzureDataLakeStorage::ReadCallback callback(flow_file->getSize(), azure_data_lake_storage_, *params, logger_);
+  session->read(flow_file, &callback);
+  auto result = callback.getResult();
+  if (result.result_code == storage::UploadResultCode::FILE_ALREADY_EXISTS) {
+    gsl_Expects(conflict_resolution_strategy_ != FileExistsResolutionStrategy::REPLACE_FILE);
+    if (conflict_resolution_strategy_ == FileExistsResolutionStrategy::FAIL_FLOW) {
+      logger_->log_error("Failed to upload file '%s/%s' to filesystem '%s' on Azure Data Lake storage because file already exists",
+        params->directory_name, params->filename, params->file_system_name);
+      session->transfer(flow_file, Failure);
+      return;
+    } else if (conflict_resolution_strategy_ == FileExistsResolutionStrategy::IGNORE_REQUEST) {
+      logger_->log_debug("Upload of file '%s/%s' was ignored because it already exits in filesystem '%s' on Azure Data Lake Storage",
+        params->directory_name, params->filename, params->file_system_name);
+      session->transfer(flow_file, Success);
+      return;
+    }
+  } else if (result.result_code == storage::UploadResultCode::FAILURE) {
+    logger_->log_error("Failed to upload file '%s/%s' to filesystem '%s' on Azure Data Lake storage", params->directory_name, params->filename, params->file_system_name);
+    session->transfer(flow_file, Failure);
+  } else {
+    session->putAttribute(flow_file, "azure.filesystem", params->file_system_name);
+    session->putAttribute(flow_file, "azure.directory", params->directory_name);
+    session->putAttribute(flow_file, "azure.filename", params->filename);
+    session->putAttribute(flow_file, "azure.primaryUri", result.primary_uri);
+    session->putAttribute(flow_file, "azure.length", std::to_string(result.length));

Review comment:
       Is it useful to have this `length` field in the `result` struct?  Can its value be different from `flow_file->getSize()`?




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