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 2022/01/24 10:25:41 UTC

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

lordgamez opened a new pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250


   https://issues.apache.org/jira/browse/MINIFICPP-1663
   
   -----------------------------------------------------
   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the LICENSE file?
   - [ ] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI results for build issues and submit an update to your PR as soon as possible.
   


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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250#discussion_r790822854



##########
File path: extensions/azure/processors/FetchAzureBlobStorage.cpp
##########
@@ -0,0 +1,118 @@
+/**
+ * @file FetchAzureBlobStorage.cpp
+ * FetchAzureBlobStorage 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 "FetchAzureBlobStorage.h"
+
+#include "core/Resource.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+const core::Property FetchAzureBlobStorage::RangeStart(
+  core::PropertyBuilder::createProperty("Range Start")
+    ->withDescription("The byte position at which to start reading from the blob. An empty value or a value of zero will start reading at the beginning of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property FetchAzureBlobStorage::RangeLength(
+  core::PropertyBuilder::createProperty("Range Length")
+    ->withDescription("The number of bytes to download from the blob, starting from the Range Start. "
+                      "An empty value or a value that extends beyond the end of the blob will read to the end of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Relationship FetchAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
+const core::Relationship FetchAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
+
+void FetchAzureBlobStorage::initialize() {
+  // Set the supported properties

Review comment:
       Updated in 81a8a876a579facf06fba6ffd8f5b81f57fbf1f1

##########
File path: extensions/azure/storage/AzureBlobStorageClient.h
##########
@@ -38,8 +39,27 @@ class AzureBlobStorageClient : public BlobStorageClient {
   Azure::Storage::Blobs::Models::UploadBlockBlobResult uploadBlob(const PutAzureBlobStorageParameters& params, gsl::span<const uint8_t> buffer) override;
   std::string getUrl(const PutAzureBlobStorageParameters& params) override;
   bool deleteBlob(const DeleteAzureBlobStorageParameters& params) override;
+  std::unique_ptr<io::InputStream> fetchBlob(const FetchAzureBlobStorageParameters& params) override;
 
  private:
+  class AzureBlobStorageInputStream : public io::InputStream {
+   public:

Review comment:
       Updated in 81a8a876a579facf06fba6ffd8f5b81f57fbf1f1




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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250#discussion_r790957820



##########
File path: extensions/azure/processors/FetchAzureBlobStorage.h
##########
@@ -0,0 +1,72 @@
+/**
+ * @file FetchAzureBlobStorage.h
+ * FetchAzureBlobStorage 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 <optional>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "core/Property.h"
+#include "AzureBlobStorageProcessorBase.h"
+#include "core/logging/LoggerConfiguration.h"
+
+template<typename T>
+class AzureBlobStorageTestsFixture;
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+class FetchAzureBlobStorage final : public AzureBlobStorageProcessorBase {
+ public:
+  // Supported Properties
+  EXTENSIONAPI static const core::Property RangeStart;
+  EXTENSIONAPI static const core::Property RangeLength;
+
+  // Supported Relationships
+  static const core::Relationship Failure;
+  static const core::Relationship Success;
+
+  explicit FetchAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
+    : FetchAzureBlobStorage(name, uuid, nullptr) {
+  }
+
+  void initialize() override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+
+ private:
+  friend class ::AzureBlobStorageTestsFixture<FetchAzureBlobStorage>;
+
+  core::annotation::Input getInputRequirement() const override {
+    return core::annotation::Input::INPUT_REQUIRED;
+  }
+
+  explicit FetchAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<storage::BlobStorageClient> blob_storage_client)
+    : AzureBlobStorageProcessorBase(name, uuid, core::logging::LoggerFactory<FetchAzureBlobStorage>::getLogger(), std::move(blob_storage_client)) {
+  }
+
+  std::optional<storage::FetchAzureBlobStorageParameters> buildFetchAzureBlobStorageParameters(
+    core::ProcessContext &context, const std::shared_ptr<core::FlowFile> &flow_file);
+
+  storage::OptionalDeletion optional_deletion_;

Review comment:
       Left it in there by mistake, removed it in 1180ddd237dae38a7f000c250be3ec3cc3c025d4

##########
File path: extensions/azure/processors/FetchAzureBlobStorage.h
##########
@@ -0,0 +1,72 @@
+/**
+ * @file FetchAzureBlobStorage.h
+ * FetchAzureBlobStorage 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 <optional>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "core/Property.h"
+#include "AzureBlobStorageProcessorBase.h"
+#include "core/logging/LoggerConfiguration.h"
+
+template<typename T>
+class AzureBlobStorageTestsFixture;
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+class FetchAzureBlobStorage final : public AzureBlobStorageProcessorBase {
+ public:
+  // Supported Properties
+  EXTENSIONAPI static const core::Property RangeStart;
+  EXTENSIONAPI static const core::Property RangeLength;
+
+  // Supported Relationships

Review comment:
       Removed in 1180ddd237dae38a7f000c250be3ec3cc3c025d4




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250#discussion_r790792930



##########
File path: extensions/azure/processors/FetchAzureBlobStorage.cpp
##########
@@ -0,0 +1,118 @@
+/**
+ * @file FetchAzureBlobStorage.cpp
+ * FetchAzureBlobStorage 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 "FetchAzureBlobStorage.h"
+
+#include "core/Resource.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+const core::Property FetchAzureBlobStorage::RangeStart(
+  core::PropertyBuilder::createProperty("Range Start")
+    ->withDescription("The byte position at which to start reading from the blob. An empty value or a value of zero will start reading at the beginning of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property FetchAzureBlobStorage::RangeLength(
+  core::PropertyBuilder::createProperty("Range Length")
+    ->withDescription("The number of bytes to download from the blob, starting from the Range Start. "
+                      "An empty value or a value that extends beyond the end of the blob will read to the end of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Relationship FetchAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
+const core::Relationship FetchAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
+
+void FetchAzureBlobStorage::initialize() {
+  // Set the supported properties

Review comment:
       Consider removing these comments, because they say the same as the code below them.

##########
File path: docker/test/integration/features/azure_storage.feature
##########
@@ -58,3 +58,21 @@ Feature: Sending data from MiNiFi-C++ to an Azure storage server
     And test blob "test" is created on Azure blob storage with a snapshot
 
     Then the blob and snapshot count becomes 1 in 30 seconds
+
+  Scenario: A MiNiFi instance can fetch a blob from Azure blob storage
+    Given a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And the "Keep Source File" property of the GetFile processor is set to "true"
+    And a file with the content "dummy" is present in "/tmp/input"
+    And a FetchAzureBlobStorage processor set up to communicate with an Azure blob storage
+    And the "Blob" property of the FetchAzureBlobStorage processor is set to "test"
+    And the "Range Start" property of the FetchAzureBlobStorage processor is set to "6"
+    And the "Range Length" property of the FetchAzureBlobStorage processor is set to "5"
+    And a PutFile processor with the "Directory" property set to "/tmp/output"
+    And the "success" relationship of the GetFile processor is connected to the FetchAzureBlobStorage
+    And the "success" relationship of the FetchAzureBlobStorage processor is connected to the PutFile
+    And an Azure storage server is set up
+
+    When all instances start up
+    And test blob "test" with the content "#test_data$123$#" is created on Azure blob storage
+
+    Then a flowfile with the content "data$" is placed in the monitored directory in less than 60 seconds

Review comment:
       Why isn't it `#test_data$123$#` in the assertion?

##########
File path: extensions/azure/processors/FetchAzureBlobStorage.h
##########
@@ -0,0 +1,100 @@
+/**
+ * @file FetchAzureBlobStorage.h
+ * FetchAzureBlobStorage 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 <optional>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "core/Property.h"
+#include "AzureBlobStorageProcessorBase.h"
+#include "core/logging/LoggerConfiguration.h"
+
+template<typename T>
+class AzureBlobStorageTestsFixture;
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+class FetchAzureBlobStorage final : public AzureBlobStorageProcessorBase {
+ public:
+  // Supported Properties
+  EXTENSIONAPI static const core::Property RangeStart;
+  EXTENSIONAPI static const core::Property RangeLength;
+
+  // Supported Relationships
+  static const core::Relationship Failure;
+  static const core::Relationship Success;
+
+  explicit FetchAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
+    : FetchAzureBlobStorage(name, uuid, nullptr) {
+  }
+
+  void initialize() override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+
+ private:
+  friend class ::AzureBlobStorageTestsFixture<FetchAzureBlobStorage>;
+
+  class WriteCallback : public OutputStreamCallback {
+   public:

Review comment:
       Consider hiding the callback class in the .cpp file, in an anonymous namespace.

##########
File path: extensions/azure/storage/AzureBlobStorageClient.h
##########
@@ -38,8 +39,27 @@ class AzureBlobStorageClient : public BlobStorageClient {
   Azure::Storage::Blobs::Models::UploadBlockBlobResult uploadBlob(const PutAzureBlobStorageParameters& params, gsl::span<const uint8_t> buffer) override;
   std::string getUrl(const PutAzureBlobStorageParameters& params) override;
   bool deleteBlob(const DeleteAzureBlobStorageParameters& params) override;
+  std::unique_ptr<io::InputStream> fetchBlob(const FetchAzureBlobStorageParameters& params) override;
 
  private:
+  class AzureBlobStorageInputStream : public io::InputStream {
+   public:

Review comment:
       Consider moving this to the .cpp file as an implementation detail, possibly in an anonymous namespace. It's only ever exposed through a base class reference.




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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250#discussion_r790809725



##########
File path: docker/test/integration/features/azure_storage.feature
##########
@@ -58,3 +58,21 @@ Feature: Sending data from MiNiFi-C++ to an Azure storage server
     And test blob "test" is created on Azure blob storage with a snapshot
 
     Then the blob and snapshot count becomes 1 in 30 seconds
+
+  Scenario: A MiNiFi instance can fetch a blob from Azure blob storage
+    Given a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And the "Keep Source File" property of the GetFile processor is set to "true"
+    And a file with the content "dummy" is present in "/tmp/input"
+    And a FetchAzureBlobStorage processor set up to communicate with an Azure blob storage
+    And the "Blob" property of the FetchAzureBlobStorage processor is set to "test"
+    And the "Range Start" property of the FetchAzureBlobStorage processor is set to "6"
+    And the "Range Length" property of the FetchAzureBlobStorage processor is set to "5"
+    And a PutFile processor with the "Directory" property set to "/tmp/output"
+    And the "success" relationship of the GetFile processor is connected to the FetchAzureBlobStorage
+    And the "success" relationship of the FetchAzureBlobStorage processor is connected to the PutFile
+    And an Azure storage server is set up
+
+    When all instances start up
+    And test blob "test" with the content "#test_data$123$#" is created on Azure blob storage
+
+    Then a flowfile with the content "data$" is placed in the monitored directory in less than 60 seconds

Review comment:
       The `Range Start` and `Range Length` properties are set to only return a substring which is `data$` in this case.




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250#discussion_r790864423



##########
File path: extensions/azure/storage/AzureBlobStorage.cpp
##########
@@ -74,4 +75,14 @@ bool AzureBlobStorage::deleteBlob(const DeleteAzureBlobStorageParameters& params
   }
 }
 
+std::optional<uint64_t> AzureBlobStorage::fetchBlob(const FetchAzureBlobStorageParameters& params, io::BaseStream& stream) {
+  try {
+    auto fetch_res = blob_storage_client_->fetchBlob(params);
+    return internal::pipe(fetch_res.get(), &stream);

Review comment:
       What do you think about returning an `InputStream` instead of piping its contents into an `OutputStream`? The caller could still do a piping if they wanted, but they may also be able to use the `InputStream` in a more efficient way.




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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250#discussion_r790955875



##########
File path: extensions/azure/processors/FetchAzureBlobStorage.cpp
##########
@@ -0,0 +1,146 @@
+/**
+ * @file FetchAzureBlobStorage.cpp
+ * FetchAzureBlobStorage 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 "FetchAzureBlobStorage.h"
+
+#include "core/Resource.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+const core::Property FetchAzureBlobStorage::RangeStart(
+  core::PropertyBuilder::createProperty("Range Start")
+    ->withDescription("The byte position at which to start reading from the blob. An empty value or a value of zero will start reading at the beginning of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property FetchAzureBlobStorage::RangeLength(
+  core::PropertyBuilder::createProperty("Range Length")
+    ->withDescription("The number of bytes to download from the blob, starting from the Range Start. "
+                      "An empty value or a value that extends beyond the end of the blob will read to the end of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Relationship FetchAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
+const core::Relationship FetchAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
+
+namespace {
+class WriteCallback : public OutputStreamCallback {
+ public:
+  WriteCallback(storage::AzureBlobStorage& azure_blob_storage, const storage::FetchAzureBlobStorageParameters& params, std::shared_ptr<core::logging::Logger> logger)
+    : azure_blob_storage_(azure_blob_storage),
+      params_(params),
+      logger_(std::move(logger)) {

Review comment:
       Removed it in 1180ddd237dae38a7f000c250be3ec3cc3c025d4




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250#discussion_r790856199



##########
File path: extensions/azure/processors/FetchAzureBlobStorage.cpp
##########
@@ -0,0 +1,146 @@
+/**
+ * @file FetchAzureBlobStorage.cpp
+ * FetchAzureBlobStorage 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 "FetchAzureBlobStorage.h"
+
+#include "core/Resource.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+const core::Property FetchAzureBlobStorage::RangeStart(
+  core::PropertyBuilder::createProperty("Range Start")
+    ->withDescription("The byte position at which to start reading from the blob. An empty value or a value of zero will start reading at the beginning of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property FetchAzureBlobStorage::RangeLength(
+  core::PropertyBuilder::createProperty("Range Length")
+    ->withDescription("The number of bytes to download from the blob, starting from the Range Start. "
+                      "An empty value or a value that extends beyond the end of the blob will read to the end of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Relationship FetchAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
+const core::Relationship FetchAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
+
+namespace {
+class WriteCallback : public OutputStreamCallback {
+ public:
+  WriteCallback(storage::AzureBlobStorage& azure_blob_storage, const storage::FetchAzureBlobStorageParameters& params, std::shared_ptr<core::logging::Logger> logger)
+    : azure_blob_storage_(azure_blob_storage),
+      params_(params),
+      logger_(std::move(logger)) {

Review comment:
       The logger is not used, consider removing it.

##########
File path: extensions/azure/processors/FetchAzureBlobStorage.h
##########
@@ -0,0 +1,72 @@
+/**
+ * @file FetchAzureBlobStorage.h
+ * FetchAzureBlobStorage 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 <optional>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "core/Property.h"
+#include "AzureBlobStorageProcessorBase.h"
+#include "core/logging/LoggerConfiguration.h"
+
+template<typename T>
+class AzureBlobStorageTestsFixture;
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+class FetchAzureBlobStorage final : public AzureBlobStorageProcessorBase {
+ public:
+  // Supported Properties
+  EXTENSIONAPI static const core::Property RangeStart;
+  EXTENSIONAPI static const core::Property RangeLength;
+
+  // Supported Relationships

Review comment:
       The comment about properties is misleading, because there are more supported properties. The one about relationships is just redundant, because it's obvious from the context. Consider removing them.

##########
File path: extensions/azure/processors/FetchAzureBlobStorage.cpp
##########
@@ -0,0 +1,146 @@
+/**
+ * @file FetchAzureBlobStorage.cpp
+ * FetchAzureBlobStorage 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 "FetchAzureBlobStorage.h"
+
+#include "core/Resource.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+const core::Property FetchAzureBlobStorage::RangeStart(
+  core::PropertyBuilder::createProperty("Range Start")
+    ->withDescription("The byte position at which to start reading from the blob. An empty value or a value of zero will start reading at the beginning of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property FetchAzureBlobStorage::RangeLength(
+  core::PropertyBuilder::createProperty("Range Length")
+    ->withDescription("The number of bytes to download from the blob, starting from the Range Start. "
+                      "An empty value or a value that extends beyond the end of the blob will read to the end of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Relationship FetchAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
+const core::Relationship FetchAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
+
+namespace {
+class WriteCallback : public OutputStreamCallback {
+ public:
+  WriteCallback(storage::AzureBlobStorage& azure_blob_storage, const storage::FetchAzureBlobStorageParameters& params, std::shared_ptr<core::logging::Logger> logger)
+    : azure_blob_storage_(azure_blob_storage),
+      params_(params),
+      logger_(std::move(logger)) {
+  }
+
+  int64_t process(const std::shared_ptr<io::BaseStream>& stream) override {
+    result_size_ = azure_blob_storage_.fetchBlob(params_, *stream);
+    if (!result_size_) {
+      return 0;
+    }
+
+    return gsl::narrow<int64_t>(*result_size_);
+  }
+
+  auto getResult() const {

Review comment:
       Clang-Tidy: Function 'getResult' should be marked [[nodiscard]]
   
   My thoughts: maybe it should be a public data member?

##########
File path: extensions/azure/processors/FetchAzureBlobStorage.h
##########
@@ -0,0 +1,72 @@
+/**
+ * @file FetchAzureBlobStorage.h
+ * FetchAzureBlobStorage 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 <optional>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "core/Property.h"
+#include "AzureBlobStorageProcessorBase.h"
+#include "core/logging/LoggerConfiguration.h"
+
+template<typename T>
+class AzureBlobStorageTestsFixture;
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+class FetchAzureBlobStorage final : public AzureBlobStorageProcessorBase {
+ public:
+  // Supported Properties
+  EXTENSIONAPI static const core::Property RangeStart;
+  EXTENSIONAPI static const core::Property RangeLength;
+
+  // Supported Relationships
+  static const core::Relationship Failure;
+  static const core::Relationship Success;
+
+  explicit FetchAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
+    : FetchAzureBlobStorage(name, uuid, nullptr) {
+  }
+
+  void initialize() override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+
+ private:
+  friend class ::AzureBlobStorageTestsFixture<FetchAzureBlobStorage>;
+
+  core::annotation::Input getInputRequirement() const override {
+    return core::annotation::Input::INPUT_REQUIRED;
+  }
+
+  explicit FetchAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<storage::BlobStorageClient> blob_storage_client)
+    : AzureBlobStorageProcessorBase(name, uuid, core::logging::LoggerFactory<FetchAzureBlobStorage>::getLogger(), std::move(blob_storage_client)) {
+  }
+
+  std::optional<storage::FetchAzureBlobStorageParameters> buildFetchAzureBlobStorageParameters(
+    core::ProcessContext &context, const std::shared_ptr<core::FlowFile> &flow_file);
+
+  storage::OptionalDeletion optional_deletion_;

Review comment:
       Unused data member. Did you mean to implement another feature?




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



[GitHub] [nifi-minifi-cpp] szaszm closed pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
szaszm closed pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250


   


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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250#discussion_r790957082



##########
File path: extensions/azure/processors/FetchAzureBlobStorage.cpp
##########
@@ -0,0 +1,146 @@
+/**
+ * @file FetchAzureBlobStorage.cpp
+ * FetchAzureBlobStorage 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 "FetchAzureBlobStorage.h"
+
+#include "core/Resource.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+const core::Property FetchAzureBlobStorage::RangeStart(
+  core::PropertyBuilder::createProperty("Range Start")
+    ->withDescription("The byte position at which to start reading from the blob. An empty value or a value of zero will start reading at the beginning of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property FetchAzureBlobStorage::RangeLength(
+  core::PropertyBuilder::createProperty("Range Length")
+    ->withDescription("The number of bytes to download from the blob, starting from the Range Start. "
+                      "An empty value or a value that extends beyond the end of the blob will read to the end of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Relationship FetchAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
+const core::Relationship FetchAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
+
+namespace {
+class WriteCallback : public OutputStreamCallback {
+ public:
+  WriteCallback(storage::AzureBlobStorage& azure_blob_storage, const storage::FetchAzureBlobStorageParameters& params, std::shared_ptr<core::logging::Logger> logger)
+    : azure_blob_storage_(azure_blob_storage),
+      params_(params),
+      logger_(std::move(logger)) {
+  }
+
+  int64_t process(const std::shared_ptr<io::BaseStream>& stream) override {
+    result_size_ = azure_blob_storage_.fetchBlob(params_, *stream);
+    if (!result_size_) {
+      return 0;
+    }
+
+    return gsl::narrow<int64_t>(*result_size_);
+  }
+
+  auto getResult() const {

Review comment:
       Updated in 1180ddd237dae38a7f000c250be3ec3cc3c025d4. I would keep the data member private to have it read only and not to be changed from the outside.




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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250#discussion_r790960789



##########
File path: extensions/azure/storage/AzureBlobStorage.cpp
##########
@@ -74,4 +75,14 @@ bool AzureBlobStorage::deleteBlob(const DeleteAzureBlobStorageParameters& params
   }
 }
 
+std::optional<uint64_t> AzureBlobStorage::fetchBlob(const FetchAzureBlobStorageParameters& params, io::BaseStream& stream) {
+  try {
+    auto fetch_res = blob_storage_client_->fetchBlob(params);
+    return internal::pipe(fetch_res.get(), &stream);

Review comment:
       I'm not sure if it's worth generalizing it at the moment as we only have one use case for this operation, and I don't think we will have any other in the future. If another use case arises I think it can be changed in that scope.




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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1250: MINIFICPP-1663 Create FetchAzureBlobStorage processor

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1250:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1250#discussion_r790822747



##########
File path: extensions/azure/processors/FetchAzureBlobStorage.h
##########
@@ -0,0 +1,100 @@
+/**
+ * @file FetchAzureBlobStorage.h
+ * FetchAzureBlobStorage 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 <optional>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "core/Property.h"
+#include "AzureBlobStorageProcessorBase.h"
+#include "core/logging/LoggerConfiguration.h"
+
+template<typename T>
+class AzureBlobStorageTestsFixture;
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+class FetchAzureBlobStorage final : public AzureBlobStorageProcessorBase {
+ public:
+  // Supported Properties
+  EXTENSIONAPI static const core::Property RangeStart;
+  EXTENSIONAPI static const core::Property RangeLength;
+
+  // Supported Relationships
+  static const core::Relationship Failure;
+  static const core::Relationship Success;
+
+  explicit FetchAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
+    : FetchAzureBlobStorage(name, uuid, nullptr) {
+  }
+
+  void initialize() override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+
+ private:
+  friend class ::AzureBlobStorageTestsFixture<FetchAzureBlobStorage>;
+
+  class WriteCallback : public OutputStreamCallback {
+   public:

Review comment:
       Updated in 81a8a876a579facf06fba6ffd8f5b81f57fbf1f1




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