You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "isapego (via GitHub)" <gi...@apache.org> on 2023/06/06 08:56:01 UTC

[GitHub] [ignite-3] isapego opened a new pull request, #2149: IGNITE-19204 Implement ODBC connection establishment

isapego opened a new pull request, #2149:
URL: https://github.com/apache/ignite-3/pull/2149

   - Ported network part from Ignite 2;
   - Implemented connection establishment;
   - Added test for connection establishment.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] isapego commented on a diff in pull request #2149: IGNITE-19204 Implement ODBC connection establishment

Posted by "isapego (via GitHub)" <gi...@apache.org>.
isapego commented on code in PR #2149:
URL: https://github.com/apache/ignite-3/pull/2149#discussion_r1219743870


##########
modules/platforms/cpp/ignite/network/detail/linux/tcp_socket_client.h:
##########
@@ -0,0 +1,247 @@
+/*
+ * 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 "ignite/network/detail/linux/sockets.h"
+#include "ignite/network/detail/utils.h"
+#include "ignite/network/socket_client.h"
+
+#include <cstdint>
+#include <sstream>
+
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <netinet/tcp.h>
+#include <netdb.h>
+#include <unistd.h>
+
+
+namespace ignite::network
+{
+
+/**
+ * Socket client implementation.
+ */
+    class tcp_socket_client : public socket_client
+    {
+    public:
+        // Delete
+        tcp_socket_client(tcp_socket_client &&) = delete;
+        tcp_socket_client(const tcp_socket_client &) = delete;
+        tcp_socket_client &operator=(tcp_socket_client &&) = delete;
+        tcp_socket_client &operator=(const tcp_socket_client &) = delete;
+
+        /** Buffers size */
+        enum { BUFFER_SIZE = 0x10000 };
+
+        // Default
+        tcp_socket_client() = default;
+
+        /**
+         * Destructor.
+         */
+        ~tcp_socket_client() override {
+            internal_close();
+        }
+
+        /**
+         * Establish connection with remote TCP service.
+         *
+         * @param hostname Remote host name.
+         * @param port TCP service port.
+         * @param timeout Timeout.
+         * @return True on success.
+         */
+        bool connect(const char* hostname, std::uint16_t port, std::int32_t timeout) override
+        {
+            internal_close();
+
+            addrinfo hints{};
+
+            hints.ai_family = AF_UNSPEC;
+            hints.ai_socktype = SOCK_STREAM;
+            hints.ai_protocol = IPPROTO_TCP;
+
+            std::stringstream converter;
+            converter << port;
+            std::string strPort = converter.str();
+
+            // Resolve the server address and port
+            addrinfo *result{NULL}; // NOLINT(modernize-use-nullptr)
+            int res = getaddrinfo(hostname, strPort.c_str(), &hints, &result);
+
+            if (res != 0)

Review Comment:
   Good catch. Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] isapego commented on a diff in pull request #2149: IGNITE-19204 Implement ODBC connection establishment

Posted by "isapego (via GitHub)" <gi...@apache.org>.
isapego commented on code in PR #2149:
URL: https://github.com/apache/ignite-3/pull/2149#discussion_r1219749377


##########
modules/platforms/cpp/ignite/network/detail/win/sockets.cpp:
##########
@@ -82,10 +83,10 @@ void try_set_socket_options(SOCKET socket, int buf_size, BOOL no_delay, BOOL out
 
     int res = setsockopt(socket, SOL_SOCKET, SO_KEEPALIVE, reinterpret_cast<char *>(&keep_alive), sizeof(keep_alive));
 
-    // TODO: IGNITE-17606 Disable keep-alive once heartbeats are implemented.
+    // TODO: Disable keep-alive once heartbeats are implemented.

Review Comment:
   Yeah, and the ticket was actually correct. Seems like I messed up something here. Reverted changes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] ptupitsyn commented on a diff in pull request #2149: IGNITE-19204 Implement ODBC connection establishment

Posted by "ptupitsyn (via GitHub)" <gi...@apache.org>.
ptupitsyn commented on code in PR #2149:
URL: https://github.com/apache/ignite-3/pull/2149#discussion_r1219469486


##########
modules/platforms/cpp/ignite/network/detail/linux/tcp_socket_client.h:
##########
@@ -0,0 +1,247 @@
+/*
+ * 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 "ignite/network/detail/linux/sockets.h"
+#include "ignite/network/detail/utils.h"
+#include "ignite/network/socket_client.h"
+
+#include <cstdint>
+#include <sstream>
+
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <netinet/tcp.h>
+#include <netdb.h>
+#include <unistd.h>
+
+
+namespace ignite::network
+{
+
+/**
+ * Socket client implementation.
+ */
+    class tcp_socket_client : public socket_client
+    {
+    public:
+        // Delete
+        tcp_socket_client(tcp_socket_client &&) = delete;
+        tcp_socket_client(const tcp_socket_client &) = delete;
+        tcp_socket_client &operator=(tcp_socket_client &&) = delete;
+        tcp_socket_client &operator=(const tcp_socket_client &) = delete;
+
+        /** Buffers size */
+        enum { BUFFER_SIZE = 0x10000 };
+
+        // Default
+        tcp_socket_client() = default;
+
+        /**
+         * Destructor.
+         */
+        ~tcp_socket_client() override {
+            internal_close();
+        }
+
+        /**
+         * Establish connection with remote TCP service.
+         *
+         * @param hostname Remote host name.
+         * @param port TCP service port.
+         * @param timeout Timeout.
+         * @return True on success.
+         */
+        bool connect(const char* hostname, std::uint16_t port, std::int32_t timeout) override
+        {
+            internal_close();
+
+            addrinfo hints{};
+
+            hints.ai_family = AF_UNSPEC;
+            hints.ai_socktype = SOCK_STREAM;
+            hints.ai_protocol = IPPROTO_TCP;
+
+            std::stringstream converter;
+            converter << port;
+            std::string strPort = converter.str();
+
+            // Resolve the server address and port
+            addrinfo *result{NULL}; // NOLINT(modernize-use-nullptr)
+            int res = getaddrinfo(hostname, strPort.c_str(), &hints, &result);
+
+            if (res != 0)

Review Comment:
   Is the `res` useful, should we include it in the error message?



##########
modules/platforms/cpp/ignite/client/ignite_client_configuration.h:
##########
@@ -77,6 +85,26 @@ class ignite_client_configuration {
         ignite_client_configuration::m_endpoints.assign(endpoints.begin(), endpoints.end());
     }
 
+    /**
+     * Set endpoints.
+     *
+     * Examples of supported formats:
+     * - 192.168.1.25 - Default port is used, see DEFAULT_PORT;
+     * - 192.168.1.25:780 - Custom port;
+     * - 192.168.1.25:780..787  - Custom port range - ports are checked from

Review Comment:
   We agreed to remove port ranges (see https://issues.apache.org/jira/browse/IGNITE-19601) - should we get rid of them right here?



##########
modules/platforms/cpp/ignite/network/detail/win/tcp_socket_client.h:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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 "ignite/network/detail/win/sockets.h"
+
+#include <cstdint>
+#include <sstream>
+
+#include "ignite/network/detail/utils.h"
+#include "ignite/network/socket_client.h"
+
+namespace ignite::network
+{
+
+/**
+ * Socket client implementation.
+ */
+class tcp_socket_client : public socket_client
+{
+public:
+    // Delete
+    tcp_socket_client(tcp_socket_client &&) = delete;
+    tcp_socket_client(const tcp_socket_client &) = delete;
+    tcp_socket_client &operator=(tcp_socket_client &&) = delete;
+    tcp_socket_client &operator=(const tcp_socket_client &) = delete;
+
+    /** Buffers size */
+    enum { BUFFER_SIZE = 0x10000 };
+
+    // Default
+    tcp_socket_client() = default;
+
+    /**
+     * Destructor.
+     */
+    ~tcp_socket_client() override {
+        internal_close();
+    }
+
+    /**
+     * Establish connection with remote TCP service.
+     *
+     * @param hostname Remote host name.
+     * @param port TCP service port.
+     * @param timeout Timeout.
+     * @return True on success.
+     */
+    bool connect(const char* hostname, std::uint16_t port, std::int32_t timeout) override
+    {
+        detail::init_wsa();
+
+        internal_close();
+
+        addrinfo hints = { 0 };
+
+        hints.ai_family = AF_UNSPEC;
+        hints.ai_socktype = SOCK_STREAM;
+        hints.ai_protocol = IPPROTO_TCP;
+
+        std::stringstream converter;
+        converter << port;
+        std::string strPort = converter.str();
+
+        // Resolve the server address and port
+        addrinfo *result{NULL}; // NOLINT(modernize-use-nullptr)
+        int res = getaddrinfo(hostname, strPort.c_str(), &hints, &result);
+
+        if (res != 0)

Review Comment:
   Same as above - is `res` useful for the error message?



##########
modules/platforms/cpp/ignite/network/detail/win/sockets.cpp:
##########
@@ -82,10 +83,10 @@ void try_set_socket_options(SOCKET socket, int buf_size, BOOL no_delay, BOOL out
 
     int res = setsockopt(socket, SOL_SOCKET, SO_KEEPALIVE, reinterpret_cast<char *>(&keep_alive), sizeof(keep_alive));
 
-    // TODO: IGNITE-17606 Disable keep-alive once heartbeats are implemented.
+    // TODO: Disable keep-alive once heartbeats are implemented.

Review Comment:
   Is there a ticket?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] isapego merged pull request #2149: IGNITE-19204 Implement ODBC connection establishment

Posted by "isapego (via GitHub)" <gi...@apache.org>.
isapego merged PR #2149:
URL: https://github.com/apache/ignite-3/pull/2149


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] isapego commented on a diff in pull request #2149: IGNITE-19204 Implement ODBC connection establishment

Posted by "isapego (via GitHub)" <gi...@apache.org>.
isapego commented on code in PR #2149:
URL: https://github.com/apache/ignite-3/pull/2149#discussion_r1219750234


##########
modules/platforms/cpp/ignite/network/detail/win/tcp_socket_client.h:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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 "ignite/network/detail/win/sockets.h"
+
+#include <cstdint>
+#include <sstream>
+
+#include "ignite/network/detail/utils.h"
+#include "ignite/network/socket_client.h"
+
+namespace ignite::network
+{
+
+/**
+ * Socket client implementation.
+ */
+class tcp_socket_client : public socket_client
+{
+public:
+    // Delete
+    tcp_socket_client(tcp_socket_client &&) = delete;
+    tcp_socket_client(const tcp_socket_client &) = delete;
+    tcp_socket_client &operator=(tcp_socket_client &&) = delete;
+    tcp_socket_client &operator=(const tcp_socket_client &) = delete;
+
+    /** Buffers size */
+    enum { BUFFER_SIZE = 0x10000 };
+
+    // Default
+    tcp_socket_client() = default;
+
+    /**
+     * Destructor.
+     */
+    ~tcp_socket_client() override {
+        internal_close();
+    }
+
+    /**
+     * Establish connection with remote TCP service.
+     *
+     * @param hostname Remote host name.
+     * @param port TCP service port.
+     * @param timeout Timeout.
+     * @return True on success.
+     */
+    bool connect(const char* hostname, std::uint16_t port, std::int32_t timeout) override
+    {
+        detail::init_wsa();
+
+        internal_close();
+
+        addrinfo hints = { 0 };
+
+        hints.ai_family = AF_UNSPEC;
+        hints.ai_socktype = SOCK_STREAM;
+        hints.ai_protocol = IPPROTO_TCP;
+
+        std::stringstream converter;
+        converter << port;
+        std::string strPort = converter.str();
+
+        // Resolve the server address and port
+        addrinfo *result{NULL}; // NOLINT(modernize-use-nullptr)
+        int res = getaddrinfo(hostname, strPort.c_str(), &hints, &result);
+
+        if (res != 0)

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] isapego commented on a diff in pull request #2149: IGNITE-19204 Implement ODBC connection establishment

Posted by "isapego (via GitHub)" <gi...@apache.org>.
isapego commented on code in PR #2149:
URL: https://github.com/apache/ignite-3/pull/2149#discussion_r1219625436


##########
modules/platforms/cpp/ignite/client/ignite_client_configuration.h:
##########
@@ -77,6 +85,26 @@ class ignite_client_configuration {
         ignite_client_configuration::m_endpoints.assign(endpoints.begin(), endpoints.end());
     }
 
+    /**
+     * Set endpoints.
+     *
+     * Examples of supported formats:
+     * - 192.168.1.25 - Default port is used, see DEFAULT_PORT;
+     * - 192.168.1.25:780 - Custom port;
+     * - 192.168.1.25:780..787  - Custom port range - ports are checked from

Review Comment:
   I'll do that for ODBC, but I'd prefer to do it in a separate ticket for client. I remove port ranges from the comments though.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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