You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by sz...@apache.org on 2021/02/01 13:46:59 UTC

[nifi-minifi-cpp] branch main updated: MINIFICPP-1453 Verify Supported TLS Protocols

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

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


The following commit(s) were added to refs/heads/main by this push:
     new f50d87d  MINIFICPP-1453 Verify Supported TLS Protocols
f50d87d is described below

commit f50d87d6048795d7ed977bfa49e7f61aeb30c500
Author: Martin Zink <ma...@protonmail.com>
AuthorDate: Mon Feb 1 14:45:28 2021 +0100

    MINIFICPP-1453 Verify Supported TLS Protocols
    
    Currently org::apache::nifi:minifi::io::TLSSocket
    (which, among other things, handles communications with C2)
    only accept TLSv1.2 protocol (hardcoded behaviour)
    
    Added integration tests to verify this behaviour
      TLSClientSocketSupportedProtocolsTest
        creates simple TLS enabled test servers with various protocols
        org::apache::nifi::minifi::io::TLSSocket only connects
        to the TLSv1.2 enabled server
    
      TLSServerSocketSupportedProtocolsTest
        creates an org::apache::nifi::minifi::io::TLSServerSocket
        various test clients try to connect to it,
        but only the TLSv1.2 enabled client succeeds
    
    Closes #978
    
    Signed-off-by: Marton Szasz <sz...@gmail.com>
---
 .../standard-processors/tests/CMakeLists.txt       |   5 +
 .../TLSClientSocketSupportedProtocolsTest.cpp      | 202 ++++++++++++++
 .../TLSServerSocketSupportedProtocolsTest.cpp      | 303 +++++++++++++++++++++
 3 files changed, 510 insertions(+)

diff --git a/extensions/standard-processors/tests/CMakeLists.txt b/extensions/standard-processors/tests/CMakeLists.txt
index f2af687..e94e683 100644
--- a/extensions/standard-processors/tests/CMakeLists.txt
+++ b/extensions/standard-processors/tests/CMakeLists.txt
@@ -22,6 +22,8 @@ file(GLOB PROCESSOR_UNIT_TESTS  "unit/*.cpp")
 file(GLOB PROCESSOR_INTEGRATION_TESTS "integration/*.cpp")
 if(OPENSSL_OFF)
 	list(REMOVE_ITEM PROCESSOR_INTEGRATION_TESTS "${CMAKE_CURRENT_SOURCE_DIR}/integration/SecureSocketGetTCPTest.cpp")
+	list(REMOVE_ITEM PROCESSOR_INTEGRATION_TESTS "${CMAKE_CURRENT_SOURCE_DIR}/integration/TLSServerSocketSupportedProtocolsTest.cpp")
+	list(REMOVE_ITEM PROCESSOR_INTEGRATION_TESTS "${CMAKE_CURRENT_SOURCE_DIR}/integration/TLSClientSocketSupportedProtocolsTest.cpp")
 endif()
 
 SET(PROCESSOR_INT_TEST_COUNT 0)
@@ -78,6 +80,9 @@ if(NOT OPENSSL_OFF)
 	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 TLSServerSocketSupportedProtocolsTest COMMAND TLSServerSocketSupportedProtocolsTest "${TEST_RESOURCES}/")
+	add_test(NAME TLSClientSocketSupportedProtocolsTest COMMAND TLSClientSocketSupportedProtocolsTest "${TEST_RESOURCES}/")
 endif()
 
 add_test(NAME TailFileTest COMMAND TailFileTest "${TEST_RESOURCES}/TestTailFile.yml"  "${TEST_RESOURCES}/")
diff --git a/extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp b/extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
new file mode 100644
index 0000000..9c9b562
--- /dev/null
+++ b/extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
@@ -0,0 +1,202 @@
+/**
+ *
+ * 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 <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, const std::string& port, const std::string& path)
+      : port_(port), had_connection_(false) {
+    ctx_ = SSL_CTX_new(method);
+    configureContext(path);
+    socket_descriptor_ = createSocket(std::stoi(port_));
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    server_read_thread_ = std::thread([this]() -> void {
+        SocketDescriptor client = accept(socket_descriptor_, nullptr, nullptr);
+        if (client != INVALID_SOCKET) {
+            ssl_ = SSL_new(ctx_);
+            SSL_set_fd(ssl_, client);
+            had_connection_ = (SSL_accept(ssl_) == 1);
+        }
+    });
+  }
+
+  void shutdownServer() {
+#ifdef WIN32
+    shutdown(socket_descriptor_, SD_BOTH);
+    closesocket(socket_descriptor_);
+#else
+    shutdown(socket_descriptor_, SHUT_RDWR);
+    close(socket_descriptor_);
+#endif
+    server_read_thread_.join();
+  }
+
+  bool hadConnection() const {
+    return had_connection_;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  std::string port_;
+  SocketDescriptor socket_descriptor_;
+  bool had_connection_;
+  std::thread server_read_thread_;
+
+  void configureContext(const std::string& path) {
+    SSL_CTX_set_ecdh_auto(ctx_, 1);
+    /* Set the key and cert */
+    assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) == 1);
+    assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) == 1);
+  }
+
+  static SocketDescriptor createSocket(int port) {
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    SocketDescriptor socket_descriptor = socket(AF_INET, SOCK_STREAM, 0);
+    assert(socket_descriptor >= 0);
+    assert(bind(socket_descriptor, (struct sockaddr*)&addr, sizeof(addr)) >= 0);
+    assert(listen(socket_descriptor, 1) >= 0);
+
+    return socket_descriptor;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(const std::string& port, const std::string& path)
+      : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(const std::string& port, const std::string& path)
+      : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(const std::string& port, const std::string& path)
+      : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  explicit TLSClientSocketSupportedProtocolsTest(const std::string& key_dir)
+      : key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    verifyTLSClientSocketExclusiveCompatibilityWithTLSv1_2();
+  }
+
+
+ protected:
+  void configureSecurity() {
+    host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+    port_ = "38777";
+    if (!key_dir_.empty()) {
+      configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+      configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+      configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+      configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+    }
+  }
+
+  void verifyTLSClientSocketExclusiveCompatibilityWithTLSv1_2() {
+    verifyTLSProtocolCompatibility<SimpleSSLTestServerTLSv1>(false);
+    verifyTLSProtocolCompatibility<SimpleSSLTestServerTLSv1_1>(false);
+    verifyTLSProtocolCompatibility<SimpleSSLTestServerTLSv1_2>(true);
+  }
+
+  template <class TLSTestSever>
+  void verifyTLSProtocolCompatibility(const bool should_be_compatible) {
+    TLSTestSever server(port_, key_dir_);
+    server.waitForConnection();
+
+    const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+    client_socket_ = utils::make_unique<org::apache::nifi::minifi::io::TLSSocket>(socket_context, host_, std::stoi(port_), 0);
+    const bool client_initialized_successfully = (client_socket_->initialize() == 0);
+    assert(client_initialized_successfully == should_be_compatible);
+    server.shutdownServer();
+    assert(server.hadConnection() == should_be_compatible);
+  }
+
+ protected:
+    std::unique_ptr<org::apache::nifi::minifi::io::TLSSocket> client_socket_;
+    std::string host_;
+    std::string port_;
+    std::string key_dir_;
+    std::shared_ptr<minifi::Configure> configuration_;
+};
+
+static void sigpipe_handle(int) {
+}
+
+int main(int argc, char **argv) {
+  std::string key_dir;
+  if (argc > 1) {
+    key_dir = argv[1];
+  }
+#ifndef WIN32
+  signal(SIGPIPE, sigpipe_handle);
+#endif
+
+  TLSClientSocketSupportedProtocolsTest client_socket_supported_protocols_verifier(key_dir);
+
+  client_socket_supported_protocols_verifier.run();
+
+  return 0;
+}
diff --git a/extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp b/extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
new file mode 100644
index 0000000..79b089f
--- /dev/null
+++ b/extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
@@ -0,0 +1,303 @@
+/**
+ *
+ * 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 WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_error("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_error("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    ~TLSServerSocketSupportedProtocolsTest() {
+      shutdownServerSocket();
+      server_socket_.reset();
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = utils::make_unique<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      auto check = [this]() -> bool {
+        return is_running_;
+      };
+      auto handler = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::string contents = "hello world";
+        *bytes_written = {contents.begin(), contents.end()};
+        bytes_written->push_back(0);
+        *size = bytes_written->size();
+        return *size;
+      };
+      server_socket_->registerCallback(check, handler, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1>(false);
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_1>(false);
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {
+      TLSTestClient client(host_, port_);
+      assert(client.canConnect() == should_be_compatible);
+    }
+
+    void shutdownServerSocket() {
+      is_running_ = false;
+    }
+
+    std::atomic<bool> is_running_;
+    std::unique_ptr<org::apache::nifi::minifi::io::TLSServerSocket> server_socket_;
+    std::string host_;
+    std::string port_;
+    std::string key_dir_;
+    std::shared_ptr<minifi::Configure> configuration_;
+};
+
+static void sigpipe_handle(int) {
+}
+
+int main(int argc, char **argv) {
+  std::string key_dir;
+  if (argc > 1) {
+    key_dir = argv[1];
+  }
+#ifndef WIN32
+  signal(SIGPIPE, sigpipe_handle);
+#endif
+
+  TLSServerSocketSupportedProtocolsTest server_socket_supported_protocols_verifier(key_dir);
+
+  server_socket_supported_protocols_verifier.run();
+
+  return 0;
+}