You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ph...@apache.org on 2019/06/26 16:04:31 UTC

[nifi-minifi-cpp] branch master updated: MINIFICPP-933: Fix raw site to site with secure comms

This is an automated email from the ASF dual-hosted git repository.

phrocker pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git


The following commit(s) were added to refs/heads/master by this push:
     new 5a4d145  MINIFICPP-933: Fix raw site to site with secure comms
5a4d145 is described below

commit 5a4d145a29405e1ef136492aa991bb9599c0508b
Author: Marc Parisi <ph...@apache.org>
AuthorDate: Tue Jun 25 17:31:59 2019 -0400

    MINIFICPP-933: Fix raw site to site with secure comms
    
    MINIFICPP-933: Add tests to cover a few corner cases. Break out password to utility file
    
    MINIFICPP-933: remove sleeps and undo unintentional changes
    
    This closes #597.
    
    Signed-off-by: Marc Parisi <ph...@apache.org>
---
 .../standard-processors/tests/CMakeLists.txt       |  3 +
 .../tests/integration/SecureSocketGetTCPTest.cpp   | 17 ++--
 libminifi/include/controllers/SSLContextService.h  | 16 +---
 libminifi/include/io/tls/TLSSocket.h               | 12 ---
 libminifi/include/io/tls/TLSUtils.h                | 40 ++++++++++
 libminifi/src/controllers/SSLContextService.cpp    |  2 +-
 libminifi/src/io/tls/TLSSocket.cpp                 | 22 +++---
 libminifi/src/io/tls/TLSUtils.cpp                  | 56 ++++++++++++++
 .../test/resources/TestGetTCPSecureEmptyPass.yml   | 90 ++++++++++++++++++++++
 .../resources/TestGetTCPSecureWithFilePass.yml     | 90 ++++++++++++++++++++++
 .../test/resources/TestGetTCPSecureWithPass.yml    | 90 ++++++++++++++++++++++
 libminifi/test/resources/cn.pass                   |  5 ++
 .../test/resources/{cn.pass => empty.cn.pass}      |  0
 libminifi/test/resources/encrypted.cn.pass         |  5 ++
 libminifi/test/resources/encrypted.key.pem         | 30 ++++++++
 15 files changed, 432 insertions(+), 46 deletions(-)

diff --git a/extensions/standard-processors/tests/CMakeLists.txt b/extensions/standard-processors/tests/CMakeLists.txt
index f87ecf4..51d95ce 100644
--- a/extensions/standard-processors/tests/CMakeLists.txt
+++ b/extensions/standard-processors/tests/CMakeLists.txt
@@ -70,6 +70,9 @@ message("-- Finished building ${INT_TEST_COUNT} integration test file(s)...")
 add_test(NAME TestExecuteProcess COMMAND TestExecuteProcess )
 
 add_test(NAME SecureSocketGetTCPTest COMMAND SecureSocketGetTCPTest "${TEST_RESOURCES}/TestGetTCPSecure.yml"  "${TEST_RESOURCES}/")
+add_test(NAME SecureSocketGetTCPTestEmptyPass COMMAND SecureSocketGetTCPTest "${TEST_RESOURCES}/TestGetTCPSecureEmptyPass.yml"  "${TEST_RESOURCES}/")
+add_test(NAME SecureSocketGetTCPTestWithPassword COMMAND SecureSocketGetTCPTest "${TEST_RESOURCES}/TestGetTCPSecureWithPass.yml"  "${TEST_RESOURCES}/")
+add_test(NAME SecureSocketGetTCPTestWithPasswordFile COMMAND SecureSocketGetTCPTest "${TEST_RESOURCES}/TestGetTCPSecureWithFilePass.yml"  "${TEST_RESOURCES}/")
 
 add_test(NAME TailFileTest COMMAND TailFileTest "${TEST_RESOURCES}/TestTailFile.yml"  "${TEST_RESOURCES}/")
 
diff --git a/extensions/standard-processors/tests/integration/SecureSocketGetTCPTest.cpp b/extensions/standard-processors/tests/integration/SecureSocketGetTCPTest.cpp
index e3a0478..73b2005 100644
--- a/extensions/standard-processors/tests/integration/SecureSocketGetTCPTest.cpp
+++ b/extensions/standard-processors/tests/integration/SecureSocketGetTCPTest.cpp
@@ -98,8 +98,13 @@ class SecureSocketTest : public IntegrationBase {
     path = key_dir + "nifi-cert.pem";
     configuration->set("nifi.security.client.ca.certificate", path);
     configuration->set("nifi.c2.enable", "false");
+    std::string endpoint;
+    inv->getProperty(minifi::processors::GetTCP::EndpointList.getName(), endpoint);
+    auto endpoints = utils::StringUtils::split(endpoint, ",");
+    assert(1 == endpoints.size());
+    auto hostAndPort = utils::StringUtils::split(endpoint, ":");
     std::shared_ptr<org::apache::nifi::minifi::io::TLSContext> socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration);
-    server_socket = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, "localhost", 8776, 3);
+    server_socket = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, hostAndPort.at(0), std::stoi(hostAndPort.at(1)), 3);
     server_socket->initialize();
 
     isRunning_ = true;
@@ -155,10 +160,6 @@ class SecureSocketTest : public IntegrationBase {
     cleanup();
   }
 
-  virtual void waitToVerifyProcessor() {
-      std::this_thread::sleep_for(std::chrono::seconds(3));
-    }
-
  protected:
   std::function<bool()> check;
   std::function<int(std::vector<uint8_t>*b, int *size)> handler;
@@ -170,11 +171,11 @@ class SecureSocketTest : public IntegrationBase {
   std::shared_ptr<org::apache::nifi::minifi::io::TLSServerSocket> server_socket;
 };
 
-static void sigpipe_handle(int x) {}
+static void sigpipe_handle(int x) {
+}
 
 int main(int argc, char **argv) {
-  std::string key_dir, test_file_location, url;
-  url = "http://localhost:8888/api/heartbeat";
+  std::string key_dir, test_file_location;
   if (argc > 1) {
     test_file_location = argv[1];
     key_dir = argv[2];
diff --git a/libminifi/include/controllers/SSLContextService.h b/libminifi/include/controllers/SSLContextService.h
index c513ed1..ce485e6 100644
--- a/libminifi/include/controllers/SSLContextService.h
+++ b/libminifi/include/controllers/SSLContextService.h
@@ -23,6 +23,7 @@
 #ifdef OPENSSL_SUPPORT
 #include <openssl/err.h>
 #include <openssl/ssl.h>
+#include "io/tls/TLSUtils.h"
 #endif
 #include <iostream>
 #include <memory>
@@ -153,7 +154,7 @@ class SSLContextService : public core::controller::ControllerService {
       }
       if (!IsNullOrEmpty(passphrase_)) {
         SSL_CTX_set_default_passwd_cb_userdata(ctx, &passphrase_);
-        SSL_CTX_set_default_passwd_cb(ctx, pemPassWordCb);
+        SSL_CTX_set_default_passwd_cb(ctx, minifi::io::tls::pemPassWordCb);
       }
     }
 
@@ -186,19 +187,6 @@ class SSLContextService : public core::controller::ControllerService {
 
  protected:
 
-  static int pemPassWordCb(char *buf, int size, int rwflag, void *userdata) {
-
-    std::string *pass = (std::string*) userdata;
-    if (pass->length() > 0) {
-
-      memset(buf, 0x00, size);
-      memcpy(buf, pass->c_str(), pass->length() - 1);
-
-      return pass->length() - 1;
-    }
-    return 0;
-  }
-
   virtual void initializeTLS();
 
   std::mutex initialization_mutex_;
diff --git a/libminifi/include/io/tls/TLSSocket.h b/libminifi/include/io/tls/TLSSocket.h
index b6f625c..7806d5a 100644
--- a/libminifi/include/io/tls/TLSSocket.h
+++ b/libminifi/include/io/tls/TLSSocket.h
@@ -88,18 +88,6 @@ class TLSContext : public SocketContext {
 
  private:
 
-  static int pemPassWordCb(char *buf, int size, int rwflag, void *userdata) {
-
-    std::string *pass = (std::string*) userdata;
-    if (pass->length() > 0) {
-
-      memset(buf, 0x00, size);
-      memcpy(buf, pass->c_str(), pass->length() - 1);
-
-      return pass->length() - 1;
-    }
-    return 0;
-  }
 
   std::shared_ptr<logging::Logger> logger_;
   std::shared_ptr<Configure> configure_;
diff --git a/libminifi/include/io/tls/TLSUtils.h b/libminifi/include/io/tls/TLSUtils.h
new file mode 100644
index 0000000..b615968
--- /dev/null
+++ b/libminifi/include/io/tls/TLSUtils.h
@@ -0,0 +1,40 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_IO_TLSUTILS_H_
+#define LIBMINIFI_INCLUDE_IO_TLSUTILS_H_
+
+#include <atomic>
+#include <cstdint>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+namespace tls{
+
+int pemPassWordCb(char *buf, int size, int rwflag, void *userdata);
+
+} /* namespace tls */
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_IO_TLSUTILS_H_ */
diff --git a/libminifi/src/controllers/SSLContextService.cpp b/libminifi/src/controllers/SSLContextService.cpp
index 9cfb2f4..9899348 100644
--- a/libminifi/src/controllers/SSLContextService.cpp
+++ b/libminifi/src/controllers/SSLContextService.cpp
@@ -69,7 +69,7 @@ std::unique_ptr<SSLContext> SSLContextService::createSSLContext() {
     }
     if (!IsNullOrEmpty(passphrase_)) {
       SSL_CTX_set_default_passwd_cb_userdata(ctx, &passphrase_);
-      SSL_CTX_set_default_passwd_cb(ctx, pemPassWordCb);
+      SSL_CTX_set_default_passwd_cb(ctx, io::tls::pemPassWordCb);
     }
   }
 
diff --git a/libminifi/src/io/tls/TLSSocket.cpp b/libminifi/src/io/tls/TLSSocket.cpp
index ed94d36..a6a6ef5 100644
--- a/libminifi/src/io/tls/TLSSocket.cpp
+++ b/libminifi/src/io/tls/TLSSocket.cpp
@@ -22,6 +22,7 @@
 #include <string>
 #include <vector>
 #include "io/tls/TLSSocket.h"
+#include "io/tls/TLSUtils.h"
 #include "properties/Configure.h"
 #include "utils/StringUtils.h"
 #include "core/Property.h"
@@ -98,21 +99,20 @@ int16_t TLSContext::initialize(bool server_method) {
     }
     if (ssl_service_ != nullptr) {
       // if the private key has passphase
-      SSL_CTX_set_default_passwd_cb(ctx, pemPassWordCb);
-      SSL_CTX_set_default_passwd_cb_userdata(ctx, static_cast<void*>(const_cast<char*>(passphrase.c_str())));
+      SSL_CTX_set_default_passwd_cb(ctx, io::tls::pemPassWordCb);
+      SSL_CTX_set_default_passwd_cb_userdata(ctx, &passphrase);
     } else {
       if (configure_->get(Configure::nifi_security_client_pass_phrase, passphrase)) {
         std::ifstream file(passphrase.c_str(), std::ifstream::in);
-        if (!file.good()) {
-          return error_value;
+        if (file.good()) {
+          // if we have been given a file copy that, otherwise treat the passphrase as a password
+          std::string password;
+          password.assign((std::istreambuf_iterator<char>(file)), std::istreambuf_iterator<char>());
+          file.close();
+          passphrase = password;
         }
-
-        std::string password;
-        password.assign((std::istreambuf_iterator<char>(file)), std::istreambuf_iterator<char>());
-        file.close();
-        passphrase = password;
-        SSL_CTX_set_default_passwd_cb(ctx, pemPassWordCb);
-        SSL_CTX_set_default_passwd_cb_userdata(ctx, static_cast<void*>(const_cast<char*>(passphrase.c_str())));
+        SSL_CTX_set_default_passwd_cb(ctx, io::tls::pemPassWordCb);
+        SSL_CTX_set_default_passwd_cb_userdata(ctx, &passphrase);
       }
     }
 
diff --git a/libminifi/src/io/tls/TLSUtils.cpp b/libminifi/src/io/tls/TLSUtils.cpp
new file mode 100644
index 0000000..b709d2c
--- /dev/null
+++ b/libminifi/src/io/tls/TLSUtils.cpp
@@ -0,0 +1,56 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "io/tls/TLSUtils.h"
+#include "utils/StringUtils.h"
+#include <string>
+#include <cstring>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+namespace tls {
+
+int pemPassWordCb(char *buf, int size, int rwflag, void *userdata) {
+  const std::string * const origPass = reinterpret_cast<std::string*>(userdata);
+  // make copy the password, trimming it
+  const auto pass = utils::StringUtils::trimRight(*origPass);
+  /**
+   * 1) Ensure that password is trimmed.
+   * 2) Don't attempt a larger copy than the buffer we are allowed.
+   * validation of the key will subsequently occur, so any truncation
+   * would be caught, but we should be defensive here *
+   */
+  if (size > 0 && pass.size() <= static_cast<size_t>(size)) {
+    // Leaving this paradigm here. I don't think this is necessary
+    // while this string will be null terminated, OpenSSL does not expect
+    // a null terminator. Per their documentation:
+    // "The actual length of the password must be returned to the calling function. "
+    memset(buf, 0x00, size);
+    return pass.copy(buf, size);
+  }
+  return -1;
+}
+
+} /* namespace tls */
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
diff --git a/libminifi/test/resources/TestGetTCPSecureEmptyPass.yml b/libminifi/test/resources/TestGetTCPSecureEmptyPass.yml
new file mode 100644
index 0000000..e25918c
--- /dev/null
+++ b/libminifi/test/resources/TestGetTCPSecureEmptyPass.yml
@@ -0,0 +1,90 @@
+#
+# 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.
+#
+Flow Controller:
+    name: MiNiFi Flow
+    id: 2438e3c8-015a-1000-79ca-83af40ec1990
+Processors:
+    - name: invoke
+      id: 2438e3c8-015a-1000-79ca-83af40ec1991
+      class: org.apache.nifi.processors.standard.GetTCP
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 10 msec
+      penalization period: 30 msec
+      yield period: 10 msec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          SSL Context Service: SSLContextService
+          endpoint-list: localhost:38776
+          end-of-message-byte: d
+          reconnect-interval: 5ms
+          connection-attempt-timeout: 2000
+    - name: LogAttribute
+      id: 2438e3c8-015a-1000-79ca-83af40ec1992
+      class: org.apache.nifi.processors.standard.LogAttribute
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list: response
+      Properties:
+        Log Level: info
+        Log Payload: true
+
+Connections:
+    - name: TransferFilesToRPG
+      id: 2438e3c8-015a-1000-79ca-83af40ec1997
+      source name: invoke
+      source id: 2438e3c8-015a-1000-79ca-83af40ec1991
+      source relationship name: success
+      destination name: LogAttribute
+      destination id: 2438e3c8-015a-1000-79ca-83af40ec1992
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+    - name: TransferFilesToRPG2
+      id: 2438e3c8-015a-1000-79ca-83af40ec1917
+      source name: LogAttribute
+      source id: 2438e3c8-015a-1000-79ca-83af40ec1992
+      destination name: LogAttribute
+      destination id: 2438e3c8-015a-1000-79ca-83af40ec1992  
+      source relationship name: success
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+
+Controller Services:
+    - name: SSLContextService
+      id: 2438e3c8-015a-1000-79ca-83af40ec1994
+      class: SSLContextService
+      Properties:
+        Client Certificate:
+            - value: cn.crt.pem
+        Private Key:
+            - value: cn.ckey.pem
+        Passphrase:
+            - value: empty.cn.pass
+        CA Certificate:
+            - value: nifi-cert.pem
+
+Remote Processing Groups:
+    
diff --git a/libminifi/test/resources/TestGetTCPSecureWithFilePass.yml b/libminifi/test/resources/TestGetTCPSecureWithFilePass.yml
new file mode 100644
index 0000000..8f7cace
--- /dev/null
+++ b/libminifi/test/resources/TestGetTCPSecureWithFilePass.yml
@@ -0,0 +1,90 @@
+#
+# 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.
+#
+Flow Controller:
+    name: MiNiFi Flow
+    id: 2438e3c8-015a-1000-79ca-83af40ec1990
+Processors:
+    - name: invoke
+      id: 2438e3c8-015a-1000-79ca-83af40ec1991
+      class: org.apache.nifi.processors.standard.GetTCP
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 10 msec
+      penalization period: 30 msec
+      yield period: 10 msec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          SSL Context Service: SSLContextService
+          endpoint-list: localhost:18776
+          end-of-message-byte: d
+          reconnect-interval: 5ms
+          connection-attempt-timeout: 2000
+    - name: LogAttribute
+      id: 2438e3c8-015a-1000-79ca-83af40ec1992
+      class: org.apache.nifi.processors.standard.LogAttribute
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list: response
+      Properties:
+        Log Level: info
+        Log Payload: true
+
+Connections:
+    - name: TransferFilesToRPG
+      id: 2438e3c8-015a-1000-79ca-83af40ec1997
+      source name: invoke
+      source id: 2438e3c8-015a-1000-79ca-83af40ec1991
+      source relationship name: success
+      destination name: LogAttribute
+      destination id: 2438e3c8-015a-1000-79ca-83af40ec1992
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+    - name: TransferFilesToRPG2
+      id: 2438e3c8-015a-1000-79ca-83af40ec1917
+      source name: LogAttribute
+      source id: 2438e3c8-015a-1000-79ca-83af40ec1992
+      destination name: LogAttribute
+      destination id: 2438e3c8-015a-1000-79ca-83af40ec1992  
+      source relationship name: success
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+
+Controller Services:
+    - name: SSLContextService
+      id: 2438e3c8-015a-1000-79ca-83af40ec1994
+      class: SSLContextService
+      Properties:
+        Client Certificate:
+            - value: cn.crt.pem
+        Private Key:
+            - value: encrypted.key.pem
+        Passphrase:
+            - value: encrypted.cn.pass
+        CA Certificate:
+            - value: nifi-cert.pem
+
+Remote Processing Groups:
+    
diff --git a/libminifi/test/resources/TestGetTCPSecureWithPass.yml b/libminifi/test/resources/TestGetTCPSecureWithPass.yml
new file mode 100644
index 0000000..d0a34f2
--- /dev/null
+++ b/libminifi/test/resources/TestGetTCPSecureWithPass.yml
@@ -0,0 +1,90 @@
+#
+# 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.
+#
+Flow Controller:
+    name: MiNiFi Flow
+    id: 2438e3c8-015a-1000-79ca-83af40ec1990
+Processors:
+    - name: invoke
+      id: 2438e3c8-015a-1000-79ca-83af40ec1991
+      class: org.apache.nifi.processors.standard.GetTCP
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 10 msec
+      penalization period: 30 msec
+      yield period: 10 msec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          SSL Context Service: SSLContextService
+          endpoint-list: localhost:28776
+          end-of-message-byte: d
+          reconnect-interval: 5ms
+          connection-attempt-timeout: 2000
+    - name: LogAttribute
+      id: 2438e3c8-015a-1000-79ca-83af40ec1992
+      class: org.apache.nifi.processors.standard.LogAttribute
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list: response
+      Properties:
+        Log Level: info
+        Log Payload: true
+
+Connections:
+    - name: TransferFilesToRPG
+      id: 2438e3c8-015a-1000-79ca-83af40ec1997
+      source name: invoke
+      source id: 2438e3c8-015a-1000-79ca-83af40ec1991
+      source relationship name: success
+      destination name: LogAttribute
+      destination id: 2438e3c8-015a-1000-79ca-83af40ec1992
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+    - name: TransferFilesToRPG2
+      id: 2438e3c8-015a-1000-79ca-83af40ec1917
+      source name: LogAttribute
+      source id: 2438e3c8-015a-1000-79ca-83af40ec1992
+      destination name: LogAttribute
+      destination id: 2438e3c8-015a-1000-79ca-83af40ec1992  
+      source relationship name: success
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+
+Controller Services:
+    - name: SSLContextService
+      id: 2438e3c8-015a-1000-79ca-83af40ec1994
+      class: SSLContextService
+      Properties:
+        Client Certificate:
+            - value: cn.crt.pem
+        Private Key:
+            - value: encrypted.key.pem
+        Passphrase:
+            - value: despiteallmyrageimstillanicholascage
+        CA Certificate:
+            - value: nifi-cert.pem
+
+Remote Processing Groups:
+    
diff --git a/libminifi/test/resources/cn.pass b/libminifi/test/resources/cn.pass
index e69de29..3f2ff2d 100644
--- a/libminifi/test/resources/cn.pass
+++ b/libminifi/test/resources/cn.pass
@@ -0,0 +1,5 @@
+
+
+
+
+
diff --git a/libminifi/test/resources/cn.pass b/libminifi/test/resources/empty.cn.pass
similarity index 100%
copy from libminifi/test/resources/cn.pass
copy to libminifi/test/resources/empty.cn.pass
diff --git a/libminifi/test/resources/encrypted.cn.pass b/libminifi/test/resources/encrypted.cn.pass
new file mode 100644
index 0000000..2518fb4
--- /dev/null
+++ b/libminifi/test/resources/encrypted.cn.pass
@@ -0,0 +1,5 @@
+despiteallmyrageimstillanicholascage
+
+
+
+
diff --git a/libminifi/test/resources/encrypted.key.pem b/libminifi/test/resources/encrypted.key.pem
new file mode 100644
index 0000000..edaf576
--- /dev/null
+++ b/libminifi/test/resources/encrypted.key.pem
@@ -0,0 +1,30 @@
+-----BEGIN RSA PRIVATE KEY-----
+Proc-Type: 4,ENCRYPTED
+DEK-Info: AES-256-CBC,A01078B17BCADA9FE6011085CC131A20
+
+80HsK6Gj2thdPlyybMuaenM2acgl1yHdIkCSX1Hi0Nr6juTSVNKH/UTpeKeRnBio
+A+0zf/d4kAKVtTPhufPKVwx/bLXzTXy7jbQb8/P7fzLVv3+aDtHCcQ5K9QSvK5I7
+NJeOaugZ1PCZoY/67fkOQGK75M/8fByJQiPyOnqY4Ck0nJ/KboyT0tHvC7U1vZT1
+3N3HkQ+2RFNR9oFq6R8tnU6XB3rxa2TxzwNiPLhYzh4eQSg4SLlCkWFsJjiopboF
+AcoQixdLkgLnvJGE7IXhYqpymjaLdLo08T+bfbLHgWXeIn2CLCCwDIh+axiq/DGO
+Sg49J45qnn/WXwAQuYYqXNE7/SZdkHHlYjgM7e7TH31L96I/9z/tBJe38mrxjn6A
+ef5V34FF3gz+TIN+TzG2vFTovo4/vK+W6kwtAMZ/DLvhxQntOwuYGZJ/OMrK5E6k
+vChKzkqplmR1qqD2AFbXhET06BC58C+npQsaSw3fpITrhhg3gx7OXycYhs3YC023
+piiRnggbDxRwZzYjkWPhbxn+cf4dUvXRfAU5z/YtfsYlqz6odQ95DNvgM010IWf+
+YdO3D0UrxTPNbOEE7ROsUYnVkQ4foo1frJY81PwUYtS7WeAvsn0cq08y811pDk+E
+ICkSSj55WfryIwY4Xv0UyiejhnvD/MPb7PCPs6myZ4Wrm79OtQVLmV+P7MKudTpS
+yzFo1BOA2IWzcuVGHR/5r2hLpRr7iWNiHLyxpmuBjw7ABDFXlKjersz3pjNbXF9J
+rkAolGMl77BV8n0YsIIrpWTY7kcuZiXl3p05AnOnCqgUQVaTozvyu4Ur2WhRVl77
+VEVczenZXbIBjTFUh2oSZA9xzaqeVojc2mAAyu4OvBN/R6RWEWVb7eI22ZHNCdsy
+ZXuDmo1+1ArpINkhehERZgrI0fd+Q55Q41e8F7RKf/WvtjX0D8RO9Y+ZBwEWISpv
+Aye8tBa6whRXwDoO+gbqJ4usvcgPS9xN7KUuwxIu3GtM2aBHbObBDqE/DDiKzjK4
+gXZK8Q2qYYFwGYmEss+c75Veux5WPxi7A4gGDubCBO+c5mc8gJQ24u4QnYCRyvLq
+WKg2QDdoN7HN23vWB0rwk7UymfNDJL2jlOnm93JbVnGNWZpaVpRCAR37PEdTdyp6
+arKW9MACOUOjWt0019AkYkOjcnbY5LN/iw0UnN+nO0KEdUZEoYv5UPV6tMRs110b
+BQM50aMoB5rJyPmDt++GKx8TFff9WLchEnlu1XLTOVF/1h8/JwpMxkpIRyQ1a6sN
+Gx1MLikExUH09PzbUh1biqpyQUYCzrVp2uPvEhlH+AZKrpNXdtc0cn33FIKDRCQ1
+b1JBJOp7wdOcJrqgYLUOsxD5CB+BE6i1s5IvMT5u7LWsH+cxSVrgX4DBtjFFhlzz
+A7yYQtpF7O350PLQmsOCnPKyQ5R0CfRfKJGWajcZgrzxwEMddB1qGyu3tz6yFOw/
+J2qBgfKcAWcBxgkcqiryca6UYAM48gQAa7G7cYAxG4OaLOWEUppw+oU2euuvDMdq
+bNZu0d3zxNvyyl/PVK+QwphPyidcTRzsasZkugZ3LxNplAZHuEAyy9IQJ1EBfvhA
+-----END RSA PRIVATE KEY-----