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 2020/09/08 14:19:07 UTC

[GitHub] [nifi-minifi-cpp] adamdebreceni opened a new pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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


   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.

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// FlowFile IO Callback functions for input and output
+// throw exception for error
+class InputStreamCallback {
+ public:
+  virtual ~InputStreamCallback() = default;
+  // virtual void process(std::ifstream *stream) = 0;
+
+  virtual int64_t process(std::shared_ptr<io::BaseStream> stream) = 0;
+};
+class OutputStreamCallback {
+ public:
+  virtual ~OutputStreamCallback() = default;
+  virtual int64_t process(std::shared_ptr<io::BaseStream> stream) = 0;

Review comment:
       changed it, I had to touch quite a few files as well




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



[GitHub] [nifi-minifi-cpp] szaszm closed pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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


   


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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.h
##########
@@ -138,67 +107,87 @@ class BinaryConcatenationMerge : public MergeBin {
       return ret;
     }
   };
+
+ private:
+  std::string header_;
+  std::string footer_;
+  std::string demarcator_;
 };
 
 
 // Archive Class
 class ArchiveMerge {
-public:
-  // Nest Callback Class for read stream
-  class ReadCallback: public InputStreamCallback {
+ public:
+ class ArchiveWriter : public io::OutputStream {

Review comment:
       The indentation went off by one space on this line.
   ```suggestion
     class ArchiveWriter : public io::OutputStream {
   ```

##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 <utility>
+#include "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// FlowFile IO Callback functions for input and output
+// throw exception for error
+class InputStreamCallback {
+ public:
+  virtual ~InputStreamCallback() = default;
+
+  virtual int64_t process(const std::shared_ptr<io::BaseStream>& stream) = 0;
+};
+class OutputStreamCallback {
+ public:
+  virtual ~OutputStreamCallback() = default;
+  virtual int64_t process(const std::shared_ptr<io::BaseStream>& stream) = 0;
+};
+
+namespace internal {
+
+inline int64_t pipe(const std::shared_ptr<io::BaseStream>& src, const std::shared_ptr<io::BaseStream>& dst) {
+  uint8_t buffer[4096U];
+  int64_t totalTransferred = 0;
+  while (true) {
+    int readRet = src->read(buffer, sizeof(buffer));
+    if (readRet < 0) {
+      return readRet;
+    }
+    if (readRet == 0) {
+      break;
+    }
+    int remaining = readRet;
+    int transferred = 0;
+    while (remaining > 0) {
+      int writeRet = dst->write(buffer + transferred, remaining);
+      // TODO(adebreceni):
+      //   write might return 0, e.g. in case of a congested server
+      //   what should we return then?
+      //     - the number of bytes read or
+      //     - the number of bytes wrote

Review comment:
       I don't think we can give strong guarantees with streams, unless we make them transactional, which would take significant effort. In case of a ZipProcessor, I'd return the number of input bytes that successfully made it through the pipeline. If this is not possible, which I think is likely, then just return Failure.
   
   What would be the meaning of `Retry`? Otherwise I'm also fine with the [`Failure`, `Success`] range.




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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I'm okay with any options of 2-5.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I'm okay with any options of 2-5.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I'm okay with any options of 2-5.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I'm okay with any options of 2-5.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I'm okay with any options of 2-5.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I'm okay with any options of 2-5.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I'm okay with any options of 2-5.




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that (third-party) callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: libminifi/CMakeLists.txt
##########
@@ -89,7 +89,7 @@ if (NOT OPENSSL_OFF)
 	set(TLS_SOURCES "src/io/tls/*.cpp")
 endif()
 
-file(GLOB SOURCES  "src/utils/file/*.cpp" "src/sitetosite/*.cpp"  "src/core/logging/*.cpp"  "src/core/state/*.cpp" "src/core/state/nodes/*.cpp" "src/c2/protocols/*.cpp" "src/c2/triggers/*.cpp" "src/c2/*.cpp" "src/io/*.cpp" ${SOCKET_SOURCES} ${TLS_SOURCES} "src/core/controller/*.cpp" "src/controllers/*.cpp" "src/controllers/keyvalue/*.cpp" "src/core/*.cpp"  "src/core/repository/*.cpp" "src/core/yaml/*.cpp" "src/core/reporting/*.cpp"  "src/provenance/*.cpp" "src/utils/*.cpp" "src/*.cpp")
+file(GLOB SOURCES  "src/utils/file/*.cpp" "src/sitetosite/*.cpp"  "src/core/logging/*.cpp"  "src/core/state/*.cpp" "src/core/state/nodes/*.cpp" "src/c2/protocols/*.cpp" "src/c2/triggers/*.cpp" "src/c2/*.cpp" "src/io/*.cpp" ${SOCKET_SOURCES} ${TLS_SOURCES} "src/core/controller/*.cpp" "src/controllers/*.cpp" "src/controllers/keyvalue/*.cpp" "src/core/*.cpp"  "src/core/repository/*.cpp" "src/core/yaml/*.cpp" "src/core/reporting/*.cpp" "src/serialization/*.cpp" "src/provenance/*.cpp" "src/utils/*.cpp" "src/*.cpp")

Review comment:
       Despite it being common in the codebase, I recommend against against using globs to match source files. Rationale: see note at https://cmake.org/cmake/help/latest/command/file.html#glob
   ```suggestion
   file(GLOB SOURCES  "src/utils/file/*.cpp" "src/sitetosite/*.cpp"  "src/core/logging/*.cpp"  "src/core/state/*.cpp" "src/core/state/nodes/*.cpp" "src/c2/protocols/*.cpp" "src/c2/triggers/*.cpp" "src/c2/*.cpp" "src/io/*.cpp" ${SOCKET_SOURCES} ${TLS_SOURCES} "src/core/controller/*.cpp" "src/controllers/*.cpp" "src/controllers/keyvalue/*.cpp" "src/core/*.cpp"  "src/core/repository/*.cpp" "src/core/yaml/*.cpp" "src/core/reporting/*.cpp" "src/provenance/*.cpp" "src/utils/*.cpp" "src/*.cpp")
   list(APPEND SOURCES "src/serialization/FlowFileV3Serializer.cpp" "src/serialization/PayloadSerializer.cpp")
   ```

##########
File path: libminifi/test/archive-tests/MergeFileTests.cpp
##########
@@ -973,3 +975,95 @@ TEST_CASE("Test Merge File Attributes Keeping All Unique Attributes", "[testMerg
 
   LogTestController::getInstance().reset();
 }
+
+void writeString(const std::string& str, const std::shared_ptr<minifi::io::BaseStream>& out) {
+  out->write(const_cast<uint8_t*>(reinterpret_cast<const uint8_t*>(str.data())), str.length());
+}
+
+TEST_CASE("FlowFile serialization", "[testFlowFileSerialization]") {
+  MergeTestController testController;
+  auto context = testController.context;
+  auto processor = testController.processor;
+  auto input = testController.input;
+  auto output = testController.output;
+
+  const std::string header = "BEGIN{";
+  const std::string footer = "}END";
+  const std::string demarcator = "_";
+
+  core::ProcessSession session(context);
+
+  minifi::PayloadSerializer payloadSerializer([&] (const std::shared_ptr<core::FlowFile>& ff, minifi::InputStreamCallback* cb) {
+    return session.read(ff, cb);
+  });
+  minifi::FlowFileV3Serializer ffV3Serializer([&] (const std::shared_ptr<core::FlowFile>& ff, minifi::InputStreamCallback* cb) {
+    return session.read(ff, cb);
+  });
+
+  minifi::FlowFileSerializer* usedSerializer;
+
+  std::vector<std::shared_ptr<core::FlowFile>> files;
+
+  for (const auto& content : std::vector<std::string>{"first ff content", "second ff content", "some other data"}) {
+    minifi::io::DataStream contentStream{reinterpret_cast<const uint8_t*>(content.data()), static_cast<int>(content.length())};
+    auto ff = session.create();
+    session.importFrom(contentStream, ff);
+    ff->addAttribute("one", "banana");
+    ff->addAttribute("two", "seven");

Review comment:
       1. 🍌
   2. 7️⃣

##########
File path: libminifi/include/FlowFileRecord.h
##########
@@ -39,6 +39,7 @@
 #include "ResourceClaim.h"
 #include "Connection.h"
 #include "io/OutputStream.h"
+#include "io/StreamPipe.h"

Review comment:
       Is this a leftover?

##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 <utility>
+#include "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+

Review comment:
       I'd expect these to be in the `io` namespace like everything else in the io directory.

##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 <utility>
+#include "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// FlowFile IO Callback functions for input and output
+// throw exception for error
+class InputStreamCallback {
+ public:
+  virtual ~InputStreamCallback() = default;
+
+  virtual int64_t process(const std::shared_ptr<io::BaseStream>& stream) = 0;
+};
+class OutputStreamCallback {
+ public:
+  virtual ~OutputStreamCallback() = default;
+  virtual int64_t process(const std::shared_ptr<io::BaseStream>& stream) = 0;
+};
+
+namespace internal {
+
+inline int64_t pipe(const std::shared_ptr<io::BaseStream>& src, const std::shared_ptr<io::BaseStream>& dst) {
+  uint8_t buffer[4096U];
+  int64_t totalTransferred = 0;
+  while (true) {
+    int readRet = src->read(buffer, sizeof(buffer));
+    if (readRet < 0) {
+      return readRet;
+    }
+    if (readRet == 0) {
+      break;
+    }
+    int remaining = readRet;
+    int transferred = 0;
+    while (remaining > 0) {
+      int writeRet = dst->write(buffer + transferred, remaining);
+      // TODO(adebreceni):
+      //   write might return 0, e.g. in case of a congested server
+      //   what should we return then?
+      //     - the number of bytes read or
+      //     - the number of bytes wrote

Review comment:
       The number of bytes wrote. The copy is successful if the data was successfully read from the source and written to the destination stream. So the number of successfully copied bytes is the number of bytes written to the destination stream. This approach also leaves some room for a robust user that can restore the data that was read from the source but not copied before retrying.

##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       NiFi uses the merge format property for this purpose, with the new serializer bound to the equivalent of `BinaryConcatenationMerge`. Why is it separate here?




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       imo the idea of WHAT we want to merge, and HOW we want to merge are orthogonal, here `Flow File Serializer` answers the WHAT and `Merge Format` answers the HOW




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   based on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       I agree on the implementation level, but on the interface, compatibility with NiFi offers a greater benefit than separation of concerns IMO. Now you can create a tar of FFv3-serialized flow files, but what would you do with it if NiFi can't handle it?
   
   Sometimes it makes sense to allow mixing and matching of mergers and serializers, but in many cases, a merge format restricts the available serializations or the other way around, so coupling these on the interface could make for a safer user (yaml) interface.
   
   Separation of merge and flow file serialization in NiFi implementation: https://github.com/apache/nifi/blob/main/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java#L448




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



[GitHub] [nifi-minifi-cpp] szaszm commented on pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

Posted by GitBox <gi...@apache.org>.
szaszm commented on pull request #900:
URL: https://github.com/apache/nifi-minifi-cpp/pull/900#issuecomment-714642657


   I did some pre-merge manual testing of FFv3. For a while everything worked fine and just as I removed the GenerateFlowFile -> MergeContent connection, NiFi started spewing out these exceptions:
   ```
   2020-10-22 19:16:28,318 ERROR [Timer-Driven Process Thread-6] o.a.n.processors.standard.UnpackContent UnpackContent[id=514003df-0175-1000-9df2-b9696fdbaa96] Unable to unpack StandardFlowFileRecord
   [uuid=6803473c-2f9c-443f-900a-1f58f8721af2,claim=,offset=0,name=1603386311828679904,size=0] due to org.apache.nifi.processor.exception.ProcessException: IOException thrown from UnpackContent[id=51
   4003df-0175-1000-9df2-b9696fdbaa96]: java.io.IOException: Not in FlowFile-v3 format; routing to failure: org.apache.nifi.processor.exception.ProcessException: IOException thrown from UnpackContent
   [id=514003df-0175-1000-9df2-b9696fdbaa96]: java.io.IOException: Not in FlowFile-v3 format
   org.apache.nifi.processor.exception.ProcessException: IOException thrown from UnpackContent[id=514003df-0175-1000-9df2-b9696fdbaa96]: java.io.IOException: Not in FlowFile-v3 format
   at org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:2770)
   at org.apache.nifi.processors.standard.UnpackContent$FlowFileStreamUnpacker$1.process(UnpackContent.java:441)
   [...]
   ```
   Apparently 0 sized flow files with the mime.type of `application/flowfile-v3` appeared in NiFi.
   If this is expected, then fine, otherwise could you check it?


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



[GitHub] [nifi-minifi-cpp] lordgamez commented on pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

Posted by GitBox <gi...@apache.org>.
lordgamez commented on pull request #900:
URL: https://github.com/apache/nifi-minifi-cpp/pull/900#issuecomment-690927794






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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Understood. I'd go for 3.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way, in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way (i.e. I'm okay with all), in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Understood. I'd go for 3.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way, in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way (i.e. I'm okay with all), in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Understood. I'd go for 3.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way, in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way (i.e. I'm okay with all), in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Understood. I'd go for 3.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way, in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way (i.e. I'm okay with all), in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Understood. I'd go for 3.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way, in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way (i.e. I'm okay with all), in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Understood. I'd go for 3.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way, in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way (i.e. I'm okay with all), in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one it should be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// FlowFile IO Callback functions for input and output
+// throw exception for error
+class InputStreamCallback {
+ public:
+  virtual ~InputStreamCallback() = default;
+  // virtual void process(std::ifstream *stream) = 0;

Review comment:
       removed

##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -147,6 +152,9 @@ void MergeContent::onSchedule(core::ProcessContext *context, core::ProcessSessio
   if (context->getProperty(AttributeStrategy.getName(), value) && !value.empty()) {
     attributeStrategy_ = value;
   }
+  if (context->getProperty(FlowFileSerializer.getName(), value) && !value.empty()) {
+    flowFileSerializer_ = value;
+  }
 
   validatePropertyOptions();

Review comment:
       done

##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -147,6 +152,9 @@ void MergeContent::onSchedule(core::ProcessContext *context, core::ProcessSessio
   if (context->getProperty(AttributeStrategy.getName(), value) && !value.empty()) {
     attributeStrategy_ = value;
   }
+  if (context->getProperty(FlowFileSerializer.getName(), value) && !value.empty()) {

Review comment:
       done




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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I'm okay with any options of 2-5.




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



[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       The current implementation makes sense in a way, but I also think that compatibility with NiFi would bring more benefits, so please merge in case it's not a big change. 




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



[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       The current implementation makes sense in a way, but I also think that compatibility with NiFi would bring more benefits, so please merge in case it's not a big change. 




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       I agree on the implementation level, but on the interface, compatibility with NiFi offers a greater benefit than separation of concerns IMO. Now you can create a tar of FFv3-serialized flow files, but what would you do with it if NiFi can't handle it?
   
   Separation of merge and flow file serialization in NiFi implementation: https://github.com/apache/nifi/blob/main/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java#L448




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 <utility>
+#include "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+

Review comment:
       as expressed above, this move is a tangential change and it would be better if the namespace change gets its own ticket [here](https://issues.apache.org/jira/browse/MINIFICPP-1386)




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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -147,6 +152,9 @@ void MergeContent::onSchedule(core::ProcessContext *context, core::ProcessSessio
   if (context->getProperty(AttributeStrategy.getName(), value) && !value.empty()) {
     attributeStrategy_ = value;
   }
+  if (context->getProperty(FlowFileSerializer.getName(), value) && !value.empty()) {
+    flowFileSerializer_ = value;
+  }
 
   validatePropertyOptions();

Review comment:
       You could add FlowFileSerializer property to the validation as well.

##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -147,6 +152,9 @@ void MergeContent::onSchedule(core::ProcessContext *context, core::ProcessSessio
   if (context->getProperty(AttributeStrategy.getName(), value) && !value.empty()) {
     attributeStrategy_ = value;
   }
+  if (context->getProperty(FlowFileSerializer.getName(), value) && !value.empty()) {

Review comment:
       The conditional is not needed for strings, I think all these ifs could be removed.

##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// FlowFile IO Callback functions for input and output
+// throw exception for error
+class InputStreamCallback {
+ public:
+  virtual ~InputStreamCallback() = default;
+  // virtual void process(std::ifstream *stream) = 0;

Review comment:
       Is this commented code still needed?

##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// FlowFile IO Callback functions for input and output
+// throw exception for error
+class InputStreamCallback {
+ public:
+  virtual ~InputStreamCallback() = default;
+  // virtual void process(std::ifstream *stream) = 0;
+
+  virtual int64_t process(std::shared_ptr<io::BaseStream> stream) = 0;
+};
+class OutputStreamCallback {
+ public:
+  virtual ~OutputStreamCallback() = default;
+  virtual int64_t process(std::shared_ptr<io::BaseStream> stream) = 0;

Review comment:
       Could these parameters be `const std::shared_ptr<io::BaseStream>&` instead? We may not need to increase the refcount here.




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 <utility>
+#include "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// FlowFile IO Callback functions for input and output
+// throw exception for error
+class InputStreamCallback {
+ public:
+  virtual ~InputStreamCallback() = default;
+
+  virtual int64_t process(const std::shared_ptr<io::BaseStream>& stream) = 0;
+};
+class OutputStreamCallback {
+ public:
+  virtual ~OutputStreamCallback() = default;
+  virtual int64_t process(const std::shared_ptr<io::BaseStream>& stream) = 0;
+};
+
+namespace internal {
+
+inline int64_t pipe(const std::shared_ptr<io::BaseStream>& src, const std::shared_ptr<io::BaseStream>& dst) {
+  uint8_t buffer[4096U];
+  int64_t totalTransferred = 0;
+  while (true) {
+    int readRet = src->read(buffer, sizeof(buffer));
+    if (readRet < 0) {
+      return readRet;
+    }
+    if (readRet == 0) {
+      break;
+    }
+    int remaining = readRet;
+    int transferred = 0;
+    while (remaining > 0) {
+      int writeRet = dst->write(buffer + transferred, remaining);
+      // TODO(adebreceni):
+      //   write might return 0, e.g. in case of a congested server
+      //   what should we return then?
+      //     - the number of bytes read or
+      //     - the number of bytes wrote

Review comment:
       given where `internal::pipe` is used I would generalize the question to "what should `*StreamCallback::process` return?"
   
   e.g. if `ZipProcessor::process` returns `10` what could that mean? should we retry `process` hoping that the (implicitly captured) `output` is able to accept the data, or does that mean that we transferred all the data? (if it returns readByteCount we are no closer to the answer as the bytes might have been read but not successfully transferred)
   
   currently I think returning anything other than [`Failure`, `Success`, `Retry`?] is of little utility as most such callbacks cannot handle retry




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream&>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one it should be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that (third-party) callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream&>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one it should be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that (third-party) callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream&>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one it should be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that (third-party) callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream&>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one it should be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that (third-party) callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream&>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one it should be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that (third-party) callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream&>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one it should be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that (third-party) callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream&>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one it should be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that (third-party) callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream&>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one it should be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that (third-party) callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream&>`




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



[GitHub] [nifi-minifi-cpp] szaszm edited a comment on pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

Posted by GitBox <gi...@apache.org>.
szaszm edited a comment on pull request #900:
URL: https://github.com/apache/nifi-minifi-cpp/pull/900#issuecomment-690302679


   There are a bunch of changes changing "pass shared_ptr by value" to "pass shared_ptr by const ref". Why?
   If we intend to share ownership, it should be passed by value. If we just want to use the underlying object, passing a reference or raw pointer to that is probably better.
   
   Related very incomplete guideline: https://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#Rr-sharedptrparam-const


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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: libminifi/CMakeLists.txt
##########
@@ -89,7 +89,7 @@ if (NOT OPENSSL_OFF)
 	set(TLS_SOURCES "src/io/tls/*.cpp")
 endif()
 
-file(GLOB SOURCES  "src/utils/file/*.cpp" "src/sitetosite/*.cpp"  "src/core/logging/*.cpp"  "src/core/state/*.cpp" "src/core/state/nodes/*.cpp" "src/c2/protocols/*.cpp" "src/c2/triggers/*.cpp" "src/c2/*.cpp" "src/io/*.cpp" ${SOCKET_SOURCES} ${TLS_SOURCES} "src/core/controller/*.cpp" "src/controllers/*.cpp" "src/controllers/keyvalue/*.cpp" "src/core/*.cpp"  "src/core/repository/*.cpp" "src/core/yaml/*.cpp" "src/core/reporting/*.cpp"  "src/provenance/*.cpp" "src/utils/*.cpp" "src/*.cpp")
+file(GLOB SOURCES  "src/utils/file/*.cpp" "src/sitetosite/*.cpp"  "src/core/logging/*.cpp"  "src/core/state/*.cpp" "src/core/state/nodes/*.cpp" "src/c2/protocols/*.cpp" "src/c2/triggers/*.cpp" "src/c2/*.cpp" "src/io/*.cpp" ${SOCKET_SOURCES} ${TLS_SOURCES} "src/core/controller/*.cpp" "src/controllers/*.cpp" "src/controllers/keyvalue/*.cpp" "src/core/*.cpp"  "src/core/repository/*.cpp" "src/core/yaml/*.cpp" "src/core/reporting/*.cpp" "src/serialization/*.cpp" "src/provenance/*.cpp" "src/utils/*.cpp" "src/*.cpp")

Review comment:
       agree with the rationale, although it is rare that we add new files, in case we go down the "explicit" path I would do it a separate PR and for all source files, imo it would be strange if we used it for some files but not for others




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       should be a straightforward change, this being an interface level decision I would also like to ask @arpadboda to confirm dropping the separate `Flow File Serializer` property and merging it into `Merge Format`

##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 <utility>
+#include "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+

Review comment:
       as expressed above, this is a tangential change and it would be better if the namespace change gets its own ticket [here](https://issues.apache.org/jira/browse/MINIFICPP-1386)

##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 <utility>
+#include "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+

Review comment:
       as expressed above, this move is a tangential change and it would be better if the namespace change gets its own ticket [here](https://issues.apache.org/jira/browse/MINIFICPP-1386)

##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       the serializer has been integrated into the Merge Format (using the NiFi-equivalent `FlowFile Stream, v3` name)




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 <utility>
+#include "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+

Review comment:
       as expressed above, this is a tangential change and it would be better if the namespace change gets its own ticket [here](https://issues.apache.org/jira/browse/MINIFICPP-1386)




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way, in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       I agree on the implementation level, but on the interface, compatibility with NiFi offers a greater benefit than separation of concerns IMO. Now you can create a tar of FFv3-serialized flow files, but what would you do with it if NiFi can't handle it?




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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I'm okay with any options of 2-5.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I'm okay with any options of 2-5.




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way (i.e. I'm okay with all), in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.h
##########
@@ -138,67 +107,87 @@ class BinaryConcatenationMerge : public MergeBin {
       return ret;
     }
   };
+
+ private:
+  std::string header_;
+  std::string footer_;
+  std::string demarcator_;
 };
 
 
 // Archive Class
 class ArchiveMerge {
-public:
-  // Nest Callback Class for read stream
-  class ReadCallback: public InputStreamCallback {
+ public:
+ class ArchiveWriter : public io::OutputStream {

Review comment:
       The indentation went off by one space on this line.
   ```suggestion
     class ArchiveWriter : public io::OutputStream {
   ```




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on pull request #900:
URL: https://github.com/apache/nifi-minifi-cpp/pull/900#issuecomment-717170830


   nice finding @szaszm! as previously discussed, it seems to be unrelated to this PR, as the same behavior is also observable on the current main, the cause of this issue is that for some reason the `FlowFileRepository::prune_stored_flowfiles` gets executed multiple times, reenqueuing an already deleted flowFile (we load it from the checkpoint we create beforehand where the deletion did not happen), but by then the content resource is deleted, yielding and empty resource (which is an invalid FFv3 format)
   
   the created jira ticket is [here](https://issues.apache.org/jira/browse/MINIFICPP-1399)


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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Understood. I'd go for 3.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way, in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way (i.e. I'm okay with all), in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Understood. I'd go for 3.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way, in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way (i.e. I'm okay with all), in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Understood. I'd go for 3.

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way, in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Note: I didn't really review this PR, just briefly looked at the diff and shared my thoughts about it. Me generally preferring 3. is because it's normally not nullable and doesn't enforce any lifetime semantics of the object, which is normally the ideal when we only care about the object. When storing as a data member, I prefer 5. or 4, because they are mutable and sometimes one needs nullable members to be able to implement move semantics.
   
   With that said, I can accept any other way (i.e. I'm okay with all), in any case, especially given good reasoning (e.g. the proliferation of shared_ptr throughout the codebase).




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: libminifi/include/FlowFileRecord.h
##########
@@ -39,6 +39,7 @@
 #include "ResourceClaim.h"
 #include "Connection.h"
 #include "io/OutputStream.h"
+#include "io/StreamPipe.h"

Review comment:
       intentional as I didn't want to add the includes, as this change of moving `*StreamCallback` to a separate file is already tangential to the issue, I would like to do it in a separate PR, likewise with moving it to the `io` namespace
   [the ticket](https://issues.apache.org/jira/browse/MINIFICPP-1386)




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: libminifi/include/io/StreamPipe.h
##########
@@ -0,0 +1,110 @@
+/**
+ * @file FlowFileRecord.h
+ * Flow file record 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 <utility>
+#include "BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// FlowFile IO Callback functions for input and output
+// throw exception for error
+class InputStreamCallback {
+ public:
+  virtual ~InputStreamCallback() = default;
+
+  virtual int64_t process(const std::shared_ptr<io::BaseStream>& stream) = 0;
+};
+class OutputStreamCallback {
+ public:
+  virtual ~OutputStreamCallback() = default;
+  virtual int64_t process(const std::shared_ptr<io::BaseStream>& stream) = 0;
+};
+
+namespace internal {
+
+inline int64_t pipe(const std::shared_ptr<io::BaseStream>& src, const std::shared_ptr<io::BaseStream>& dst) {
+  uint8_t buffer[4096U];
+  int64_t totalTransferred = 0;
+  while (true) {
+    int readRet = src->read(buffer, sizeof(buffer));
+    if (readRet < 0) {
+      return readRet;
+    }
+    if (readRet == 0) {
+      break;
+    }
+    int remaining = readRet;
+    int transferred = 0;
+    while (remaining > 0) {
+      int writeRet = dst->write(buffer + transferred, remaining);
+      // TODO(adebreceni):
+      //   write might return 0, e.g. in case of a congested server
+      //   what should we return then?
+      //     - the number of bytes read or
+      //     - the number of bytes wrote

Review comment:
       I don't think we can give strong guarantees with streams, unless we make them transactional, which would take significant effort. In case of a ZipProcessor, I'd return the number of input bytes that successfully made it through the pipeline. If this is not possible, which I think is likely, then just return Failure.
   
   What would be the meaning of `Retry`? Otherwise I'm also fine with the [`Failure`, `Success`] range.




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.h
##########
@@ -138,67 +107,87 @@ class BinaryConcatenationMerge : public MergeBin {
       return ret;
     }
   };
+
+ private:
+  std::string header_;
+  std::string footer_;
+  std::string demarcator_;
 };
 
 
 // Archive Class
 class ArchiveMerge {
-public:
-  // Nest Callback Class for read stream
-  class ReadCallback: public InputStreamCallback {
+ public:
+ class ArchiveWriter : public io::OutputStream {

Review comment:
       c5021a9




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       the serializer has been integrated into the Merge Format (using the NiFi-equivalent `FlowFile Stream, v3` name)




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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream&>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one it should be:
   
   1. `shared_ptr<BaseStream>`
   2. `const shared_ptr<BaseStream>&`
   3. `BaseStream&`
   4. `BaseStream*`
   5. `gsl::not_null<BaseStream*>`

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that (third-party) callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR

##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       as it turns out there are a number of implementations that store the stream, so that callbacks may access it, e.g. `CompressContent::WriteCallback`, `JniByteInputStream`, `FocusArchiveEntry::ReadCallback`, `ArchiveMerge::WriteCallback`, `UnfocusArchiveEntry::WriteCallback`, `LuaInputStreamCallback`, `PyOutputStreamCallback`, moreover the hash functions in `HashContent` expect a `shared_ptr`
   
   base on this coupled with the diff explosion that would result from rewriting all the `stream->write` to `stream.write`, I propose we leave this be for now, and return to it in a later PR




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



[GitHub] [nifi-minifi-cpp] lordgamez commented on pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

Posted by GitBox <gi...@apache.org>.
lordgamez commented on pull request #900:
URL: https://github.com/apache/nifi-minifi-cpp/pull/900#issuecomment-688953715


   Please check the linter as well. It warns about some missing includes.


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



[GitHub] [nifi-minifi-cpp] lordgamez commented on pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

Posted by GitBox <gi...@apache.org>.
lordgamez commented on pull request #900:
URL: https://github.com/apache/nifi-minifi-cpp/pull/900#issuecomment-690927794


   > There are a bunch of changes changing "pass shared_ptr by value" to "pass shared_ptr by const ref". Why?
   > If we intend to share ownership, it should be passed by value. If we just want to use the underlying object, passing a reference or raw pointer to that is probably better.
   > 
   > Related very incomplete guideline: https://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#Rr-sharedptrparam-const
   
   In this case I suppose we only want to use the underlying object so we do not need to pass by value and we can use a raw pointer or a simple reference here. The only difference maybe is that if we decide later that we need to share the ownership with someone in the call chain we have the possibility. As I recall clang static analyzer usually suggests using shared_ptr const refs in these cases.


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



[GitHub] [nifi-minifi-cpp] szaszm edited a comment on pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

Posted by GitBox <gi...@apache.org>.
szaszm edited a comment on pull request #900:
URL: https://github.com/apache/nifi-minifi-cpp/pull/900#issuecomment-714642657


   I did some pre-merge manual testing of FFv3. For a while everything worked fine and just as I removed the GenerateFlowFile -> MergeContent connection, NiFi started spewing out these exceptions:
   ```
   2020-10-22 19:16:28,318 ERROR [Timer-Driven Process Thread-6] o.a.n.processors.standard.UnpackContent UnpackContent[id=514003df-0175-1000-9df2-b9696fdbaa96] Unable to unpack StandardFlowFileRecord
   [uuid=6803473c-2f9c-443f-900a-1f58f8721af2,claim=,offset=0,name=1603386311828679904,size=0] due to org.apache.nifi.processor.exception.ProcessException: IOException thrown from UnpackContent[id=51
   4003df-0175-1000-9df2-b9696fdbaa96]: java.io.IOException: Not in FlowFile-v3 format; routing to failure: org.apache.nifi.processor.exception.ProcessException: IOException thrown from UnpackContent
   [id=514003df-0175-1000-9df2-b9696fdbaa96]: java.io.IOException: Not in FlowFile-v3 format
   org.apache.nifi.processor.exception.ProcessException: IOException thrown from UnpackContent[id=514003df-0175-1000-9df2-b9696fdbaa96]: java.io.IOException: Not in FlowFile-v3 format
   at org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:2770)
   at org.apache.nifi.processors.standard.UnpackContent$FlowFileStreamUnpacker$1.process(UnpackContent.java:441)
   [...]
   ```
   Apparently 0 sized flow files with the mime.type of `application/flowfile-v3` appeared in NiFi after I stopped feeding flow files to MiNiFi C++ MergeContent.
   If this is expected, then fine, otherwise could you check it?


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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       I would say, that using `const shared_ptr<>&` is certainly an improvement, as @lordgamez said it does not force us to take ownership, and it is agreement with the proliferation of managed pointers in the rest of the system (so it lets us call into interfaces expecting a managed ptr), I don't have a strong opinion about it though, because I plan on removing the whole `*StreamCallback` stuff, so if you feel like it should be changed, decide which one should it be:
   
   1. shared_ptr<BaseStream>
   2. const shared_ptr<BaseStream>&
   3. BaseStream&
   4. BaseStream*
   5. gsl::not_null<BaseStream&>




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



[GitHub] [nifi-minifi-cpp] szaszm commented on pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

Posted by GitBox <gi...@apache.org>.
szaszm commented on pull request #900:
URL: https://github.com/apache/nifi-minifi-cpp/pull/900#issuecomment-690302679


   There are a bunch of changes changing "pass shared_ptr by value" to "pass shared_ptr by const ref". Why?
   If we intend to share ownership, it should be passed by value. If we just want to use the underlying object, passing a reference or raw pointer to that is probably better.


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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -72,10 +74,12 @@ core::Property MergeContent::AttributeStrategy(
                     "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
   ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
   ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+core::Property MergeContent::FlowFileSerializer(
+  core::PropertyBuilder::createProperty("Flow File Serializer")
+  ->withDescription("Determines how to flow files should be serialized before merging")
+  ->withAllowableValues<std::string>({merge_content_options::SERIALIZER_PAYLOAD, merge_content_options::SERIALIZER_FLOW_FILE_V3})
+  ->withDefaultValue(merge_content_options::SERIALIZER_PAYLOAD)->build());

Review comment:
       should be a straightforward change, this being an interface level decision I would also like to ask @arpadboda to confirm dropping the separate `Flow File Serializer` property and merging it into `Merge Format`




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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #900: MINIFICPP-1350 - Explicit serializer for MergeContent

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



##########
File path: extensions/bustache/ApplyTemplate.h
##########
@@ -61,7 +61,7 @@ class ApplyTemplate : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(std::shared_ptr<io::BaseStream> stream);
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream);

Review comment:
       Understood. I'd go for 3.




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