Skip to content

Commit

Permalink
Merge pull request #336 from Enmk/fix_issue_335_no_connection_attempts
Browse files Browse the repository at this point in the history
Fixed issue #335 by making at least one connection attempt
  • Loading branch information
Enmk authored Sep 30, 2023
2 parents 911ce93 + dc76dac commit 68e592c
Show file tree
Hide file tree
Showing 4 changed files with 128 additions and 9 deletions.
7 changes: 5 additions & 2 deletions clickhouse/client.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -385,14 +385,17 @@ void Client::Impl::ResetConnectionEndpoint() {
}

void Client::Impl::CreateConnection() {
for (size_t i = 0; i < options_.send_retries;)
// make sure to try to connect to each endpoint at least once even if `options_.send_retries` is 0
const size_t max_attempts = (options_.send_retries ? options_.send_retries : 1);
for (size_t i = 0; i < max_attempts;)
{
try
{
// Try to connect to each endpoint before throwing exception.
ResetConnectionEndpoint();
return;
} catch (const std::system_error&) {
if (++i == options_.send_retries)
if (++i >= max_attempts)
{
throw;
}
Expand Down
2 changes: 1 addition & 1 deletion clickhouse/columns/string.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,7 @@ ColumnString::ColumnString(const std::vector<std::string>& data)
for (const auto & s : data) {
AppendUnsafe(s);
}
};
}

ColumnString::ColumnString(std::vector<std::string>&& data)
: ColumnString()
Expand Down
121 changes: 115 additions & 6 deletions ut/client_ut.cpp
Original file line number Diff line number Diff line change
@@ -1,17 +1,42 @@
#include <clickhouse/client.h>

#include "clickhouse/base/socket.h"
#include "readonly_client_test.h"
#include "connection_failed_client_test.h"
#include "ut/utils_comparison.h"
#include "utils.h"
#include "ut/roundtrip_column.h"
#include "ut/value_generators.h"

#include <gtest/gtest.h>

#include <memory>
#include <optional>
#include <ostream>
#include <string_view>
#include <thread>
#include <chrono>

using namespace clickhouse;

namespace clickhouse {
std::ostream & operator << (std::ostream & ostr, const Endpoint & endpoint) {
return ostr << endpoint.host << ":" << endpoint.port;
}
}

template <typename T>
std::shared_ptr<T> createTableWithOneColumn(Client & client, const std::string & table_name, const std::string & column_name)
{
auto col = std::make_shared<T>();
const auto type_name = col->GetType().GetName();

client.Execute("DROP TEMPORARY TABLE IF EXISTS " + table_name + ";");
client.Execute("CREATE TEMPORARY TABLE IF NOT EXISTS " + table_name + "( " + column_name + " " + type_name + " )");

return col;
}

// Use value-parameterized tests to run same tests with different client
// options.
class ClientCase : public testing::TestWithParam<ClientOptions> {
Expand All @@ -27,13 +52,9 @@ class ClientCase : public testing::TestWithParam<ClientOptions> {
template <typename T>
std::shared_ptr<T> createTableWithOneColumn(Block & block)
{
auto col = std::make_shared<T>();
const auto type_name = col->GetType().GetName();

client_->Execute("DROP TEMPORARY TABLE IF EXISTS " + table_name + ";");
client_->Execute("CREATE TEMPORARY TABLE IF NOT EXISTS " + table_name + "( " + column_name + " " + type_name + " )");
auto col = ::createTableWithOneColumn<T>(*client_, table_name, column_name);

block.AppendColumn("test_column", col);
block.AppendColumn(column_name, col);

return col;
}
Expand Down Expand Up @@ -1352,3 +1373,91 @@ INSTANTIATE_TEST_SUITE_P(ResetConnectionClientTest, ResetConnectionTestCase,
.SetRetryTimeout(std::chrono::seconds(1))
}
));

struct CountingSocketFactoryAdapter : public SocketFactory {

using ConnectRequests = std::vector<std::pair<ClientOptions, Endpoint>>;

SocketFactory & socket_factory;
ConnectRequests & connect_requests;

CountingSocketFactoryAdapter(SocketFactory & socket_factory, ConnectRequests& connect_requests)
: socket_factory(socket_factory)
, connect_requests(connect_requests)
{}

std::unique_ptr<SocketBase> connect(const ClientOptions& opts, const Endpoint& endpoint) {
connect_requests.emplace_back(opts, endpoint);

return socket_factory.connect(opts, endpoint);
}

void sleepFor(const std::chrono::milliseconds& duration) {
return socket_factory.sleepFor(duration);
}

size_t GetConnectRequestsCount() const {
return connect_requests.size();
}

};

TEST(SimpleClientTest, issue_335) {
// Make sure Client connects to server even with ClientOptions.SetSendRetries(0)
auto vals = MakeStrings();
auto col = std::make_shared<ColumnString>(vals);

CountingSocketFactoryAdapter::ConnectRequests connect_requests;
std::unique_ptr<SocketFactory> wrapped_socket_factory = std::make_unique<NonSecureSocketFactory>();
std::unique_ptr<SocketFactory> socket_factory = std::make_unique<CountingSocketFactoryAdapter>(*wrapped_socket_factory, connect_requests);

Client client(ClientOptions(LocalHostEndpoint)
.SetSendRetries(0), // <<=== crucial for reproducing https://github.com/ClickHouse/clickhouse-cpp/issues/335
std::move(socket_factory));

EXPECT_EQ(1u, connect_requests.size());
EXPECT_TRUE(CompareRecursive(vals, *RoundtripColumnValuesTyped(client, col)));

connect_requests.clear();

client.ResetConnection();
EXPECT_EQ(1u, connect_requests.size());
EXPECT_TRUE(CompareRecursive(vals, *RoundtripColumnValuesTyped(client, col)));

connect_requests.clear();

client.ResetConnectionEndpoint();
EXPECT_EQ(1u, connect_requests.size());
EXPECT_TRUE(CompareRecursive(vals, *RoundtripColumnValuesTyped(client, col)));
}

TEST(SimpleClientTest, issue_335_reconnects_count) {
// Make sure that client attempts to connect to each endpoint at least once.
CountingSocketFactoryAdapter::ConnectRequests connect_requests;
std::unique_ptr<SocketFactory> wrapped_socket_factory = std::make_unique<NonSecureSocketFactory>();
std::unique_ptr<SocketFactory> socket_factory = std::make_unique<CountingSocketFactoryAdapter>(*wrapped_socket_factory, connect_requests);

const std::vector<Endpoint> endpoints = {
Endpoint{"foo-invalid-hostname", 1234},
Endpoint{"bar-invalid-hostname", 4567},
};

EXPECT_ANY_THROW(
Client(ClientOptions()
.SetEndpoints(endpoints)
.SetSendRetries(0), // <<=== crucial for reproducing https://github.com/ClickHouse/clickhouse-cpp/issues/335
std::move(socket_factory));
);

EXPECT_EQ(endpoints.size(), connect_requests.size());
// make sure there was an attempt to connect to each endpoint at least once.
for (const auto & endpoint : endpoints)
{
auto p = std::find_if(connect_requests.begin(), connect_requests.end(), [&endpoint](const auto & connect_request) {
return connect_request.second == endpoint;
});

EXPECT_TRUE(connect_requests.end() != p)
<< "\tThere was no attempt to connect to endpoint " << endpoint;
}
}
7 changes: 7 additions & 0 deletions ut/roundtrip_column.h
Original file line number Diff line number Diff line change
@@ -1,9 +1,16 @@
#pragma once

#include <clickhouse/columns/column.h>
#include <memory>

namespace clickhouse {
class Client;
}

clickhouse::ColumnRef RoundtripColumnValues(clickhouse::Client& client, clickhouse::ColumnRef expected);

template <typename T>
auto RoundtripColumnValuesTyped(clickhouse::Client& client, std::shared_ptr<T> expected_col)
{
return RoundtripColumnValues(client, expected_col)->template As<T>();
}

0 comments on commit 68e592c

Please sign in to comment.