You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by is...@apache.org on 2023/03/20 15:51:02 UTC

[ignite-3] 05/19: IGNITE-17607 get_cluster_nodes implemented

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

isapego pushed a commit to branch ignite-17607
in repository https://gitbox.apache.org/repos/asf/ignite-3.git

commit f0aaea1e296f89cb0523d3e447c58ccd0fda080e
Author: Igor Sapego <is...@apache.org>
AuthorDate: Tue Mar 7 15:49:11 2023 +0300

    IGNITE-17607 get_cluster_nodes implemented
---
 modules/platforms/cpp/ignite/client/CMakeLists.txt |  1 +
 .../ignite/client/detail/ignite_client_impl.cpp    | 49 +++++++++++++++
 .../cpp/ignite/client/detail/ignite_client_impl.h  | 19 ++++++
 .../cpp/ignite/client/detail/node_connection.cpp   |  2 +-
 .../platforms/cpp/ignite/client/ignite_client.cpp  | 14 +++++
 .../platforms/cpp/ignite/client/ignite_client.h    | 25 ++++++++
 .../platforms/cpp/tests/client-test/CMakeLists.txt |  1 +
 .../cpp/tests/client-test/compute_test.cpp         | 70 ++++++++++++++++++++++
 8 files changed, 180 insertions(+), 1 deletion(-)

diff --git a/modules/platforms/cpp/ignite/client/CMakeLists.txt b/modules/platforms/cpp/ignite/client/CMakeLists.txt
index 626b172a60..e77c9f1f0f 100644
--- a/modules/platforms/cpp/ignite/client/CMakeLists.txt
+++ b/modules/platforms/cpp/ignite/client/CMakeLists.txt
@@ -31,6 +31,7 @@ set(SOURCES
     transaction/transaction.cpp
     transaction/transactions.cpp
     detail/cluster_connection.cpp
+    detail/ignite_client_impl.cpp
     detail/utils.cpp
     detail/node_connection.cpp
     detail/compute/compute_impl.cpp
diff --git a/modules/platforms/cpp/ignite/client/detail/ignite_client_impl.cpp b/modules/platforms/cpp/ignite/client/detail/ignite_client_impl.cpp
new file mode 100644
index 0000000000..e5cffefcb9
--- /dev/null
+++ b/modules/platforms/cpp/ignite/client/detail/ignite_client_impl.cpp
@@ -0,0 +1,49 @@
+/*
+ * 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 "ignite/client/detail/ignite_client_impl.h"
+
+#include <ignite/protocol/utils.h>
+
+namespace ignite::detail {
+
+void ignite_client_impl::get_cluster_nodes_async(ignite_callback<std::vector<cluster_node>> callback) {
+    auto reader_func = [](protocol::reader &reader) -> std::vector<cluster_node> {
+        std::vector<cluster_node> nodes;
+        nodes.reserve(reader.read_array_size());
+
+        reader.read_array_raw([&nodes](auto, const msgpack_object &object) {
+            auto fields = object.via.array;
+            assert(fields.size >= 4);
+
+            auto id = protocol::unpack_object<std::string>(fields.ptr[0]);
+            auto name = protocol::unpack_object<std::string>(fields.ptr[1]);
+            auto host = protocol::unpack_object<std::string>(fields.ptr[2]);
+            auto port = protocol::unpack_object<std::int32_t>(fields.ptr[3]);
+
+            nodes.emplace_back(std::move(id), std::move(name),
+                network::end_point{std::move(host), std::uint16_t(port)});
+        });
+
+        return nodes;
+    };
+
+    m_connection->perform_request_rd<std::vector<cluster_node>>(
+        client_operation::CLUSTER_GET_NODES, std::move(reader_func), std::move(callback));
+}
+
+} // namespace ignite::detail
diff --git a/modules/platforms/cpp/ignite/client/detail/ignite_client_impl.h b/modules/platforms/cpp/ignite/client/detail/ignite_client_impl.h
index 0d23ce681e..8e9123332e 100644
--- a/modules/platforms/cpp/ignite/client/detail/ignite_client_impl.h
+++ b/modules/platforms/cpp/ignite/client/detail/ignite_client_impl.h
@@ -17,6 +17,7 @@
 
 #pragma once
 
+#include <ignite/client/detail/compute/compute_impl.h>
 #include <ignite/client/detail/cluster_connection.h>
 #include <ignite/client/detail/sql/sql_impl.h>
 #include <ignite/client/detail/table/tables_impl.h>
@@ -93,6 +94,13 @@ public:
      */
     [[nodiscard]] std::shared_ptr<sql_impl> get_sql_impl() const { return m_sql; }
 
+    /**
+     * Get Compute management API implementation.
+     *
+     * @return Compute management API implementation.
+     */
+    [[nodiscard]] std::shared_ptr<compute_impl> get_compute_impl() const { return m_compute; }
+
     /**
      * Get transactions management API implementation.
      *
@@ -100,6 +108,14 @@ public:
      */
     [[nodiscard]] std::shared_ptr<transactions_impl> get_transactions_impl() const { return m_transactions; }
 
+    /**
+     * Gets the cluster nodes asynchronously.
+     * NOTE: Temporary API to enable Compute until we have proper Cluster API.
+     *
+     * @param callback Callback called with the list of cluster nodes upon success.
+     */
+    void get_cluster_nodes_async(ignite_callback<std::vector<cluster_node>> callback);
+
 private:
     /** Configuration. */
     const ignite_client_configuration m_configuration;
@@ -113,6 +129,9 @@ private:
     /** SQL. */
     std::shared_ptr<sql_impl> m_sql;
 
+    /** Compute. */
+    std::shared_ptr<compute_impl> m_compute;
+
     /** Transactions. */
     std::shared_ptr<transactions_impl> m_transactions;
 };
diff --git a/modules/platforms/cpp/ignite/client/detail/node_connection.cpp b/modules/platforms/cpp/ignite/client/detail/node_connection.cpp
index 5811331b6e..a17c2b944f 100644
--- a/modules/platforms/cpp/ignite/client/detail/node_connection.cpp
+++ b/modules/platforms/cpp/ignite/client/detail/node_connection.cpp
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-#include "node_connection.h"
+#include "ignite/client/detail/node_connection.h"
 
 #include <ignite/protocol/utils.h>
 
diff --git a/modules/platforms/cpp/ignite/client/ignite_client.cpp b/modules/platforms/cpp/ignite/client/ignite_client.cpp
index 8c33215b60..09475940d0 100644
--- a/modules/platforms/cpp/ignite/client/ignite_client.cpp
+++ b/modules/platforms/cpp/ignite/client/ignite_client.cpp
@@ -74,10 +74,24 @@ sql ignite_client::get_sql() const noexcept {
     return sql(impl().get_sql_impl());
 }
 
+compute ignite_client::get_compute() const noexcept {
+    return compute(impl().get_compute_impl());
+}
+
 transactions ignite_client::get_transactions() const noexcept {
     return transactions(impl().get_transactions_impl());
 }
 
+void ignite_client::get_cluster_nodes_async(ignite_callback<std::vector<cluster_node>> callback) {
+    return impl().get_cluster_nodes_async(std::move(callback));
+}
+
+std::vector<cluster_node> ignite_client::get_cluster_nodes() {
+    return sync<std::vector<cluster_node>>([this](auto callback) mutable {
+        get_cluster_nodes_async(std::move(callback));
+    });
+}
+
 detail::ignite_client_impl &ignite_client::impl() noexcept {
     return *((detail::ignite_client_impl *) (m_impl.get()));
 }
diff --git a/modules/platforms/cpp/ignite/client/ignite_client.h b/modules/platforms/cpp/ignite/client/ignite_client.h
index acc02807bb..b20e10e8dd 100644
--- a/modules/platforms/cpp/ignite/client/ignite_client.h
+++ b/modules/platforms/cpp/ignite/client/ignite_client.h
@@ -17,7 +17,9 @@
 
 #pragma once
 
+#include "ignite/client/compute/compute.h"
 #include "ignite/client/ignite_client_configuration.h"
+#include "ignite/client/network/cluster_node.h"
 #include "ignite/client/sql/sql.h"
 #include "ignite/client/table/tables.h"
 #include "ignite/client/transaction/transactions.h"
@@ -108,6 +110,13 @@ public:
      */
     [[nodiscard]] IGNITE_API sql get_sql() const noexcept;
 
+    /**
+     * Gets the Compute API.
+     *
+     * @return Compute API.
+     */
+    [[nodiscard]] IGNITE_API compute get_compute() const noexcept;
+
     /**
      * Gets the Transactions API.
      *
@@ -115,6 +124,22 @@ public:
      */
     [[nodiscard]] IGNITE_API transactions get_transactions() const noexcept;
 
+    /**
+     * Gets the cluster nodes asynchronously.
+     * NOTE: Temporary API to enable Compute until we have proper Cluster API.
+     *
+     * @param callback Callback called with the list of cluster nodes upon success.
+     */
+    IGNITE_API void get_cluster_nodes_async(ignite_callback<std::vector<cluster_node>> callback);
+
+    /**
+     * Gets the cluster nodes.
+     * NOTE: Temporary API to enable Compute until we have proper Cluster API.
+     *
+     * @return The list of cluster nodes upon success.
+     */
+    [[nodiscard]] IGNITE_API std::vector<cluster_node> get_cluster_nodes();
+
 private:
     /**
      * Constructor
diff --git a/modules/platforms/cpp/tests/client-test/CMakeLists.txt b/modules/platforms/cpp/tests/client-test/CMakeLists.txt
index 90b43f2ad3..2d9611905f 100644
--- a/modules/platforms/cpp/tests/client-test/CMakeLists.txt
+++ b/modules/platforms/cpp/tests/client-test/CMakeLists.txt
@@ -20,6 +20,7 @@ project(ignite-client-test)
 set(TARGET ${PROJECT_NAME})
 
 set(SOURCES
+    compute_test.cpp
     gtest_logger.h
     ignite_client_test.cpp
     ignite_runner_suite.h
diff --git a/modules/platforms/cpp/tests/client-test/compute_test.cpp b/modules/platforms/cpp/tests/client-test/compute_test.cpp
new file mode 100644
index 0000000000..10b6e3aa1b
--- /dev/null
+++ b/modules/platforms/cpp/tests/client-test/compute_test.cpp
@@ -0,0 +1,70 @@
+/*
+ * 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 "ignite_runner_suite.h"
+
+#include "ignite/client/ignite_client.h"
+#include "ignite/client/ignite_client_configuration.h"
+
+#include <gmock/gmock-matchers.h>
+#include <gtest/gtest.h>
+
+#include <chrono>
+
+using namespace ignite;
+
+/**
+ * Test suite.
+ */
+class compute_test : public ignite_runner_suite {
+protected:
+    void SetUp() override {
+        ignite_client_configuration cfg{get_node_addrs()};
+        cfg.set_logger(get_logger());
+
+        m_client = ignite_client::start(cfg, std::chrono::seconds(30));
+    }
+
+    void TearDown() override {
+        // remove all
+    }
+
+    /** Ignite client. */
+    ignite_client m_client;
+};
+
+TEST_F(compute_test, get_cluster_nodes) {
+    auto cluster_nodes = m_client.get_cluster_nodes();
+
+    std::sort(cluster_nodes.begin(), cluster_nodes.end(), [] (const auto &n1, const auto &n2) {
+        return n1.get_name() < n2.get_name();
+    });
+
+    EXPECT_FALSE(cluster_nodes.empty());
+    EXPECT_EQ(2, cluster_nodes.size());
+
+    EXPECT_FALSE(cluster_nodes[0].get_id().empty());
+    EXPECT_FALSE(cluster_nodes[1].get_id().empty());
+
+    EXPECT_EQ(3344, cluster_nodes[0].get_address().port);
+    EXPECT_EQ(3345, cluster_nodes[1].get_address().port);
+
+    EXPECT_FALSE(cluster_nodes[0].get_address().host.empty());
+    EXPECT_FALSE(cluster_nodes[1].get_address().host.empty());
+
+    EXPECT_EQ(cluster_nodes[0].get_address().host, cluster_nodes[1].get_address().host);
+}