This commit is contained in:
Vitaliy Zakaznikov 2019-06-17 12:40:14 -04:00
commit 25063f7b66
100 changed files with 2215 additions and 598 deletions

View File

@ -13,5 +13,6 @@ ClickHouse is an open-source column-oriented database management system that all
## Upcoming Events ## Upcoming Events
* [ClickHouse on HighLoad++ Siberia](https://www.highload.ru/siberia/2019/abstracts/5348) on June 24-25. * [ClickHouse on HighLoad++ Siberia](https://www.highload.ru/siberia/2019/abstracts/5348) on June 24-25.
* [ClickHouse Community Meetup in Shenzhen](https://www.huodongxing.com/event/3483759917300) on October 20. * [ClickHouse Meetup in Novosibirsk](https://events.yandex.ru/events/ClickHouse/26-June-2019/) on June 26.
* [ClickHouse Community Meetup in Shanghai](https://www.huodongxing.com/event/4483760336000) on October 27. * [ClickHouse Meetup in Shenzhen](https://www.huodongxing.com/event/3483759917300) on October 20.
* [ClickHouse Meetup in Shanghai](https://www.huodongxing.com/event/4483760336000) on October 27.

View File

@ -54,10 +54,10 @@ public:
const String & host_, UInt16 port_, bool secure_, const String & default_database_, const String & host_, UInt16 port_, bool secure_, const String & default_database_,
const String & user_, const String & password_, const String & stage, const String & user_, const String & password_, const String & stage,
bool randomize_, size_t max_iterations_, double max_time_, bool randomize_, size_t max_iterations_, double max_time_,
const String & json_path_, const ConnectionTimeouts & timeouts, const Settings & settings_) const String & json_path_, const Settings & settings_)
: :
concurrency(concurrency_), delay(delay_), queue(concurrency), concurrency(concurrency_), delay(delay_), queue(concurrency),
connections(concurrency, host_, port_, default_database_, user_, password_, timeouts, "benchmark", Protocol::Compression::Enable, secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable), connections(concurrency, host_, port_, default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable),
randomize(randomize_), max_iterations(max_iterations_), max_time(max_time_), randomize(randomize_), max_iterations(max_iterations_), max_time(max_time_),
json_path(json_path_), settings(settings_), global_context(Context::createGlobal()), pool(concurrency) json_path(json_path_), settings(settings_), global_context(Context::createGlobal()), pool(concurrency)
{ {
@ -240,7 +240,8 @@ private:
std::uniform_int_distribution<size_t> distribution(0, queries.size() - 1); std::uniform_int_distribution<size_t> distribution(0, queries.size() - 1);
for (size_t i = 0; i < concurrency; ++i) for (size_t i = 0; i < concurrency; ++i)
pool.schedule(std::bind(&Benchmark::thread, this, connections.get())); pool.schedule(std::bind(&Benchmark::thread, this,
connections.get(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings))));
InterruptListener interrupt_listener; InterruptListener interrupt_listener;
info_per_interval.watch.restart(); info_per_interval.watch.restart();
@ -310,7 +311,9 @@ private:
void execute(ConnectionPool::Entry & connection, Query & query) void execute(ConnectionPool::Entry & connection, Query & query)
{ {
Stopwatch watch; Stopwatch watch;
RemoteBlockInputStream stream(*connection, query, {}, global_context, &settings, nullptr, Tables(), query_processing_stage); RemoteBlockInputStream stream(
*connection,
query, {}, global_context, &settings, nullptr, Tables(), query_processing_stage);
Progress progress; Progress progress;
stream.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); }); stream.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); });
@ -485,7 +488,6 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
options["iterations"].as<size_t>(), options["iterations"].as<size_t>(),
options["timelimit"].as<double>(), options["timelimit"].as<double>(),
options["json"].as<std::string>(), options["json"].as<std::string>(),
ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings),
settings); settings);
return benchmark.run(); return benchmark.run();
} }

View File

@ -204,7 +204,6 @@ private:
ConnectionParameters connection_parameters; ConnectionParameters connection_parameters;
void initialize(Poco::Util::Application & self) void initialize(Poco::Util::Application & self)
{ {
Poco::Util::Application::initialize(self); Poco::Util::Application::initialize(self);
@ -337,7 +336,7 @@ private:
DateLUT::instance(); DateLUT::instance();
if (!context.getSettingsRef().use_client_time_zone) if (!context.getSettingsRef().use_client_time_zone)
{ {
const auto & time_zone = connection->getServerTimezone(); const auto & time_zone = connection->getServerTimezone(connection_parameters.timeouts);
if (!time_zone.empty()) if (!time_zone.empty())
{ {
try try
@ -521,7 +520,6 @@ private:
connection_parameters.default_database, connection_parameters.default_database,
connection_parameters.user, connection_parameters.user,
connection_parameters.password, connection_parameters.password,
connection_parameters.timeouts,
"client", "client",
connection_parameters.compression, connection_parameters.compression,
connection_parameters.security); connection_parameters.security);
@ -537,11 +535,14 @@ private:
connection->setThrottler(throttler); connection->setThrottler(throttler);
} }
connection->getServerVersion(server_name, server_version_major, server_version_minor, server_version_patch, server_revision); connection->getServerVersion(connection_parameters.timeouts,
server_name, server_version_major, server_version_minor, server_version_patch, server_revision);
server_version = toString(server_version_major) + "." + toString(server_version_minor) + "." + toString(server_version_patch); server_version = toString(server_version_major) + "." + toString(server_version_minor) + "." + toString(server_version_patch);
if (server_display_name = connection->getServerDisplayName(); server_display_name.length() == 0) if (
server_display_name = connection->getServerDisplayName(connection_parameters.timeouts);
server_display_name.length() == 0)
{ {
server_display_name = config().getString("host", "localhost"); server_display_name = config().getString("host", "localhost");
} }
@ -752,7 +753,7 @@ private:
} }
if (!test_hint.checkActual(actual_server_error, actual_client_error, got_exception, last_exception)) if (!test_hint.checkActual(actual_server_error, actual_client_error, got_exception, last_exception))
connection->forceConnected(); connection->forceConnected(connection_parameters.timeouts);
if (got_exception && !ignore_error) if (got_exception && !ignore_error)
{ {
@ -828,7 +829,7 @@ private:
if (with_output && with_output->settings_ast) if (with_output && with_output->settings_ast)
apply_query_settings(*with_output->settings_ast); apply_query_settings(*with_output->settings_ast);
connection->forceConnected(); connection->forceConnected(connection_parameters.timeouts);
/// INSERT query for which data transfer is needed (not an INSERT SELECT) is processed separately. /// INSERT query for which data transfer is needed (not an INSERT SELECT) is processed separately.
if (insert && !insert->select) if (insert && !insert->select)
@ -899,7 +900,7 @@ private:
/// Process the query that doesn't require transferring data blocks to the server. /// Process the query that doesn't require transferring data blocks to the server.
void processOrdinaryQuery() void processOrdinaryQuery()
{ {
connection->sendQuery(query, query_id, QueryProcessingStage::Complete, &context.getSettingsRef(), nullptr, true); connection->sendQuery(connection_parameters.timeouts, query, query_id, QueryProcessingStage::Complete, &context.getSettingsRef(), nullptr, true);
sendExternalTables(); sendExternalTables();
receiveResult(); receiveResult();
} }
@ -917,7 +918,7 @@ private:
if (!parsed_insert_query.data && (is_interactive || (stdin_is_not_tty && std_in.eof()))) if (!parsed_insert_query.data && (is_interactive || (stdin_is_not_tty && std_in.eof())))
throw Exception("No data to insert", ErrorCodes::NO_DATA_TO_INSERT); throw Exception("No data to insert", ErrorCodes::NO_DATA_TO_INSERT);
connection->sendQuery(query_without_data, query_id, QueryProcessingStage::Complete, &context.getSettingsRef(), nullptr, true); connection->sendQuery(connection_parameters.timeouts, query_without_data, query_id, QueryProcessingStage::Complete, &context.getSettingsRef(), nullptr, true);
sendExternalTables(); sendExternalTables();
/// Receive description of table structure. /// Receive description of table structure.
@ -1064,7 +1065,7 @@ private:
bool cancelled = false; bool cancelled = false;
// TODO: get the poll_interval from commandline. // TODO: get the poll_interval from commandline.
const auto receive_timeout = connection->getTimeouts().receive_timeout; const auto receive_timeout = connection_parameters.timeouts.receive_timeout;
constexpr size_t default_poll_interval = 1000000; /// in microseconds constexpr size_t default_poll_interval = 1000000; /// in microseconds
constexpr size_t min_poll_interval = 5000; /// in microseconds constexpr size_t min_poll_interval = 5000; /// in microseconds
const size_t poll_interval const size_t poll_interval

View File

@ -14,6 +14,7 @@
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Client/Connection.h> #include <Client/Connection.h>
#include <IO/ConnectionTimeouts.h>
namespace DB namespace DB
@ -42,7 +43,7 @@ private:
"KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE" "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE"
}; };
/// Words are fetched asynchonously. /// Words are fetched asynchronously.
std::thread loading_thread; std::thread loading_thread;
std::atomic<bool> ready{false}; std::atomic<bool> ready{false};
@ -71,7 +72,7 @@ private:
return word; return word;
} }
void loadImpl(Connection & connection, size_t suggestion_limit) void loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit)
{ {
std::stringstream query; std::stringstream query;
query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM ("
@ -104,12 +105,12 @@ private:
query << ") WHERE notEmpty(res)"; query << ") WHERE notEmpty(res)";
fetch(connection, query.str()); fetch(connection, timeouts, query.str());
} }
void fetch(Connection & connection, const std::string & query) void fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query)
{ {
connection.sendQuery(query); connection.sendQuery(timeouts, query);
while (true) while (true)
{ {
@ -175,12 +176,11 @@ public:
connection_parameters.default_database, connection_parameters.default_database,
connection_parameters.user, connection_parameters.user,
connection_parameters.password, connection_parameters.password,
connection_parameters.timeouts,
"client", "client",
connection_parameters.compression, connection_parameters.compression,
connection_parameters.security); connection_parameters.security);
loadImpl(connection, suggestion_limit); loadImpl(connection, connection_parameters.timeouts, suggestion_limit);
} }
catch (...) catch (...)
{ {

View File

@ -54,6 +54,7 @@
#include <DataStreams/AsynchronousBlockInputStream.h> #include <DataStreams/AsynchronousBlockInputStream.h>
#include <DataStreams/copyData.h> #include <DataStreams/copyData.h>
#include <DataStreams/NullBlockOutputStream.h> #include <DataStreams/NullBlockOutputStream.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
@ -798,13 +799,13 @@ public:
} }
void discoverShardPartitions(const TaskShardPtr & task_shard) void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard)
{ {
TaskTable & task_table = task_shard->task_table; TaskTable & task_table = task_shard->task_table;
LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription()); LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription());
auto get_partitions = [&] () { return getShardPartitions(*task_shard); }; auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); };
auto existing_partitions_names = retry(get_partitions, 60); auto existing_partitions_names = retry(get_partitions, 60);
Strings filtered_partitions_names; Strings filtered_partitions_names;
Strings missing_partitions; Strings missing_partitions;
@ -880,14 +881,14 @@ public:
} }
/// Compute set of partitions, assume set of partitions aren't changed during the processing /// Compute set of partitions, assume set of partitions aren't changed during the processing
void discoverTablePartitions(TaskTable & task_table, UInt64 num_threads = 0) void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0)
{ {
/// Fetch partitions list from a shard /// Fetch partitions list from a shard
{ {
ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores());
for (const TaskShardPtr & task_shard : task_table.all_shards) for (const TaskShardPtr & task_shard : task_table.all_shards)
thread_pool.schedule([this, task_shard]() { discoverShardPartitions(task_shard); }); thread_pool.schedule([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); });
LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs"); LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs");
thread_pool.wait(); thread_pool.wait();
@ -955,7 +956,7 @@ public:
task_descprtion_current_version = version_to_update; task_descprtion_current_version = version_to_update;
} }
void process() void process(const ConnectionTimeouts & timeouts)
{ {
for (TaskTable & task_table : task_cluster->table_tasks) for (TaskTable & task_table : task_cluster->table_tasks)
{ {
@ -969,7 +970,7 @@ public:
if (!task_table.has_enabled_partitions) if (!task_table.has_enabled_partitions)
{ {
/// If there are no specified enabled_partitions, we must discover them manually /// If there are no specified enabled_partitions, we must discover them manually
discoverTablePartitions(task_table); discoverTablePartitions(timeouts, task_table);
/// After partitions of each shard are initialized, initialize cluster partitions /// After partitions of each shard are initialized, initialize cluster partitions
for (const TaskShardPtr & task_shard : task_table.all_shards) for (const TaskShardPtr & task_shard : task_table.all_shards)
@ -1009,7 +1010,7 @@ public:
bool table_is_done = false; bool table_is_done = false;
for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries) for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries)
{ {
if (tryProcessTable(task_table)) if (tryProcessTable(timeouts, task_table))
{ {
table_is_done = true; table_is_done = true;
break; break;
@ -1053,8 +1054,10 @@ protected:
return getWorkersPath() + "/" + host_id; return getWorkersPath() + "/" + host_id;
} }
zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed(const zkutil::ZooKeeperPtr & zookeeper, zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed(
const String & description, bool unprioritized) const zkutil::ZooKeeperPtr & zookeeper,
const String & description,
bool unprioritized)
{ {
std::chrono::milliseconds current_sleep_time = default_sleep_time; std::chrono::milliseconds current_sleep_time = default_sleep_time;
static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec
@ -1329,7 +1332,7 @@ protected:
static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_table_tries = 1000;
static constexpr UInt64 max_shard_partition_tries = 600; static constexpr UInt64 max_shard_partition_tries = 600;
bool tryProcessTable(TaskTable & task_table) bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table)
{ {
/// An heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint /// An heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint
bool previous_shard_is_instantly_finished = false; bool previous_shard_is_instantly_finished = false;
@ -1360,7 +1363,7 @@ protected:
/// If not, did we check existence of that partition previously? /// If not, did we check existence of that partition previously?
if (shard->checked_partitions.count(partition_name) == 0) if (shard->checked_partitions.count(partition_name) == 0)
{ {
auto check_shard_has_partition = [&] () { return checkShardHasPartition(*shard, partition_name); }; auto check_shard_has_partition = [&] () { return checkShardHasPartition(timeouts, *shard, partition_name); };
bool has_partition = retry(check_shard_has_partition); bool has_partition = retry(check_shard_has_partition);
shard->checked_partitions.emplace(partition_name); shard->checked_partitions.emplace(partition_name);
@ -1397,7 +1400,7 @@ protected:
bool was_error = false; bool was_error = false;
for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num)
{ {
task_status = tryProcessPartitionTask(partition, is_unprioritized_task); task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task);
/// Exit if success /// Exit if success
if (task_status == PartitionTaskStatus::Finished) if (task_status == PartitionTaskStatus::Finished)
@ -1483,13 +1486,13 @@ protected:
Error, Error,
}; };
PartitionTaskStatus tryProcessPartitionTask(ShardPartition & task_partition, bool is_unprioritized_task) PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task)
{ {
PartitionTaskStatus res; PartitionTaskStatus res;
try try
{ {
res = processPartitionTaskImpl(task_partition, is_unprioritized_task); res = processPartitionTaskImpl(timeouts, task_partition, is_unprioritized_task);
} }
catch (...) catch (...)
{ {
@ -1510,7 +1513,7 @@ protected:
return res; return res;
} }
PartitionTaskStatus processPartitionTaskImpl(ShardPartition & task_partition, bool is_unprioritized_task) PartitionTaskStatus processPartitionTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task)
{ {
TaskShard & task_shard = task_partition.task_shard; TaskShard & task_shard = task_partition.task_shard;
TaskTable & task_table = task_shard.task_table; TaskTable & task_table = task_shard.task_table;
@ -1611,7 +1614,7 @@ protected:
zookeeper->createAncestors(current_task_status_path); zookeeper->createAncestors(current_task_status_path);
/// We need to update table definitions for each partition, it could be changed after ALTER /// We need to update table definitions for each partition, it could be changed after ALTER
createShardInternalTables(task_shard); createShardInternalTables(timeouts, task_shard);
/// Check that destination partition is empty if we are first worker /// Check that destination partition is empty if we are first worker
/// NOTE: this check is incorrect if pull and push tables have different partition key! /// NOTE: this check is incorrect if pull and push tables have different partition key!
@ -1828,23 +1831,25 @@ protected:
return typeid_cast<const ColumnString &>(*block.safeGetByPosition(0).column).getDataAt(0).toString(); return typeid_cast<const ColumnString &>(*block.safeGetByPosition(0).column).getDataAt(0).toString();
} }
ASTPtr getCreateTableForPullShard(TaskShard & task_shard) ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard)
{ {
/// Fetch and parse (possibly) new definition /// Fetch and parse (possibly) new definition
auto connection_entry = task_shard.info.pool->get(&task_cluster->settings_pull); auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_pull);
String create_query_pull_str = getRemoteCreateTable(task_shard.task_table.table_pull, *connection_entry, String create_query_pull_str = getRemoteCreateTable(
task_shard.task_table.table_pull,
*connection_entry,
&task_cluster->settings_pull); &task_cluster->settings_pull);
ParserCreateQuery parser_create_query; ParserCreateQuery parser_create_query;
return parseQuery(parser_create_query, create_query_pull_str, 0); return parseQuery(parser_create_query, create_query_pull_str, 0);
} }
void createShardInternalTables(TaskShard & task_shard, bool create_split = true) void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true)
{ {
TaskTable & task_table = task_shard.task_table; TaskTable & task_table = task_shard.task_table;
/// We need to update table definitions for each part, it could be changed after ALTER /// We need to update table definitions for each part, it could be changed after ALTER
task_shard.current_pull_table_create_query = getCreateTableForPullShard(task_shard); task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard);
/// Create local Distributed tables: /// Create local Distributed tables:
/// a table fetching data from current shard and a table inserting data to the whole destination cluster /// a table fetching data from current shard and a table inserting data to the whole destination cluster
@ -1872,9 +1877,9 @@ protected:
} }
std::set<String> getShardPartitions(TaskShard & task_shard) std::set<String> getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard)
{ {
createShardInternalTables(task_shard, false); createShardInternalTables(timeouts, task_shard, false);
TaskTable & task_table = task_shard.task_table; TaskTable & task_table = task_shard.task_table;
@ -1914,9 +1919,9 @@ protected:
return res; return res;
} }
bool checkShardHasPartition(TaskShard & task_shard, const String & partition_quoted_name) bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name)
{ {
createShardInternalTables(task_shard, false); createShardInternalTables(timeouts, task_shard, false);
TaskTable & task_table = task_shard.task_table; TaskTable & task_table = task_shard.task_table;
@ -1998,7 +2003,8 @@ protected:
Settings current_settings = settings ? *settings : task_cluster->settings_common; Settings current_settings = settings ? *settings : task_cluster->settings_common;
current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1;
auto connections = shard.pool->getMany(&current_settings, pool_mode); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time);
auto connections = shard.pool->getMany(timeouts, &current_settings, pool_mode);
for (auto & connection : connections) for (auto & connection : connections)
{ {
@ -2187,7 +2193,7 @@ void ClusterCopierApp::mainImpl()
copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false)); copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false));
copier->init(); copier->init();
copier->process(); copier->process(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context->getSettingsRef()));
} }

View File

@ -3,6 +3,7 @@
#include <Core/Types.h> #include <Core/Types.h>
#include <Common/CpuId.h> #include <Common/CpuId.h>
#include <common/getMemoryAmount.h> #include <common/getMemoryAmount.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFile.h> #include <IO/WriteBufferFromFile.h>
@ -24,6 +25,7 @@ namespace
void waitQuery(Connection & connection) void waitQuery(Connection & connection)
{ {
bool finished = false; bool finished = false;
while (true) while (true)
{ {
if (!connection.poll(1000000)) if (!connection.poll(1000000))
@ -50,12 +52,14 @@ namespace fs = boost::filesystem;
PerformanceTest::PerformanceTest( PerformanceTest::PerformanceTest(
const XMLConfigurationPtr & config_, const XMLConfigurationPtr & config_,
Connection & connection_, Connection & connection_,
const ConnectionTimeouts & timeouts_,
InterruptListener & interrupt_listener_, InterruptListener & interrupt_listener_,
const PerformanceTestInfo & test_info_, const PerformanceTestInfo & test_info_,
Context & context_, Context & context_,
const std::vector<size_t> & queries_to_run_) const std::vector<size_t> & queries_to_run_)
: config(config_) : config(config_)
, connection(connection_) , connection(connection_)
, timeouts(timeouts_)
, interrupt_listener(interrupt_listener_) , interrupt_listener(interrupt_listener_)
, test_info(test_info_) , test_info(test_info_)
, context(context_) , context(context_)
@ -108,7 +112,7 @@ bool PerformanceTest::checkPreconditions() const
size_t exist = 0; size_t exist = 0;
connection.sendQuery(query, "", QueryProcessingStage::Complete, &test_info.settings, nullptr, false); connection.sendQuery(timeouts, query, "", QueryProcessingStage::Complete, &test_info.settings, nullptr, false);
while (true) while (true)
{ {
@ -188,7 +192,7 @@ void PerformanceTest::prepare() const
for (const auto & query : test_info.create_and_fill_queries) for (const auto & query : test_info.create_and_fill_queries)
{ {
LOG_INFO(log, "Executing create or fill query \"" << query << '\"'); LOG_INFO(log, "Executing create or fill query \"" << query << '\"');
connection.sendQuery(query, "", QueryProcessingStage::Complete, &test_info.settings, nullptr, false); connection.sendQuery(timeouts, query, "", QueryProcessingStage::Complete, &test_info.settings, nullptr, false);
waitQuery(connection); waitQuery(connection);
LOG_INFO(log, "Query finished"); LOG_INFO(log, "Query finished");
} }
@ -200,7 +204,7 @@ void PerformanceTest::finish() const
for (const auto & query : test_info.drop_queries) for (const auto & query : test_info.drop_queries)
{ {
LOG_INFO(log, "Executing drop query \"" << query << '\"'); LOG_INFO(log, "Executing drop query \"" << query << '\"');
connection.sendQuery(query, "", QueryProcessingStage::Complete, &test_info.settings, nullptr, false); connection.sendQuery(timeouts, query, "", QueryProcessingStage::Complete, &test_info.settings, nullptr, false);
waitQuery(connection); waitQuery(connection);
LOG_INFO(log, "Query finished"); LOG_INFO(log, "Query finished");
} }

View File

@ -5,6 +5,7 @@
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Poco/Util/XMLConfiguration.h> #include <Poco/Util/XMLConfiguration.h>
#include <IO/ConnectionTimeouts.h>
#include "PerformanceTestInfo.h" #include "PerformanceTestInfo.h"
namespace DB namespace DB
@ -20,6 +21,7 @@ public:
PerformanceTest( PerformanceTest(
const XMLConfigurationPtr & config_, const XMLConfigurationPtr & config_,
Connection & connection_, Connection & connection_,
const ConnectionTimeouts & timeouts_,
InterruptListener & interrupt_listener_, InterruptListener & interrupt_listener_,
const PerformanceTestInfo & test_info_, const PerformanceTestInfo & test_info_,
Context & context_, Context & context_,
@ -45,6 +47,7 @@ private:
private: private:
XMLConfigurationPtr config; XMLConfigurationPtr config;
Connection & connection; Connection & connection;
const ConnectionTimeouts & timeouts;
InterruptListener & interrupt_listener; InterruptListener & interrupt_listener;
PerformanceTestInfo test_info; PerformanceTestInfo test_info;

View File

@ -72,10 +72,11 @@ public:
Strings && tests_names_regexp_, Strings && tests_names_regexp_,
Strings && skip_names_regexp_, Strings && skip_names_regexp_,
const std::unordered_map<std::string, std::vector<size_t>> query_indexes_, const std::unordered_map<std::string, std::vector<size_t>> query_indexes_,
const ConnectionTimeouts & timeouts) const ConnectionTimeouts & timeouts_)
: connection(host_, port_, default_database_, user_, : connection(host_, port_, default_database_, user_,
password_, timeouts, "performance-test", Protocol::Compression::Enable, password_, "performance-test", Protocol::Compression::Enable,
secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable) secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable)
, timeouts(timeouts_)
, tests_tags(std::move(tests_tags_)) , tests_tags(std::move(tests_tags_))
, tests_names(std::move(tests_names_)) , tests_names(std::move(tests_names_))
, tests_names_regexp(std::move(tests_names_regexp_)) , tests_names_regexp(std::move(tests_names_regexp_))
@ -100,7 +101,7 @@ public:
UInt64 version_minor; UInt64 version_minor;
UInt64 version_patch; UInt64 version_patch;
UInt64 version_revision; UInt64 version_revision;
connection.getServerVersion(name, version_major, version_minor, version_patch, version_revision); connection.getServerVersion(timeouts, name, version_major, version_minor, version_patch, version_revision);
std::stringstream ss; std::stringstream ss;
ss << version_major << "." << version_minor << "." << version_patch; ss << version_major << "." << version_minor << "." << version_patch;
@ -115,6 +116,7 @@ public:
private: private:
Connection connection; Connection connection;
const ConnectionTimeouts & timeouts;
const Strings & tests_tags; const Strings & tests_tags;
const Strings & tests_names; const Strings & tests_names;
@ -195,7 +197,7 @@ private:
{ {
PerformanceTestInfo info(test_config, profiles_file, global_context.getSettingsRef()); PerformanceTestInfo info(test_config, profiles_file, global_context.getSettingsRef());
LOG_INFO(log, "Config for test '" << info.test_name << "' parsed"); LOG_INFO(log, "Config for test '" << info.test_name << "' parsed");
PerformanceTest current(test_config, connection, interrupt_listener, info, global_context, query_indexes[info.path]); PerformanceTest current(test_config, connection, timeouts, interrupt_listener, info, global_context, query_indexes[info.path]);
if (current.checkPreconditions()) if (current.checkPreconditions())
{ {

View File

@ -48,6 +48,7 @@ MySQLHandler::MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & so
void MySQLHandler::run() void MySQLHandler::run()
{ {
connection_context = server.context(); connection_context = server.context();
connection_context.setSessionContext(connection_context);
connection_context.setDefaultFormat("MySQLWire"); connection_context.setDefaultFormat("MySQLWire");
in = std::make_shared<ReadBufferFromPocoSocket>(socket()); in = std::make_shared<ReadBufferFromPocoSocket>(socket());
@ -306,7 +307,7 @@ void MySQLHandler::authenticate(const HandshakeResponse & handshake_response, co
try try
{ {
connection_context.setUser(handshake_response.username, password, socket().address(), ""); connection_context.setUser(handshake_response.username, password, socket().address(), "");
connection_context.setCurrentDatabase(handshake_response.database); if (!handshake_response.database.empty()) connection_context.setCurrentDatabase(handshake_response.database);
connection_context.setCurrentQueryId(""); connection_context.setCurrentQueryId("");
LOG_ERROR(log, "Authentication for user " << handshake_response.username << " succeeded."); LOG_ERROR(log, "Authentication for user " << handshake_response.username << " succeeded.");
} }

View File

@ -43,8 +43,8 @@ private:
size_t connection_id = 0; size_t connection_id = 0;
size_t server_capability_flags; size_t server_capability_flags = 0;
size_t client_capability_flags; size_t client_capability_flags = 0;
RSA & public_key; RSA & public_key;
RSA & private_key; RSA & private_key;

View File

@ -411,6 +411,7 @@ public:
return "covarSamp"; return "covarSamp";
if constexpr (StatFunc::kind == StatisticsFunctionKind::corr) if constexpr (StatFunc::kind == StatisticsFunctionKind::corr)
return "corr"; return "corr";
__builtin_unreachable();
} }
DataTypePtr getReturnType() const override DataTypePtr getReturnType() const override

View File

@ -48,7 +48,7 @@ namespace ErrorCodes
} }
void Connection::connect() void Connection::connect(const ConnectionTimeouts & timeouts)
{ {
try try
{ {
@ -230,10 +230,15 @@ UInt16 Connection::getPort() const
return port; return port;
} }
void Connection::getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & version_patch, UInt64 & revision) void Connection::getServerVersion(const ConnectionTimeouts & timeouts,
String & name,
UInt64 & version_major,
UInt64 & version_minor,
UInt64 & version_patch,
UInt64 & revision)
{ {
if (!connected) if (!connected)
connect(); connect(timeouts);
name = server_name; name = server_name;
version_major = server_version_major; version_major = server_version_major;
@ -242,40 +247,40 @@ void Connection::getServerVersion(String & name, UInt64 & version_major, UInt64
revision = server_revision; revision = server_revision;
} }
UInt64 Connection::getServerRevision() UInt64 Connection::getServerRevision(const ConnectionTimeouts & timeouts)
{ {
if (!connected) if (!connected)
connect(); connect(timeouts);
return server_revision; return server_revision;
} }
const String & Connection::getServerTimezone() const String & Connection::getServerTimezone(const ConnectionTimeouts & timeouts)
{ {
if (!connected) if (!connected)
connect(); connect(timeouts);
return server_timezone; return server_timezone;
} }
const String & Connection::getServerDisplayName() const String & Connection::getServerDisplayName(const ConnectionTimeouts & timeouts)
{ {
if (!connected) if (!connected)
connect(); connect(timeouts);
return server_display_name; return server_display_name;
} }
void Connection::forceConnected() void Connection::forceConnected(const ConnectionTimeouts & timeouts)
{ {
if (!connected) if (!connected)
{ {
connect(); connect(timeouts);
} }
else if (!ping()) else if (!ping())
{ {
LOG_TRACE(log_wrapper.get(), "Connection was closed, will reconnect."); LOG_TRACE(log_wrapper.get(), "Connection was closed, will reconnect.");
connect(); connect(timeouts);
} }
} }
@ -318,10 +323,11 @@ bool Connection::ping()
return true; return true;
} }
TablesStatusResponse Connection::getTablesStatus(const TablesStatusRequest & request) TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & timeouts,
const TablesStatusRequest & request)
{ {
if (!connected) if (!connected)
connect(); connect(timeouts);
TimeoutSetter timeout_setter(*socket, sync_request_timeout, true); TimeoutSetter timeout_setter(*socket, sync_request_timeout, true);
@ -344,6 +350,7 @@ TablesStatusResponse Connection::getTablesStatus(const TablesStatusRequest & req
void Connection::sendQuery( void Connection::sendQuery(
const ConnectionTimeouts & timeouts,
const String & query, const String & query,
const String & query_id_, const String & query_id_,
UInt64 stage, UInt64 stage,
@ -352,7 +359,9 @@ void Connection::sendQuery(
bool with_pending_data) bool with_pending_data)
{ {
if (!connected) if (!connected)
connect(); connect(timeouts);
TimeoutSetter timeout_setter(*socket, timeouts.send_timeout, timeouts.receive_timeout, true);
if (settings) if (settings)
{ {

View File

@ -57,7 +57,6 @@ public:
Connection(const String & host_, UInt16 port_, Connection(const String & host_, UInt16 port_,
const String & default_database_, const String & default_database_,
const String & user_, const String & password_, const String & user_, const String & password_,
const ConnectionTimeouts & timeouts_,
const String & client_name_ = "client", const String & client_name_ = "client",
Protocol::Compression compression_ = Protocol::Compression::Enable, Protocol::Compression compression_ = Protocol::Compression::Enable,
Protocol::Secure secure_ = Protocol::Secure::Disable, Protocol::Secure secure_ = Protocol::Secure::Disable,
@ -68,7 +67,6 @@ public:
client_name(client_name_), client_name(client_name_),
compression(compression_), compression(compression_),
secure(secure_), secure(secure_),
timeouts(timeouts_),
sync_request_timeout(sync_request_timeout_), sync_request_timeout(sync_request_timeout_),
log_wrapper(*this) log_wrapper(*this)
{ {
@ -106,11 +104,16 @@ public:
/// Change default database. Changes will take effect on next reconnect. /// Change default database. Changes will take effect on next reconnect.
void setDefaultDatabase(const String & database); void setDefaultDatabase(const String & database);
void getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & version_patch, UInt64 & revision); void getServerVersion(const ConnectionTimeouts & timeouts,
UInt64 getServerRevision(); String & name,
UInt64 & version_major,
UInt64 & version_minor,
UInt64 & version_patch,
UInt64 & revision);
UInt64 getServerRevision(const ConnectionTimeouts & timeouts);
const String & getServerTimezone(); const String & getServerTimezone(const ConnectionTimeouts & timeouts);
const String & getServerDisplayName(); const String & getServerDisplayName(const ConnectionTimeouts & timeouts);
/// For log and exception messages. /// For log and exception messages.
const String & getDescription() const; const String & getDescription() const;
@ -118,14 +121,9 @@ public:
UInt16 getPort() const; UInt16 getPort() const;
const String & getDefaultDatabase() const; const String & getDefaultDatabase() const;
/// For proper polling.
inline const auto & getTimeouts() const
{
return timeouts;
}
/// If last flag is true, you need to call sendExternalTablesData after. /// If last flag is true, you need to call sendExternalTablesData after.
void sendQuery( void sendQuery(
const ConnectionTimeouts & timeouts,
const String & query, const String & query,
const String & query_id_ = "", const String & query_id_ = "",
UInt64 stage = QueryProcessingStage::Complete, UInt64 stage = QueryProcessingStage::Complete,
@ -156,9 +154,10 @@ public:
Packet receivePacket(); Packet receivePacket();
/// If not connected yet, or if connection is broken - then connect. If cannot connect - throw an exception. /// If not connected yet, or if connection is broken - then connect. If cannot connect - throw an exception.
void forceConnected(); void forceConnected(const ConnectionTimeouts & timeouts);
TablesStatusResponse getTablesStatus(const TablesStatusRequest & request); TablesStatusResponse getTablesStatus(const ConnectionTimeouts & timeouts,
const TablesStatusRequest & request);
/** Disconnect. /** Disconnect.
* This may be used, if connection is left in unsynchronised state * This may be used, if connection is left in unsynchronised state
@ -216,7 +215,6 @@ private:
*/ */
ThrottlerPtr throttler; ThrottlerPtr throttler;
ConnectionTimeouts timeouts;
Poco::Timespan sync_request_timeout; Poco::Timespan sync_request_timeout;
/// From where to read query execution result. /// From where to read query execution result.
@ -252,7 +250,7 @@ private:
LoggerWrapper log_wrapper; LoggerWrapper log_wrapper;
void connect(); void connect(const ConnectionTimeouts & timeouts);
void sendHello(); void sendHello();
void receiveHello(); void receiveHello();
bool ping(); bool ping();

View File

@ -30,7 +30,9 @@ public:
/// Selects the connection to work. /// Selects the connection to work.
/// If force_connected is false, the client must manually ensure that returned connection is good. /// If force_connected is false, the client must manually ensure that returned connection is good.
virtual Entry get(const Settings * settings = nullptr, bool force_connected = true) = 0; virtual Entry get(const ConnectionTimeouts & timeouts,
const Settings * settings = nullptr,
bool force_connected = true) = 0;
}; };
using ConnectionPoolPtr = std::shared_ptr<IConnectionPool>; using ConnectionPoolPtr = std::shared_ptr<IConnectionPool>;
@ -50,7 +52,6 @@ public:
const String & default_database_, const String & default_database_,
const String & user_, const String & user_,
const String & password_, const String & password_,
const ConnectionTimeouts & timeouts,
const String & client_name_ = "client", const String & client_name_ = "client",
Protocol::Compression compression_ = Protocol::Compression::Enable, Protocol::Compression compression_ = Protocol::Compression::Enable,
Protocol::Secure secure_ = Protocol::Secure::Disable) Protocol::Secure secure_ = Protocol::Secure::Disable)
@ -63,12 +64,13 @@ public:
password(password_), password(password_),
client_name(client_name_), client_name(client_name_),
compression(compression_), compression(compression_),
secure{secure_}, secure{secure_}
timeouts(timeouts)
{ {
} }
Entry get(const Settings * settings = nullptr, bool force_connected = true) override Entry get(const ConnectionTimeouts & timeouts,
const Settings * settings = nullptr,
bool force_connected = true) override
{ {
Entry entry; Entry entry;
if (settings) if (settings)
@ -77,7 +79,7 @@ public:
entry = Base::get(-1); entry = Base::get(-1);
if (force_connected) if (force_connected)
entry->forceConnected(); entry->forceConnected(timeouts);
return entry; return entry;
} }
@ -93,7 +95,7 @@ protected:
{ {
return std::make_shared<Connection>( return std::make_shared<Connection>(
host, port, host, port,
default_database, user, password, timeouts, default_database, user, password,
client_name, compression, secure); client_name, compression, secure);
} }
@ -108,7 +110,6 @@ private:
Protocol::Compression compression; /// Whether to compress data when interacting with the server. Protocol::Compression compression; /// Whether to compress data when interacting with the server.
Protocol::Secure secure; /// Whether to encrypt data when interacting with the server. Protocol::Secure secure; /// Whether to encrypt data when interacting with the server.
ConnectionTimeouts timeouts;
}; };
} }

View File

@ -8,6 +8,8 @@
#include <Common/ProfileEvents.h> #include <Common/ProfileEvents.h>
#include <Core/Settings.h> #include <Core/Settings.h>
#include <IO/ConnectionTimeouts.h>
namespace ProfileEvents namespace ProfileEvents
{ {
@ -29,9 +31,8 @@ namespace ErrorCodes
ConnectionPoolWithFailover::ConnectionPoolWithFailover( ConnectionPoolWithFailover::ConnectionPoolWithFailover(
ConnectionPoolPtrs nested_pools_, ConnectionPoolPtrs nested_pools_,
LoadBalancing load_balancing, LoadBalancing load_balancing,
size_t max_tries_,
time_t decrease_error_period_) time_t decrease_error_period_)
: Base(std::move(nested_pools_), max_tries_, decrease_error_period_, &Logger::get("ConnectionPoolWithFailover")) : Base(std::move(nested_pools_), decrease_error_period_, &Logger::get("ConnectionPoolWithFailover"))
, default_load_balancing(load_balancing) , default_load_balancing(load_balancing)
{ {
const std::string & local_hostname = getFQDNOrHostName(); const std::string & local_hostname = getFQDNOrHostName();
@ -44,11 +45,13 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover(
} }
} }
IConnectionPool::Entry ConnectionPoolWithFailover::get(const Settings * settings, bool /*force_connected*/) IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts & timeouts,
const Settings * settings,
bool /*force_connected*/)
{ {
TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message) TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message)
{ {
return tryGetEntry(pool, fail_message, settings); return tryGetEntry(pool, timeouts, fail_message, settings);
}; };
GetPriorityFunc get_priority; GetPriorityFunc get_priority;
@ -70,11 +73,13 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const Settings * settings
return Base::get(try_get_entry, get_priority); return Base::get(try_get_entry, get_priority);
} }
std::vector<IConnectionPool::Entry> ConnectionPoolWithFailover::getMany(const Settings * settings, PoolMode pool_mode) std::vector<IConnectionPool::Entry> ConnectionPoolWithFailover::getMany(const ConnectionTimeouts & timeouts,
const Settings * settings,
PoolMode pool_mode)
{ {
TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message) TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message)
{ {
return tryGetEntry(pool, fail_message, settings); return tryGetEntry(pool, timeouts, fail_message, settings);
}; };
std::vector<TryResult> results = getManyImpl(settings, pool_mode, try_get_entry); std::vector<TryResult> results = getManyImpl(settings, pool_mode, try_get_entry);
@ -86,22 +91,27 @@ std::vector<IConnectionPool::Entry> ConnectionPoolWithFailover::getMany(const Se
return entries; return entries;
} }
std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::getManyForTableFunction(const Settings * settings, PoolMode pool_mode) std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::getManyForTableFunction(
const ConnectionTimeouts & timeouts,
const Settings * settings,
PoolMode pool_mode)
{ {
TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message) TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message)
{ {
return tryGetEntry(pool, fail_message, settings); return tryGetEntry(pool, timeouts, fail_message, settings);
}; };
return getManyImpl(settings, pool_mode, try_get_entry); return getManyImpl(settings, pool_mode, try_get_entry);
} }
std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::getManyChecked( std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::getManyChecked(
const Settings * settings, PoolMode pool_mode, const QualifiedTableName & table_to_check) const ConnectionTimeouts & timeouts,
const Settings * settings, PoolMode pool_mode,
const QualifiedTableName & table_to_check)
{ {
TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message) TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message)
{ {
return tryGetEntry(pool, fail_message, settings, &table_to_check); return tryGetEntry(pool, timeouts, fail_message, settings, &table_to_check);
}; };
return getManyImpl(settings, pool_mode, try_get_entry); return getManyImpl(settings, pool_mode, try_get_entry);
@ -113,6 +123,9 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
const TryGetEntryFunc & try_get_entry) const TryGetEntryFunc & try_get_entry)
{ {
size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1; size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1;
size_t max_tries = (settings ?
size_t{settings->connections_with_failover_max_tries} :
size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES});
size_t max_entries; size_t max_entries;
if (pool_mode == PoolMode::GET_ALL) if (pool_mode == PoolMode::GET_ALL)
{ {
@ -144,12 +157,13 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
bool fallback_to_stale_replicas = settings ? bool(settings->fallback_to_stale_replicas_for_distributed_queries) : true; bool fallback_to_stale_replicas = settings ? bool(settings->fallback_to_stale_replicas_for_distributed_queries) : true;
return Base::getMany(min_entries, max_entries, try_get_entry, get_priority, fallback_to_stale_replicas); return Base::getMany(min_entries, max_entries, max_tries, try_get_entry, get_priority, fallback_to_stale_replicas);
} }
ConnectionPoolWithFailover::TryResult ConnectionPoolWithFailover::TryResult
ConnectionPoolWithFailover::tryGetEntry( ConnectionPoolWithFailover::tryGetEntry(
IConnectionPool & pool, IConnectionPool & pool,
const ConnectionTimeouts & timeouts,
std::string & fail_message, std::string & fail_message,
const Settings * settings, const Settings * settings,
const QualifiedTableName * table_to_check) const QualifiedTableName * table_to_check)
@ -157,15 +171,15 @@ ConnectionPoolWithFailover::tryGetEntry(
TryResult result; TryResult result;
try try
{ {
result.entry = pool.get(settings, /* force_connected = */ false); result.entry = pool.get(timeouts, settings, /* force_connected = */ false);
UInt64 server_revision = 0; UInt64 server_revision = 0;
if (table_to_check) if (table_to_check)
server_revision = result.entry->getServerRevision(); server_revision = result.entry->getServerRevision(timeouts);
if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS)
{ {
result.entry->forceConnected(); result.entry->forceConnected(timeouts);
result.is_usable = true; result.is_usable = true;
result.is_up_to_date = true; result.is_up_to_date = true;
return result; return result;
@ -176,7 +190,7 @@ ConnectionPoolWithFailover::tryGetEntry(
TablesStatusRequest status_request; TablesStatusRequest status_request;
status_request.tables.emplace(*table_to_check); status_request.tables.emplace(*table_to_check);
TablesStatusResponse status_response = result.entry->getTablesStatus(status_request); TablesStatusResponse status_response = result.entry->getTablesStatus(timeouts, status_request);
auto table_status_it = status_response.table_states_by_id.find(*table_to_check); auto table_status_it = status_response.table_states_by_id.find(*table_to_check);
if (table_status_it == status_response.table_states_by_id.end()) if (table_status_it == status_response.table_states_by_id.end())
{ {

View File

@ -34,21 +34,24 @@ public:
ConnectionPoolWithFailover( ConnectionPoolWithFailover(
ConnectionPoolPtrs nested_pools_, ConnectionPoolPtrs nested_pools_,
LoadBalancing load_balancing, LoadBalancing load_balancing,
size_t max_tries_ = DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES,
time_t decrease_error_period_ = DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD); time_t decrease_error_period_ = DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD);
using Entry = IConnectionPool::Entry; using Entry = IConnectionPool::Entry;
/** Allocates connection to work. */ /** Allocates connection to work. */
Entry get(const Settings * settings = nullptr, bool force_connected = true) override; /// From IConnectionPool Entry get(const ConnectionTimeouts & timeouts,
const Settings * settings = nullptr,
bool force_connected = true) override; /// From IConnectionPool
/** Allocates up to the specified number of connections to work. /** Allocates up to the specified number of connections to work.
* Connections provide access to different replicas of one shard. * Connections provide access to different replicas of one shard.
*/ */
std::vector<Entry> getMany(const Settings * settings, PoolMode pool_mode); std::vector<Entry> getMany(const ConnectionTimeouts & timeouts,
const Settings * settings, PoolMode pool_mode);
/// The same as getMany(), but return std::vector<TryResult>. /// The same as getMany(), but return std::vector<TryResult>.
std::vector<TryResult> getManyForTableFunction(const Settings * settings, PoolMode pool_mode); std::vector<TryResult> getManyForTableFunction(const ConnectionTimeouts & timeouts,
const Settings * settings, PoolMode pool_mode);
using Base = PoolWithFailoverBase<IConnectionPool>; using Base = PoolWithFailoverBase<IConnectionPool>;
using TryResult = Base::TryResult; using TryResult = Base::TryResult;
@ -56,7 +59,10 @@ public:
/// The same as getMany(), but check that replication delay for table_to_check is acceptable. /// The same as getMany(), but check that replication delay for table_to_check is acceptable.
/// Delay threshold is taken from settings. /// Delay threshold is taken from settings.
std::vector<TryResult> getManyChecked( std::vector<TryResult> getManyChecked(
const Settings * settings, PoolMode pool_mode, const QualifiedTableName & table_to_check); const ConnectionTimeouts & timeouts,
const Settings * settings,
PoolMode pool_mode,
const QualifiedTableName & table_to_check);
private: private:
/// Get the values of relevant settings and call Base::getMany() /// Get the values of relevant settings and call Base::getMany()
@ -70,6 +76,7 @@ private:
/// for this table is not too large. /// for this table is not too large.
TryResult tryGetEntry( TryResult tryGetEntry(
IConnectionPool & pool, IConnectionPool & pool,
const ConnectionTimeouts & timeouts,
std::string & fail_message, std::string & fail_message,
const Settings * settings, const Settings * settings,
const QualifiedTableName * table_to_check = nullptr); const QualifiedTableName * table_to_check = nullptr);

View File

@ -1,4 +1,5 @@
#include <Client/MultiplexedConnections.h> #include <Client/MultiplexedConnections.h>
#include <IO/ConnectionTimeouts.h>
namespace DB namespace DB
{ {
@ -73,6 +74,7 @@ void MultiplexedConnections::sendExternalTablesData(std::vector<ExternalTablesDa
} }
void MultiplexedConnections::sendQuery( void MultiplexedConnections::sendQuery(
const ConnectionTimeouts & timeouts,
const String & query, const String & query,
const String & query_id, const String & query_id,
UInt64 stage, UInt64 stage,
@ -91,7 +93,7 @@ void MultiplexedConnections::sendQuery(
if (!replica.connection) if (!replica.connection)
throw Exception("MultiplexedConnections: Internal error", ErrorCodes::LOGICAL_ERROR); throw Exception("MultiplexedConnections: Internal error", ErrorCodes::LOGICAL_ERROR);
if (replica.connection->getServerRevision() < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) if (replica.connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD)
{ {
/// Disable two-level aggregation due to version incompatibility. /// Disable two-level aggregation due to version incompatibility.
modified_settings.group_by_two_level_threshold = 0; modified_settings.group_by_two_level_threshold = 0;
@ -107,13 +109,15 @@ void MultiplexedConnections::sendQuery(
for (size_t i = 0; i < num_replicas; ++i) for (size_t i = 0; i < num_replicas; ++i)
{ {
modified_settings.parallel_replica_offset = i; modified_settings.parallel_replica_offset = i;
replica_states[i].connection->sendQuery(query, query_id, stage, &modified_settings, client_info, with_pending_data); replica_states[i].connection->sendQuery(timeouts, query, query_id,
stage, &modified_settings, client_info, with_pending_data);
} }
} }
else else
{ {
/// Use single replica. /// Use single replica.
replica_states[0].connection->sendQuery(query, query_id, stage, &modified_settings, client_info, with_pending_data); replica_states[0].connection->sendQuery(timeouts, query, query_id, stage,
&modified_settings, client_info, with_pending_data);
} }
sent_query = true; sent_query = true;

View File

@ -1,9 +1,10 @@
#pragma once #pragma once
#include <mutex>
#include <Common/Throttler.h> #include <Common/Throttler.h>
#include <Client/Connection.h> #include <Client/Connection.h>
#include <Client/ConnectionPoolWithFailover.h> #include <Client/ConnectionPoolWithFailover.h>
#include <mutex> #include <IO/ConnectionTimeouts.h>
namespace DB namespace DB
{ {
@ -31,6 +32,7 @@ public:
/// Send request to replicas. /// Send request to replicas.
void sendQuery( void sendQuery(
const ConnectionTimeouts & timeouts,
const String & query, const String & query,
const String & query_id = "", const String & query_id = "",
UInt64 stage = QueryProcessingStage::Complete, UInt64 stage = QueryProcessingStage::Complete,

View File

@ -430,6 +430,7 @@ namespace ErrorCodes
extern const int MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES = 453; extern const int MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES = 453;
extern const int OPENSSL_ERROR = 454; extern const int OPENSSL_ERROR = 454;
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY = 455; extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY = 455;
extern const int CANNOT_UNLINK = 458;
extern const int KEEPER_EXCEPTION = 999; extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000; extern const int POCO_EXCEPTION = 1000;

View File

@ -55,11 +55,9 @@ public:
PoolWithFailoverBase( PoolWithFailoverBase(
NestedPools nested_pools_, NestedPools nested_pools_,
size_t max_tries_,
time_t decrease_error_period_, time_t decrease_error_period_,
Logger * log_) Logger * log_)
: nested_pools(std::move(nested_pools_)) : nested_pools(std::move(nested_pools_))
, max_tries(max_tries_)
, decrease_error_period(decrease_error_period_) , decrease_error_period(decrease_error_period_)
, shared_pool_states(nested_pools.size()) , shared_pool_states(nested_pools.size())
, log(log_) , log(log_)
@ -108,7 +106,7 @@ public:
/// The method will throw if it is unable to get min_entries alive connections or /// The method will throw if it is unable to get min_entries alive connections or
/// if fallback_to_stale_replicas is false and it is unable to get min_entries connections to up-to-date replicas. /// if fallback_to_stale_replicas is false and it is unable to get min_entries connections to up-to-date replicas.
std::vector<TryResult> getMany( std::vector<TryResult> getMany(
size_t min_entries, size_t max_entries, size_t min_entries, size_t max_entries, size_t max_tries,
const TryGetEntryFunc & try_get_entry, const TryGetEntryFunc & try_get_entry,
const GetPriorityFunc & get_priority = GetPriorityFunc(), const GetPriorityFunc & get_priority = GetPriorityFunc(),
bool fallback_to_stale_replicas = true); bool fallback_to_stale_replicas = true);
@ -125,8 +123,6 @@ protected:
NestedPools nested_pools; NestedPools nested_pools;
const size_t max_tries;
const time_t decrease_error_period; const time_t decrease_error_period;
std::mutex pool_states_mutex; std::mutex pool_states_mutex;
@ -141,7 +137,7 @@ template <typename TNestedPool>
typename TNestedPool::Entry typename TNestedPool::Entry
PoolWithFailoverBase<TNestedPool>::get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority) PoolWithFailoverBase<TNestedPool>::get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority)
{ {
std::vector<TryResult> results = getMany(1, 1, try_get_entry, get_priority); std::vector<TryResult> results = getMany(1, 1, 1, try_get_entry, get_priority);
if (results.empty() || results[0].entry.isNull()) if (results.empty() || results[0].entry.isNull())
throw DB::Exception( throw DB::Exception(
"PoolWithFailoverBase::getMany() returned less than min_entries entries.", "PoolWithFailoverBase::getMany() returned less than min_entries entries.",
@ -152,7 +148,7 @@ PoolWithFailoverBase<TNestedPool>::get(const TryGetEntryFunc & try_get_entry, co
template <typename TNestedPool> template <typename TNestedPool>
std::vector<typename PoolWithFailoverBase<TNestedPool>::TryResult> std::vector<typename PoolWithFailoverBase<TNestedPool>::TryResult>
PoolWithFailoverBase<TNestedPool>::getMany( PoolWithFailoverBase<TNestedPool>::getMany(
size_t min_entries, size_t max_entries, size_t min_entries, size_t max_entries, size_t max_tries,
const TryGetEntryFunc & try_get_entry, const TryGetEntryFunc & try_get_entry,
const GetPriorityFunc & get_priority, const GetPriorityFunc & get_priority,
bool fallback_to_stale_replicas) bool fallback_to_stale_replicas)
@ -167,9 +163,9 @@ PoolWithFailoverBase<TNestedPool>::getMany(
struct ShuffledPool struct ShuffledPool
{ {
NestedPool * pool; NestedPool * pool{};
const PoolState * state; const PoolState * state{};
size_t index; size_t index = 0;
size_t error_count = 0; size_t error_count = 0;
}; };
@ -192,7 +188,7 @@ PoolWithFailoverBase<TNestedPool>::getMany(
size_t up_to_date_count = 0; size_t up_to_date_count = 0;
size_t failed_pools_count = 0; size_t failed_pools_count = 0;
/// At exit update shared error counts with error counts occured during this call. /// At exit update shared error counts with error counts occurred during this call.
SCOPE_EXIT( SCOPE_EXIT(
{ {
std::lock_guard lock(pool_states_mutex); std::lock_guard lock(pool_states_mutex);

View File

@ -90,12 +90,17 @@ struct NetlinkMessage
const char * request_buf = reinterpret_cast<const char *>(this); const char * request_buf = reinterpret_cast<const char *>(this);
ssize_t request_size = header.nlmsg_len; ssize_t request_size = header.nlmsg_len;
union
{
::sockaddr_nl nladdr{}; ::sockaddr_nl nladdr{};
::sockaddr sockaddr;
};
nladdr.nl_family = AF_NETLINK; nladdr.nl_family = AF_NETLINK;
while (true) while (true)
{ {
ssize_t bytes_sent = ::sendto(fd, request_buf, request_size, 0, reinterpret_cast<const ::sockaddr *>(&nladdr), sizeof(nladdr)); ssize_t bytes_sent = ::sendto(fd, request_buf, request_size, 0, &sockaddr, sizeof(nladdr));
if (bytes_sent <= 0) if (bytes_sent <= 0)
{ {
@ -236,10 +241,14 @@ TaskStatsInfoGetter::TaskStatsInfoGetter()
if (0 != ::setsockopt(netlink_socket_fd, SOL_SOCKET, SO_RCVTIMEO, reinterpret_cast<const char *>(&tv), sizeof(tv))) if (0 != ::setsockopt(netlink_socket_fd, SOL_SOCKET, SO_RCVTIMEO, reinterpret_cast<const char *>(&tv), sizeof(tv)))
throwFromErrno("Can't set timeout on PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); throwFromErrno("Can't set timeout on PF_NETLINK socket", ErrorCodes::NETLINK_ERROR);
union
{
::sockaddr_nl addr{}; ::sockaddr_nl addr{};
::sockaddr sockaddr;
};
addr.nl_family = AF_NETLINK; addr.nl_family = AF_NETLINK;
if (::bind(netlink_socket_fd, reinterpret_cast<const ::sockaddr *>(&addr), sizeof(addr)) < 0) if (::bind(netlink_socket_fd, &sockaddr, sizeof(addr)) < 0)
throwFromErrno("Can't bind PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); throwFromErrno("Can't bind PF_NETLINK socket", ErrorCodes::NETLINK_ERROR);
taskstats_family_id = getFamilyId(netlink_socket_fd); taskstats_family_id = getFamilyId(netlink_socket_fd);

View File

@ -187,8 +187,8 @@ namespace VolnitskyTraits
/// put ngram for uppercase /// put ngram for uppercase
utf8.convert(u_u32, seq, sizeof(seq)); utf8.convert(u_u32, seq, sizeof(seq));
chars.c0 = seq[seq_ngram_offset]; chars.c0 = seq[seq_ngram_offset]; //-V519
chars.c1 = seq[seq_ngram_offset + 1]; chars.c1 = seq[seq_ngram_offset + 1]; //-V519
putNGramBase(n, offset); putNGramBase(n, offset);
} }
} }

View File

@ -8,17 +8,9 @@
#include <Common/ThreadPool.h> #include <Common/ThreadPool.h>
int x = 0; int value = 0;
void f() { ++x; }
/*void f()
{
std::vector<std::string> vec;
for (size_t i = 0; i < 100; ++i)
vec.push_back(std::string(rand() % 10, ' '));
}*/
void f() { ++value; }
void * g(void *) { f(); return {}; } void * g(void *) { f(); return {}; }
@ -34,7 +26,7 @@ namespace DB
template <typename F> template <typename F>
void test(size_t n, const char * name, F && kernel) void test(size_t n, const char * name, F && kernel)
{ {
x = 0; value = 0;
Stopwatch watch; Stopwatch watch;
Stopwatch watch_one; Stopwatch watch_one;
@ -62,7 +54,7 @@ void test(size_t n, const char * name, F && kernel)
<< n / watch.elapsedSeconds() << " ops/sec., " << n / watch.elapsedSeconds() << " ops/sec., "
<< "avg latency: " << watch.elapsedSeconds() / n * 1000000 << " μs, " << "avg latency: " << watch.elapsedSeconds() / n * 1000000 << " μs, "
<< "max latency: " << max_seconds * 1000000 << " μs " << "max latency: " << max_seconds * 1000000 << " μs "
<< "(res = " << x << ")" << "(res = " << value << ")"
<< std::endl; << std::endl;
} }
@ -71,13 +63,6 @@ int main(int argc, char ** argv)
{ {
size_t n = argc == 2 ? DB::parse<UInt64>(argv[1]) : 100000; size_t n = argc == 2 ? DB::parse<UInt64>(argv[1]) : 100000;
/* test(n, "Create and destroy boost::threadpool each iteration", []
{
boost::threadpool::pool tp(1);
tp.schedule(f);
tp.wait();
});*/
test(n, "Create and destroy ThreadPool each iteration", [] test(n, "Create and destroy ThreadPool each iteration", []
{ {
ThreadPool tp(1); ThreadPool tp(1);
@ -100,16 +85,6 @@ int main(int argc, char ** argv)
thread.join(); thread.join();
}); });
/* {
boost::threadpool::pool tp(1);
test(n, "Schedule job for boost::threadpool each iteration", [&tp]
{
tp.schedule(f);
tp.wait();
});
}*/
{ {
ThreadPool tp(1); ThreadPool tp(1);
@ -120,16 +95,6 @@ int main(int argc, char ** argv)
}); });
} }
/* {
boost::threadpool::pool tp(128);
test(n, "Schedule job for boost::threadpool with 128 threads each iteration", [&tp]
{
tp.schedule(f);
tp.wait();
});
}*/
{ {
ThreadPool tp(128); ThreadPool tp(128);

View File

@ -0,0 +1,520 @@
#include <cstring>
#include <Compression/CompressionCodecT64.h>
#include <Compression/CompressionFactory.h>
#include <common/unaligned.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;
extern const int CANNOT_DECOMPRESS;
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int ILLEGAL_CODEC_PARAMETER;
extern const int LOGICAL_ERROR;
}
namespace
{
UInt8 codecId()
{
return static_cast<UInt8>(CompressionMethodByte::T64);
}
TypeIndex baseType(TypeIndex type_idx)
{
switch (type_idx)
{
case TypeIndex::Int8:
return TypeIndex::Int8;
case TypeIndex::Int16:
return TypeIndex::Int16;
case TypeIndex::Int32:
case TypeIndex::Decimal32:
return TypeIndex::Int32;
case TypeIndex::Int64:
case TypeIndex::Decimal64:
return TypeIndex::Int64;
case TypeIndex::UInt8:
case TypeIndex::Enum8:
return TypeIndex::UInt8;
case TypeIndex::UInt16:
case TypeIndex::Enum16:
case TypeIndex::Date:
return TypeIndex::UInt16;
case TypeIndex::UInt32:
case TypeIndex::DateTime:
return TypeIndex::UInt32;
case TypeIndex::UInt64:
return TypeIndex::UInt64;
default:
break;
}
return TypeIndex::Nothing;
}
TypeIndex typeIdx(const DataTypePtr & data_type)
{
if (!data_type)
return TypeIndex::Nothing;
WhichDataType which(*data_type);
switch (which.idx)
{
case TypeIndex::Int8:
case TypeIndex::UInt8:
case TypeIndex::Enum8:
case TypeIndex::Int16:
case TypeIndex::UInt16:
case TypeIndex::Enum16:
case TypeIndex::Date:
case TypeIndex::Int32:
case TypeIndex::UInt32:
case TypeIndex::DateTime:
case TypeIndex::Decimal32:
case TypeIndex::Int64:
case TypeIndex::UInt64:
case TypeIndex::Decimal64:
return which.idx;
default:
break;
}
return TypeIndex::Nothing;
}
void transpose64x8(const UInt64 * src, UInt64 * dst, UInt32)
{
auto * src8 = reinterpret_cast<const UInt8 *>(src);
for (UInt32 i = 0; i < 64; ++i)
{
UInt64 value = src8[i];
dst[0] |= (value & 0x1) << i;
dst[1] |= ((value >> 1) & 0x1) << i;
dst[2] |= ((value >> 2) & 0x1) << i;
dst[3] |= ((value >> 3) & 0x1) << i;
dst[4] |= ((value >> 4) & 0x1) << i;
dst[5] |= ((value >> 5) & 0x1) << i;
dst[6] |= ((value >> 6) & 0x1) << i;
dst[7] |= ((value >> 7) & 0x1) << i;
}
}
void revTranspose64x8(const UInt64 * src, UInt64 * dst, UInt32)
{
auto * dst8 = reinterpret_cast<UInt8 *>(dst);
for (UInt32 i = 0; i < 64; ++i)
{
dst8[i] = ((src[0] >> i) & 0x1)
| (((src[1] >> i) & 0x1) << 1)
| (((src[2] >> i) & 0x1) << 2)
| (((src[3] >> i) & 0x1) << 3)
| (((src[4] >> i) & 0x1) << 4)
| (((src[5] >> i) & 0x1) << 5)
| (((src[6] >> i) & 0x1) << 6)
| (((src[7] >> i) & 0x1) << 7);
}
}
template <typename _T>
void transposeBytes(_T value, UInt64 * mx, UInt32 col)
{
UInt8 * mx8 = reinterpret_cast<UInt8 *>(mx);
const UInt8 * value8 = reinterpret_cast<const UInt8 *>(&value);
if constexpr (sizeof(_T) > 4)
{
mx8[64 * 7 + col] = value8[7];
mx8[64 * 6 + col] = value8[6];
mx8[64 * 5 + col] = value8[5];
mx8[64 * 4 + col] = value8[4];
}
if constexpr (sizeof(_T) > 2)
{
mx8[64 * 3 + col] = value8[3];
mx8[64 * 2 + col] = value8[2];
}
if constexpr (sizeof(_T) > 1)
mx8[64 * 1 + col] = value8[1];
mx8[64 * 0 + col] = value8[0];
}
template <typename _T>
void revTransposeBytes(const UInt64 * mx, UInt32 col, _T & value)
{
auto * mx8 = reinterpret_cast<const UInt8 *>(mx);
if constexpr (sizeof(_T) > 4)
{
value |= UInt64(mx8[64 * 7 + col]) << (8 * 7);
value |= UInt64(mx8[64 * 6 + col]) << (8 * 6);
value |= UInt64(mx8[64 * 5 + col]) << (8 * 5);
value |= UInt64(mx8[64 * 4 + col]) << (8 * 4);
}
if constexpr (sizeof(_T) > 2)
{
value |= UInt32(mx8[64 * 3 + col]) << (8 * 3);
value |= UInt32(mx8[64 * 2 + col]) << (8 * 2);
}
if constexpr (sizeof(_T) > 1)
value |= UInt32(mx8[64 * 1 + col]) << (8 * 1);
value |= UInt32(mx8[col]);
}
/// UIntX[64] -> UInt64[N] transposed matrix, N <= X
template <typename _T>
void transpose(const char * src, char * dst, UInt32 num_bits, UInt32 tail = 64)
{
UInt32 full_bytes = num_bits / 8;
UInt32 part_bits = num_bits % 8;
UInt64 mx[64] = {};
const char * ptr = src;
for (UInt32 col = 0; col < tail; ++col, ptr += sizeof(_T))
{
_T value = unalignedLoad<_T>(ptr);
transposeBytes(value, mx, col);
}
UInt32 full_size = sizeof(UInt64) * (num_bits - part_bits);
memcpy(dst, mx, full_size);
dst += full_size;
/// transpose only partially filled last byte
if (part_bits)
{
UInt64 * partial = &mx[full_bytes * 8];
UInt64 res[8] = {};
transpose64x8(partial, res, part_bits);
memcpy(dst, res, part_bits * sizeof(UInt64));
}
}
/// UInt64[N] transposed matrix -> UIntX[64]
template <typename _T, typename _MinMaxT = std::conditional_t<std::is_signed_v<_T>, Int64, UInt64>>
void revTranspose(const char * src, char * dst, UInt32 num_bits, _MinMaxT min, _MinMaxT max [[maybe_unused]], UInt32 tail = 64)
{
UInt64 mx[64] = {};
memcpy(mx, src, num_bits * sizeof(UInt64));
UInt32 full_bytes = num_bits / 8;
UInt32 part_bits = num_bits % 8;
UInt64 * partial = &mx[full_bytes * 8];
if (part_bits)
{
UInt64 res[8] = {};
revTranspose64x8(partial, res, part_bits);
memcpy(partial, res, 8 * sizeof(UInt64));
}
_T upper_min = 0;
if (num_bits < 64)
upper_min = UInt64(min) >> num_bits << num_bits;
_T buf[64] = {};
if constexpr (std::is_signed_v<_T>)
{
/// Restore some data as negatives and others as positives
if (min < 0 && max >= 0 && num_bits < 64)
{
_T sign_bit = 1ull << (num_bits - 1);
_T upper_max = UInt64(max) >> num_bits << num_bits;
for (UInt32 col = 0; col < tail; ++col)
{
_T & value = buf[col];
revTransposeBytes(mx, col, value);
if (value & sign_bit)
value |= upper_min;
else
value |= upper_max;
}
memcpy(dst, buf, tail * sizeof(_T));
return;
}
}
for (UInt32 col = 0; col < tail; ++col)
{
_T & value = buf[col];
revTransposeBytes(mx, col, value);
value |= upper_min;
}
memcpy(dst, buf, tail * sizeof(_T));
}
UInt32 getValuableBitsNumber(UInt64 min, UInt64 max)
{
UInt64 diff_bits = min ^ max;
if (diff_bits)
return 64 - __builtin_clzll(diff_bits);
return 0;
}
UInt32 getValuableBitsNumber(Int64 min, Int64 max)
{
if (min < 0 && max >= 0)
{
if (min + max >= 0)
return getValuableBitsNumber(0ull, UInt64(max)) + 1;
else
return getValuableBitsNumber(0ull, UInt64(~min)) + 1;
}
else
return getValuableBitsNumber(UInt64(min), UInt64(max));
}
template <typename _T>
void findMinMax(const char * src, UInt32 src_size, _T & min, _T & max)
{
min = unalignedLoad<_T>(src);
max = unalignedLoad<_T>(src);
const char * end = src + src_size;
for (; src < end; src += sizeof(_T))
{
auto current = unalignedLoad<_T>(src);
if (current < min)
min = current;
if (current > max)
max = current;
}
}
template <typename _T>
UInt32 compressData(const char * src, UInt32 bytes_size, char * dst)
{
using MinMaxType = std::conditional_t<std::is_signed_v<_T>, Int64, UInt64>;
const UInt32 mx_size = 64;
const UInt32 header_size = 2 * sizeof(UInt64);
if (bytes_size % sizeof(_T))
throw Exception("Cannot compress, data size " + toString(bytes_size) + " is not multiplier of " + toString(sizeof(_T)),
ErrorCodes::CANNOT_COMPRESS);
UInt32 src_size = bytes_size / sizeof(_T);
UInt32 num_full = src_size / mx_size;
UInt32 tail = src_size % mx_size;
_T min, max;
findMinMax<_T>(src, bytes_size, min, max);
MinMaxType min64 = min;
MinMaxType max64 = max;
/// Write header
{
memcpy(dst, &min64, sizeof(MinMaxType));
memcpy(dst + 8, &max64, sizeof(MinMaxType));
dst += header_size;
}
UInt32 num_bits = getValuableBitsNumber(min64, max64);
if (!num_bits)
return header_size;
UInt32 src_shift = sizeof(_T) * mx_size;
UInt32 dst_shift = sizeof(UInt64) * num_bits;
for (UInt32 i = 0; i < num_full; ++i)
{
transpose<_T>(src, dst, num_bits);
src += src_shift;
dst += dst_shift;
}
UInt32 dst_bytes = num_full * dst_shift;
if (tail)
{
transpose<_T>(src, dst, num_bits, tail);
dst_bytes += dst_shift;
}
return header_size + dst_bytes;
}
template <typename _T>
void decompressData(const char * src, UInt32 bytes_size, char * dst, UInt32 uncompressed_size)
{
using MinMaxType = std::conditional_t<std::is_signed_v<_T>, Int64, UInt64>;
const UInt32 header_size = 2 * sizeof(UInt64);
if (bytes_size < header_size)
throw Exception("Cannot decompress, data size " + toString(bytes_size) + " is less then T64 header",
ErrorCodes::CANNOT_DECOMPRESS);
if (uncompressed_size % sizeof(_T))
throw Exception("Cannot decompress, unexpected uncompressed size " + toString(uncompressed_size),
ErrorCodes::CANNOT_DECOMPRESS);
UInt64 num_elements = uncompressed_size / sizeof(_T);
MinMaxType min;
MinMaxType max;
/// Read header
{
memcpy(&min, src, sizeof(MinMaxType));
memcpy(&max, src + 8, sizeof(MinMaxType));
src += header_size;
bytes_size -= header_size;
}
UInt32 num_bits = getValuableBitsNumber(min, max);
if (!num_bits)
{
_T min_value = min;
for (UInt32 i = 0; i < num_elements; ++i, dst += sizeof(_T))
unalignedStore(dst, min_value);
return;
}
UInt32 src_shift = sizeof(UInt64) * num_bits;
UInt32 dst_shift = sizeof(_T) * 64;
if (!bytes_size || bytes_size % src_shift)
throw Exception("Cannot decompress, data size " + toString(bytes_size) + " is not multiplier of " + toString(src_shift),
ErrorCodes::CANNOT_DECOMPRESS);
UInt32 num_full = bytes_size / src_shift;
UInt32 tail = num_elements % 64;
if (tail)
--num_full;
for (UInt32 i = 0; i < num_full; ++i)
{
revTranspose<_T>(src, dst, num_bits, min, max);
src += src_shift;
dst += dst_shift;
}
if (tail)
revTranspose<_T>(src, dst, num_bits, min, max, tail);
}
}
UInt32 CompressionCodecT64::doCompressData(const char * src, UInt32 src_size, char * dst) const
{
memcpy(dst, &type_idx, 1);
dst += 1;
switch (baseType(type_idx))
{
case TypeIndex::Int8:
return 1 + compressData<Int8>(src, src_size, dst);
case TypeIndex::Int16:
return 1 + compressData<Int16>(src, src_size, dst);
case TypeIndex::Int32:
return 1 + compressData<Int32>(src, src_size, dst);
case TypeIndex::Int64:
return 1 + compressData<Int64>(src, src_size, dst);
case TypeIndex::UInt8:
return 1 + compressData<UInt8>(src, src_size, dst);
case TypeIndex::UInt16:
return 1 + compressData<UInt16>(src, src_size, dst);
case TypeIndex::UInt32:
return 1 + compressData<UInt32>(src, src_size, dst);
case TypeIndex::UInt64:
return 1 + compressData<UInt64>(src, src_size, dst);
default:
break;
}
throw Exception("Connot compress with T64", ErrorCodes::CANNOT_COMPRESS);
}
void CompressionCodecT64::doDecompressData(const char * src, UInt32 src_size, char * dst, UInt32 uncompressed_size) const
{
if (!src_size)
throw Exception("Connot decompress with T64", ErrorCodes::CANNOT_DECOMPRESS);
UInt8 saved_type_id = unalignedLoad<UInt8>(src);
src += 1;
src_size -= 1;
TypeIndex actual_type_id = type_idx;
if (actual_type_id == TypeIndex::Nothing)
actual_type_id = static_cast<TypeIndex>(saved_type_id);
switch (baseType(actual_type_id))
{
case TypeIndex::Int8:
return decompressData<Int8>(src, src_size, dst, uncompressed_size);
case TypeIndex::Int16:
return decompressData<Int16>(src, src_size, dst, uncompressed_size);
case TypeIndex::Int32:
return decompressData<Int32>(src, src_size, dst, uncompressed_size);
case TypeIndex::Int64:
return decompressData<Int64>(src, src_size, dst, uncompressed_size);
case TypeIndex::UInt8:
return decompressData<UInt8>(src, src_size, dst, uncompressed_size);
case TypeIndex::UInt16:
return decompressData<UInt16>(src, src_size, dst, uncompressed_size);
case TypeIndex::UInt32:
return decompressData<UInt32>(src, src_size, dst, uncompressed_size);
case TypeIndex::UInt64:
return decompressData<UInt64>(src, src_size, dst, uncompressed_size);
default:
break;
}
throw Exception("Connot decompress with T64", ErrorCodes::CANNOT_DECOMPRESS);
}
void CompressionCodecT64::useInfoAboutType(DataTypePtr data_type)
{
if (data_type)
{
type_idx = typeIdx(data_type);
if (type_idx == TypeIndex::Nothing)
throw Exception("T64 codec is not supported for specified type", ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
}
}
UInt8 CompressionCodecT64::getMethodByte() const
{
return codecId();
}
void registerCodecT64(CompressionCodecFactory & factory)
{
auto reg_func = [&](const ASTPtr & arguments, DataTypePtr type) -> CompressionCodecPtr
{
if (arguments && !arguments->children.empty())
throw Exception("T64 codec should not have parameters", ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
auto type_idx = typeIdx(type);
if (type && type_idx == TypeIndex::Nothing)
throw Exception("T64 codec is not supported for specified type", ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
return std::make_shared<CompressionCodecT64>(type_idx);
};
factory.registerCompressionCodecWithType("T64", codecId(), reg_func);
}
}

View File

@ -0,0 +1,38 @@
#pragma once
#include <Core/Types.h>
#include <Compression/ICompressionCodec.h>
namespace DB
{
class CompressionCodecT64 : public ICompressionCodec
{
public:
static constexpr UInt32 HEADER_SIZE = 1 + 2 * sizeof(UInt64);
static constexpr UInt32 MAX_COMPRESSED_BLOCK_SIZE = sizeof(UInt64) * 64;
CompressionCodecT64(TypeIndex type_idx_)
: type_idx(type_idx_)
{}
UInt8 getMethodByte() const override;
String getCodecDesc() const override { return "T64"; }
void useInfoAboutType(DataTypePtr data_type) override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override
{
/// uncompressed_size - (uncompressed_size % (sizeof(T) * 64)) + sizeof(UInt64) * sizeof(T) + header_size
return uncompressed_size + MAX_COMPRESSED_BLOCK_SIZE + HEADER_SIZE;
}
private:
TypeIndex type_idx;
};
}

View File

@ -137,6 +137,7 @@ void registerCodecZSTD(CompressionCodecFactory & factory);
void registerCodecMultiple(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory);
void registerCodecLZ4HC(CompressionCodecFactory & factory); void registerCodecLZ4HC(CompressionCodecFactory & factory);
void registerCodecDelta(CompressionCodecFactory & factory); void registerCodecDelta(CompressionCodecFactory & factory);
void registerCodecT64(CompressionCodecFactory & factory);
CompressionCodecFactory::CompressionCodecFactory() CompressionCodecFactory::CompressionCodecFactory()
{ {
@ -147,6 +148,7 @@ CompressionCodecFactory::CompressionCodecFactory()
registerCodecMultiple(*this); registerCodecMultiple(*this);
registerCodecLZ4HC(*this); registerCodecLZ4HC(*this);
registerCodecDelta(*this); registerCodecDelta(*this);
registerCodecT64(*this);
} }
} }

View File

@ -40,6 +40,7 @@ enum class CompressionMethodByte : uint8_t
ZSTD = 0x90, ZSTD = 0x90,
Multiple = 0x91, Multiple = 0x91,
Delta = 0x92, Delta = 0x92,
T64 = 0x93,
}; };
} }

View File

@ -7,6 +7,8 @@
#include <Interpreters/InternalTextLogsQueue.h> #include <Interpreters/InternalTextLogsQueue.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <IO/ConnectionTimeouts.h>
namespace DB namespace DB
{ {
@ -61,17 +63,17 @@ RemoteBlockInputStream::RemoteBlockInputStream(
create_multiplexed_connections = [this, pool, throttler]() create_multiplexed_connections = [this, pool, throttler]()
{ {
const Settings & current_settings = context.getSettingsRef(); const Settings & current_settings = context.getSettingsRef();
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
std::vector<IConnectionPool::Entry> connections; std::vector<IConnectionPool::Entry> connections;
if (main_table) if (main_table)
{ {
auto try_results = pool->getManyChecked(&current_settings, pool_mode, *main_table); auto try_results = pool->getManyChecked(timeouts, &current_settings, pool_mode, *main_table);
connections.reserve(try_results.size()); connections.reserve(try_results.size());
for (auto & try_result : try_results) for (auto & try_result : try_results)
connections.emplace_back(std::move(try_result.entry)); connections.emplace_back(std::move(try_result.entry));
} }
else else
connections = pool->getMany(&current_settings, pool_mode); connections = pool->getMany(timeouts, &current_settings, pool_mode);
return std::make_unique<MultiplexedConnections>( return std::make_unique<MultiplexedConnections>(
std::move(connections), current_settings, throttler); std::move(connections), current_settings, throttler);
@ -283,12 +285,14 @@ void RemoteBlockInputStream::sendQuery()
{ {
multiplexed_connections = create_multiplexed_connections(); multiplexed_connections = create_multiplexed_connections();
if (context.getSettingsRef().skip_unavailable_shards && 0 == multiplexed_connections->size()) const auto& settings = context.getSettingsRef();
if (settings.skip_unavailable_shards && 0 == multiplexed_connections->size())
return; return;
established = true; established = true;
multiplexed_connections->sendQuery(query, "", stage, &context.getClientInfo(), true); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings);
multiplexed_connections->sendQuery(timeouts, query, "", stage, &context.getClientInfo(), true);
established = false; established = false;
sent_query = true; sent_query = true;

View File

@ -96,6 +96,7 @@ private:
const String query; const String query;
Context context; Context context;
/// Temporary tables needed to be sent to remote servers /// Temporary tables needed to be sent to remote servers
Tables external_tables; Tables external_tables;
QueryProcessingStage::Enum stage; QueryProcessingStage::Enum stage;
@ -118,7 +119,7 @@ private:
*/ */
std::atomic<bool> finished { false }; std::atomic<bool> finished { false };
/** Cancel query request was sent to all replicas beacuse data is not needed anymore /** Cancel query request was sent to all replicas because data is not needed anymore
* This behaviour may occur when: * This behaviour may occur when:
* - data size is already satisfactory (when using LIMIT, for example) * - data size is already satisfactory (when using LIMIT, for example)
* - an exception was thrown from client side * - an exception was thrown from client side

View File

@ -6,6 +6,7 @@
#include <Common/NetException.h> #include <Common/NetException.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
#include <Interpreters/InternalTextLogsQueue.h> #include <Interpreters/InternalTextLogsQueue.h>
#include <IO/ConnectionTimeouts.h>
namespace DB namespace DB
@ -18,13 +19,16 @@ namespace ErrorCodes
} }
RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const String & query_, const Settings * settings_) RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_,
const ConnectionTimeouts & timeouts,
const String & query_,
const Settings * settings_)
: connection(connection_), query(query_), settings(settings_) : connection(connection_), query(query_), settings(settings_)
{ {
/** Send query and receive "header", that describe table structure. /** Send query and receive "header", that describe table structure.
* Header is needed to know, what structure is required for blocks to be passed to 'write' method. * Header is needed to know, what structure is required for blocks to be passed to 'write' method.
*/ */
connection.sendQuery(query, "", QueryProcessingStage::Complete, settings, nullptr); connection.sendQuery(timeouts, query, "", QueryProcessingStage::Complete, settings, nullptr);
while (true) while (true)
{ {

View File

@ -3,6 +3,7 @@
#include <Core/Block.h> #include <Core/Block.h>
#include <DataStreams/IBlockOutputStream.h> #include <DataStreams/IBlockOutputStream.h>
#include <Common/Throttler.h> #include <Common/Throttler.h>
#include <IO/ConnectionTimeouts.h>
namespace DB namespace DB
@ -18,7 +19,10 @@ struct Settings;
class RemoteBlockOutputStream : public IBlockOutputStream class RemoteBlockOutputStream : public IBlockOutputStream
{ {
public: public:
RemoteBlockOutputStream(Connection & connection_, const String & query_, const Settings * settings_ = nullptr); RemoteBlockOutputStream(Connection & connection_,
const ConnectionTimeouts & timeouts,
const String & query_,
const Settings * settings_ = nullptr);
Block getHeader() const override { return header; } Block getHeader() const override { return header; }

View File

@ -43,10 +43,13 @@ private:
public: public:
using ExceptionCallback = std::function<void()>; using ExceptionCallback = std::function<void()>;
UnionBlockInputStream(BlockInputStreams inputs, BlockInputStreamPtr additional_input_at_end, size_t max_threads, UnionBlockInputStream(
ExceptionCallback exception_callback_ = ExceptionCallback()) : BlockInputStreams inputs,
output_queue(std::min(inputs.size(), max_threads)), BlockInputStreamPtr additional_input_at_end,
handler(*this), size_t max_threads,
ExceptionCallback exception_callback_ = ExceptionCallback()
) :
output_queue(std::min(inputs.size(), max_threads)), handler(*this),
processor(inputs, additional_input_at_end, max_threads, handler), processor(inputs, additional_input_at_end, max_threads, handler),
exception_callback(exception_callback_) exception_callback(exception_callback_)
{ {

View File

@ -1,11 +1,5 @@
add_executable (data_types_number_fixed data_types_number_fixed.cpp) add_executable (data_types_number_fixed data_types_number_fixed.cpp)
target_link_libraries (data_types_number_fixed PRIVATE dbms) target_link_libraries (data_types_number_fixed PRIVATE dbms)
add_executable (data_type_string data_type_string.cpp) add_executable (data_type_string data_type_string.cpp)
target_link_libraries (data_type_string PRIVATE dbms) target_link_libraries (data_type_string PRIVATE dbms)
if(USE_GTEST)
add_executable(data_type_get_common_type data_type_get_common_type.cpp)
target_link_libraries(data_type_get_common_type PRIVATE dbms ${GTEST_BOTH_LIBRARIES})
endif()

View File

@ -1,166 +0,0 @@
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/getLeastSupertype.h>
#include <DataTypes/getMostSubtype.h>
#include <sstream>
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wundef"
#endif
#include <gtest/gtest.h>
using namespace DB;
TEST(data_type, data_type_get_common_type_Test)
{
try
{
auto & data_type_factory = DataTypeFactory::instance();
auto typeFromString = [& data_type_factory](const std::string & str)
{
return data_type_factory.get(str);
};
auto typesFromString = [& typeFromString](const std::string & str)
{
std::istringstream data_types_stream(str);
DataTypes data_types;
std::string data_type;
while (data_types_stream >> data_type)
data_types.push_back(typeFromString(data_type));
return data_types;
};
ASSERT_TRUE(getLeastSupertype(typesFromString(""))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nothing"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Int8 Int8"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 Int8"))->equals(*typeFromString("Int16")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 Int16"))->equals(*typeFromString("Int16")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 UInt32 UInt64"))->equals(*typeFromString("UInt64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Int8 Int32 Int64"))->equals(*typeFromString("Int64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 UInt32 Int64"))->equals(*typeFromString("Int64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 Float64"))->equals(*typeFromString("Float64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 UInt16 Int16"))->equals(*typeFromString("Float32")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 UInt16 Int32"))->equals(*typeFromString("Float64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 Int16 UInt32"))->equals(*typeFromString("Float64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Date Date"))->equals(*typeFromString("Date")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Date DateTime"))->equals(*typeFromString("DateTime")));
ASSERT_TRUE(getLeastSupertype(typesFromString("String FixedString(32) FixedString(8)"))->equals(*typeFromString("String")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(UInt8) Array(UInt8)"))->equals(*typeFromString("Array(UInt8)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(UInt8) Array(Int8)"))->equals(*typeFromString("Array(Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Float32) Array(Int16) Array(UInt32)"))->equals(*typeFromString("Array(Float64)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Array(UInt8)) Array(Array(UInt8))"))->equals(*typeFromString("Array(Array(UInt8))")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Array(UInt8)) Array(Array(Int8))"))->equals(*typeFromString("Array(Array(Int16))")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Date) Array(DateTime)"))->equals(*typeFromString("Array(DateTime)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(String) Array(FixedString(32))"))->equals(*typeFromString("Array(String)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nullable(Nothing) Nothing"))->equals(*typeFromString("Nullable(Nothing)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nullable(UInt8) Int8"))->equals(*typeFromString("Nullable(Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nullable(Nothing) UInt8 Int8"))->equals(*typeFromString("Nullable(Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Tuple(Int8,UInt8) Tuple(UInt8,Int8)"))->equals(*typeFromString("Tuple(Int16,Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Tuple(Nullable(Nothing)) Tuple(Nullable(UInt8))"))->equals(*typeFromString("Tuple(Nullable(UInt8))")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Int8 String")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Int64 UInt64")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Float32 UInt64")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Float64 Int64")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Tuple(Int64) Tuple(UInt64)")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Tuple(Int64, Int8) Tuple(UInt64)")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Array(Int64) Array(String)")));
ASSERT_TRUE(getMostSubtype(typesFromString(""))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nothing"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Int8 Int8"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 Int8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Int8 UInt16"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 UInt32 UInt64"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Int8 Int32 Int64"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 Int64 UInt64"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 Float64"))->equals(*typeFromString("Float32")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 UInt16 Int16"))->equals(*typeFromString("UInt16")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 UInt16 Int32"))->equals(*typeFromString("UInt16")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 Int16 UInt32"))->equals(*typeFromString("Int16")));
ASSERT_TRUE(getMostSubtype(typesFromString("DateTime DateTime"))->equals(*typeFromString("DateTime")));
ASSERT_TRUE(getMostSubtype(typesFromString("Date DateTime"))->equals(*typeFromString("Date")));
ASSERT_TRUE(getMostSubtype(typesFromString("String FixedString(8)"))->equals(*typeFromString("FixedString(8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("FixedString(16) FixedString(8)"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(UInt8) Array(UInt8)"))->equals(*typeFromString("Array(UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(UInt8) Array(Int8)"))->equals(*typeFromString("Array(UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Float32) Array(Int16) Array(UInt32)"))->equals(*typeFromString("Array(Int16)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Array(UInt8)) Array(Array(UInt8))"))->equals(*typeFromString("Array(Array(UInt8))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Array(UInt8)) Array(Array(Int8))"))->equals(*typeFromString("Array(Array(UInt8))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Date) Array(DateTime)"))->equals(*typeFromString("Array(Date)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(String) Array(FixedString(32))"))->equals(*typeFromString("Array(FixedString(32))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(String) Array(FixedString(32))"))->equals(*typeFromString("Array(FixedString(32))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(Nothing) Nothing"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(UInt8) Int8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(Nothing) UInt8 Int8"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(UInt8) Nullable(Int8)"))->equals(*typeFromString("Nullable(UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(Nothing) Nullable(Int8)"))->equals(*typeFromString("Nullable(Nothing)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Tuple(Int8,UInt8) Tuple(UInt8,Int8)"))->equals(*typeFromString("Tuple(UInt8,UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Tuple(Nullable(Nothing)) Tuple(Nullable(UInt8))"))->equals(*typeFromString("Tuple(Nullable(Nothing))")));
EXPECT_ANY_THROW(getMostSubtype(typesFromString("Int8 String"), true));
EXPECT_ANY_THROW(getMostSubtype(typesFromString("Nothing"), true));
EXPECT_ANY_THROW(getMostSubtype(typesFromString("FixedString(16) FixedString(8) String"), true));
}
catch (const Exception & e)
{
std::string text = e.displayText();
bool print_stack_trace = true;
auto embedded_stack_trace_pos = text.find("Stack trace");
if (std::string::npos != embedded_stack_trace_pos && !print_stack_trace)
text.resize(embedded_stack_trace_pos);
std::cerr << "Code: " << e.code() << ". " << text << std::endl << std::endl;
if (print_stack_trace && std::string::npos == embedded_stack_trace_pos)
{
std::cerr << "Stack trace:" << std::endl
<< e.getStackTrace().toString();
}
throw;
}
catch (const Poco::Exception & e)
{
std::cerr << "Poco::Exception: " << e.displayText() << std::endl;
throw;
}
catch (const std::exception & e)
{
std::cerr << "std::exception: " << e.what() << std::endl;
throw;
}
catch (...)
{
std::cerr << "Unknown exception" << std::endl;
throw;
}
}

View File

@ -0,0 +1,127 @@
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/getLeastSupertype.h>
#include <DataTypes/getMostSubtype.h>
#include <sstream>
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wundef"
#endif
#include <gtest/gtest.h>
using namespace DB;
TEST(data_type, data_type_get_common_type_Test)
{
auto & data_type_factory = DataTypeFactory::instance();
auto typeFromString = [& data_type_factory](const std::string & str)
{
return data_type_factory.get(str);
};
auto typesFromString = [& typeFromString](const std::string & str)
{
std::istringstream data_types_stream(str);
DataTypes data_types;
std::string data_type;
while (data_types_stream >> data_type)
data_types.push_back(typeFromString(data_type));
return data_types;
};
ASSERT_TRUE(getLeastSupertype(typesFromString(""))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nothing"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Int8 Int8"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 Int8"))->equals(*typeFromString("Int16")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 Int16"))->equals(*typeFromString("Int16")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 UInt32 UInt64"))->equals(*typeFromString("UInt64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Int8 Int32 Int64"))->equals(*typeFromString("Int64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 UInt32 Int64"))->equals(*typeFromString("Int64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 Float64"))->equals(*typeFromString("Float64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 UInt16 Int16"))->equals(*typeFromString("Float32")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 UInt16 Int32"))->equals(*typeFromString("Float64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 Int16 UInt32"))->equals(*typeFromString("Float64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Date Date"))->equals(*typeFromString("Date")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Date DateTime"))->equals(*typeFromString("DateTime")));
ASSERT_TRUE(getLeastSupertype(typesFromString("String FixedString(32) FixedString(8)"))->equals(*typeFromString("String")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(UInt8) Array(UInt8)"))->equals(*typeFromString("Array(UInt8)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(UInt8) Array(Int8)"))->equals(*typeFromString("Array(Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Float32) Array(Int16) Array(UInt32)"))->equals(*typeFromString("Array(Float64)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Array(UInt8)) Array(Array(UInt8))"))->equals(*typeFromString("Array(Array(UInt8))")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Array(UInt8)) Array(Array(Int8))"))->equals(*typeFromString("Array(Array(Int16))")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Date) Array(DateTime)"))->equals(*typeFromString("Array(DateTime)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(String) Array(FixedString(32))"))->equals(*typeFromString("Array(String)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nullable(Nothing) Nothing"))->equals(*typeFromString("Nullable(Nothing)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nullable(UInt8) Int8"))->equals(*typeFromString("Nullable(Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nullable(Nothing) UInt8 Int8"))->equals(*typeFromString("Nullable(Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Tuple(Int8,UInt8) Tuple(UInt8,Int8)"))->equals(*typeFromString("Tuple(Int16,Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Tuple(Nullable(Nothing)) Tuple(Nullable(UInt8))"))->equals(*typeFromString("Tuple(Nullable(UInt8))")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Int8 String")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Int64 UInt64")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Float32 UInt64")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Float64 Int64")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Tuple(Int64) Tuple(UInt64)")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Tuple(Int64, Int8) Tuple(UInt64)")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Array(Int64) Array(String)")));
ASSERT_TRUE(getMostSubtype(typesFromString(""))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nothing"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Int8 Int8"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 Int8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Int8 UInt16"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 UInt32 UInt64"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Int8 Int32 Int64"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 Int64 UInt64"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 Float64"))->equals(*typeFromString("Float32")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 UInt16 Int16"))->equals(*typeFromString("UInt16")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 UInt16 Int32"))->equals(*typeFromString("UInt16")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 Int16 UInt32"))->equals(*typeFromString("Int16")));
ASSERT_TRUE(getMostSubtype(typesFromString("DateTime DateTime"))->equals(*typeFromString("DateTime")));
ASSERT_TRUE(getMostSubtype(typesFromString("Date DateTime"))->equals(*typeFromString("Date")));
ASSERT_TRUE(getMostSubtype(typesFromString("String FixedString(8)"))->equals(*typeFromString("FixedString(8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("FixedString(16) FixedString(8)"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(UInt8) Array(UInt8)"))->equals(*typeFromString("Array(UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(UInt8) Array(Int8)"))->equals(*typeFromString("Array(UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Float32) Array(Int16) Array(UInt32)"))->equals(*typeFromString("Array(Int16)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Array(UInt8)) Array(Array(UInt8))"))->equals(*typeFromString("Array(Array(UInt8))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Array(UInt8)) Array(Array(Int8))"))->equals(*typeFromString("Array(Array(UInt8))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Date) Array(DateTime)"))->equals(*typeFromString("Array(Date)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(String) Array(FixedString(32))"))->equals(*typeFromString("Array(FixedString(32))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(String) Array(FixedString(32))"))->equals(*typeFromString("Array(FixedString(32))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(Nothing) Nothing"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(UInt8) Int8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(Nothing) UInt8 Int8"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(UInt8) Nullable(Int8)"))->equals(*typeFromString("Nullable(UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(Nothing) Nullable(Int8)"))->equals(*typeFromString("Nullable(Nothing)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Tuple(Int8,UInt8) Tuple(UInt8,Int8)"))->equals(*typeFromString("Tuple(UInt8,UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Tuple(Nullable(Nothing)) Tuple(Nullable(UInt8))"))->equals(*typeFromString("Tuple(Nullable(Nothing))")));
EXPECT_ANY_THROW(getMostSubtype(typesFromString("Int8 String"), true));
EXPECT_ANY_THROW(getMostSubtype(typesFromString("Nothing"), true));
EXPECT_ANY_THROW(getMostSubtype(typesFromString("FixedString(16) FixedString(8) String"), true));
}

View File

@ -30,10 +30,8 @@ static ConnectionPoolWithFailoverPtr createPool(
bool secure, bool secure,
const std::string & db, const std::string & db,
const std::string & user, const std::string & user,
const std::string & password, const std::string & password)
const Context & context)
{ {
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(context.getSettingsRef());
ConnectionPoolPtrs pools; ConnectionPoolPtrs pools;
pools.emplace_back(std::make_shared<ConnectionPool>( pools.emplace_back(std::make_shared<ConnectionPool>(
MAX_CONNECTIONS, MAX_CONNECTIONS,
@ -42,7 +40,6 @@ static ConnectionPoolWithFailoverPtr createPool(
db, db,
user, user,
password, password,
timeouts,
"ClickHouseDictionarySource", "ClickHouseDictionarySource",
Protocol::Compression::Enable, Protocol::Compression::Enable,
secure ? Protocol::Secure::Enable : Protocol::Secure::Disable)); secure ? Protocol::Secure::Enable : Protocol::Secure::Disable));
@ -72,7 +69,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
, sample_block{sample_block} , sample_block{sample_block}
, context(context_) , context(context_)
, is_local{isLocalAddress({host, port}, context.getTCPPort())} , is_local{isLocalAddress({host, port}, context.getTCPPort())}
, pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)} , pool{is_local ? nullptr : createPool(host, port, secure, db, user, password)}
, load_all_query{query_builder.composeLoadAllQuery()} , load_all_query{query_builder.composeLoadAllQuery()}
{ {
/// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication).
@ -98,7 +95,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar
, sample_block{other.sample_block} , sample_block{other.sample_block}
, context(other.context) , context(other.context)
, is_local{other.is_local} , is_local{other.is_local}
, pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)} , pool{is_local ? nullptr : createPool(host, port, secure, db, user, password)}
, load_all_query{other.load_all_query} , load_all_query{other.load_all_query}
{ {
} }
@ -179,6 +176,7 @@ BlockInputStreamPtr ClickHouseDictionarySource::createStreamForSelectiveLoad(con
{ {
if (is_local) if (is_local)
return executeQuery(query, context, true).in; return executeQuery(query, context, true).in;
return std::make_shared<RemoteBlockInputStream>(pool, query, sample_block, context); return std::make_shared<RemoteBlockInputStream>(pool, query, sample_block, context);
} }

View File

@ -48,9 +48,9 @@ private:
struct Action struct Action
{ {
enum Type { POP, PUSH, READ }; enum Type { POP, PUSH, READ };
Type type; Type type{};
capnp::StructSchema::Field field = {}; capnp::StructSchema::Field field{};
BlockPositionList columns = {}; BlockPositionList columns{};
}; };
// Wrapper for classes that could throw in destructor // Wrapper for classes that could throw in destructor

View File

@ -394,10 +394,10 @@ inline void parseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb
template <typename DataType> template <typename DataType>
bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
{ {
if constexpr (std::is_integral_v<typename DataType::FieldType>) if constexpr (std::is_floating_point_v<typename DataType::FieldType>)
return tryReadIntText(x, rb);
else if constexpr (std::is_floating_point_v<typename DataType::FieldType>)
return tryReadFloatText(x, rb); return tryReadFloatText(x, rb);
else /*if constexpr (std::is_integral_v<typename DataType::FieldType>)*/
return tryReadIntText(x, rb);
} }
template <> template <>

View File

@ -95,7 +95,7 @@ struct SimdJSONParser
{ {
do do
++size; ++size;
while (it2.next() && it2.next()); while (it2.next() && it2.next()); //-V501
} }
return size; return size;
} }
@ -116,13 +116,13 @@ struct SimdJSONParser
if (!it.down()) if (!it.down())
return false; return false;
while (index--) while (index--)
if (!it.next() || !it.next()) if (!it.next() || !it.next()) //-V501
return false; return false;
return it.next(); return it.next();
} }
static bool objectMemberByName(Iterator & it, const StringRef & name) { return it.move_to_key(name.data); } static bool objectMemberByName(Iterator & it, const StringRef & name) { return it.move_to_key(name.data); }
static bool nextObjectMember(Iterator & it) { return it.next() && it.next(); } static bool nextObjectMember(Iterator & it) { return it.next() && it.next(); } //-V501
static bool nextObjectMember(Iterator & it, StringRef & next_key) static bool nextObjectMember(Iterator & it, StringRef & next_key)
{ {

View File

@ -41,7 +41,7 @@ public:
protected: protected:
ProfileCallback profile_callback; ProfileCallback profile_callback;
clockid_t clock_type; clockid_t clock_type{};
/// Children implementation should be able to seek backwards /// Children implementation should be able to seek backwards
virtual off_t doSeek(off_t off, int whence) = 0; virtual off_t doSeek(off_t off, int whence) = 0;

View File

@ -12,7 +12,7 @@ namespace DB
class WriteBufferFromOStream : public BufferWithOwnMemory<WriteBuffer> class WriteBufferFromOStream : public BufferWithOwnMemory<WriteBuffer>
{ {
protected: protected:
std::ostream * ostr; std::ostream * ostr{};
void nextImpl() override; void nextImpl() override;

View File

@ -312,7 +312,7 @@ template <typename T, typename ReturnType>
ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
{ {
static_assert(std::is_same_v<T, double> || std::is_same_v<T, float>, "Argument for readFloatTextImpl must be float or double"); static_assert(std::is_same_v<T, double> || std::is_same_v<T, float>, "Argument for readFloatTextImpl must be float or double");
static_assert('a' > '.' && 'A' > '.' && '\n' < '.' && '\t' < '.' && '\'' < '.' && '"' < '.', "Layout of char is not like ASCII"); //-V501 static_assert('a' > '.' && 'A' > '.' && '\n' < '.' && '\t' < '.' && '\'' < '.' && '"' < '.', "Layout of char is not like ASCII"); //-V590
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>; static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;

View File

@ -13,6 +13,7 @@ namespace ActionLocks
extern const StorageActionBlockType PartsFetch = 2; extern const StorageActionBlockType PartsFetch = 2;
extern const StorageActionBlockType PartsSend = 3; extern const StorageActionBlockType PartsSend = 3;
extern const StorageActionBlockType ReplicationQueue = 4; extern const StorageActionBlockType ReplicationQueue = 4;
extern const StorageActionBlockType DistributedSend = 5;
} }

View File

@ -250,11 +250,10 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting
settings.distributed_connections_pool_size, settings.distributed_connections_pool_size,
address.host_name, address.port, address.host_name, address.port,
address.default_database, address.user, address.password, address.default_database, address.user, address.password,
ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings).getSaturated(settings.max_execution_time),
"server", address.compression, address.secure); "server", address.compression, address.secure);
info.pool = std::make_shared<ConnectionPoolWithFailover>( info.pool = std::make_shared<ConnectionPoolWithFailover>(
ConnectionPoolPtrs{pool}, settings.load_balancing, settings.connections_with_failover_max_tries); ConnectionPoolPtrs{pool}, settings.load_balancing);
info.per_replica_pools = {std::move(pool)}; info.per_replica_pools = {std::move(pool)};
if (weight) if (weight)
@ -322,7 +321,6 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting
settings.distributed_connections_pool_size, settings.distributed_connections_pool_size,
replica.host_name, replica.port, replica.host_name, replica.port,
replica.default_database, replica.user, replica.password, replica.default_database, replica.user, replica.password,
ConnectionTimeouts::getTCPTimeoutsWithFailover(settings).getSaturated(settings.max_execution_time),
"server", replica.compression, replica.secure); "server", replica.compression, replica.secure);
all_replicas_pools.emplace_back(replica_pool); all_replicas_pools.emplace_back(replica_pool);
@ -331,7 +329,7 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting
} }
ConnectionPoolWithFailoverPtr shard_pool = std::make_shared<ConnectionPoolWithFailover>( ConnectionPoolWithFailoverPtr shard_pool = std::make_shared<ConnectionPoolWithFailover>(
all_replicas_pools, settings.load_balancing, settings.connections_with_failover_max_tries); all_replicas_pools, settings.load_balancing);
if (weight) if (weight)
slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size()); slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size());
@ -375,7 +373,6 @@ Cluster::Cluster(const Settings & settings, const std::vector<std::vector<String
settings.distributed_connections_pool_size, settings.distributed_connections_pool_size,
replica.host_name, replica.port, replica.host_name, replica.port,
replica.default_database, replica.user, replica.password, replica.default_database, replica.user, replica.password,
ConnectionTimeouts::getTCPTimeoutsWithFailover(settings).getSaturated(settings.max_execution_time),
"server", replica.compression, replica.secure); "server", replica.compression, replica.secure);
all_replicas.emplace_back(replica_pool); all_replicas.emplace_back(replica_pool);
if (replica.is_local && !treat_local_as_remote) if (replica.is_local && !treat_local_as_remote)
@ -383,7 +380,7 @@ Cluster::Cluster(const Settings & settings, const std::vector<std::vector<String
} }
ConnectionPoolWithFailoverPtr shard_pool = std::make_shared<ConnectionPoolWithFailover>( ConnectionPoolWithFailoverPtr shard_pool = std::make_shared<ConnectionPoolWithFailover>(
all_replicas, settings.load_balancing, settings.connections_with_failover_max_tries); all_replicas, settings.load_balancing);
slot_to_shard.insert(std::end(slot_to_shard), default_weight, shards_info.size()); slot_to_shard.insert(std::end(slot_to_shard), default_weight, shards_info.size());
shards_info.push_back({{}, current_shard_num, default_weight, std::move(shard_local_addresses), std::move(shard_pool), shards_info.push_back({{}, current_shard_num, default_weight, std::move(shard_local_addresses), std::move(shard_pool),

View File

@ -184,13 +184,17 @@ void SelectStreamFactory::createForShard(
local_delay]() local_delay]()
-> BlockInputStreamPtr -> BlockInputStreamPtr
{ {
auto current_settings = context.getSettingsRef();
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(
current_settings).getSaturated(
current_settings.max_execution_time);
std::vector<ConnectionPoolWithFailover::TryResult> try_results; std::vector<ConnectionPoolWithFailover::TryResult> try_results;
try try
{ {
if (table_func_ptr) if (table_func_ptr)
try_results = pool->getManyForTableFunction(&context.getSettingsRef(), PoolMode::GET_MANY); try_results = pool->getManyForTableFunction(timeouts, &current_settings, PoolMode::GET_MANY);
else else
try_results = pool->getManyChecked(&context.getSettingsRef(), PoolMode::GET_MANY, main_table); try_results = pool->getManyChecked(timeouts, &current_settings, PoolMode::GET_MANY, main_table);
} }
catch (const Exception & ex) catch (const Exception & ex)
{ {

View File

@ -69,7 +69,7 @@ public:
ADD_ALIASES, ADD_ALIASES,
}; };
Type type; Type type{};
/// For ADD/REMOVE/COPY_COLUMN. /// For ADD/REMOVE/COPY_COLUMN.
std::string source_name; std::string source_name;

View File

@ -15,6 +15,7 @@
#include <Interpreters/PartLog.h> #include <Interpreters/PartLog.h>
#include <Interpreters/QueryThreadLog.h> #include <Interpreters/QueryThreadLog.h>
#include <Databases/IDatabase.h> #include <Databases/IDatabase.h>
#include <Storages/StorageDistributed.h>
#include <Storages/StorageReplicatedMergeTree.h> #include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/StorageFactory.h> #include <Storages/StorageFactory.h>
#include <Parsers/ASTSystemQuery.h> #include <Parsers/ASTSystemQuery.h>
@ -22,6 +23,7 @@
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <csignal> #include <csignal>
#include <algorithm> #include <algorithm>
#include "InterpreterSystemQuery.h"
namespace DB namespace DB
@ -42,6 +44,7 @@ namespace ActionLocks
extern StorageActionBlockType PartsFetch; extern StorageActionBlockType PartsFetch;
extern StorageActionBlockType PartsSend; extern StorageActionBlockType PartsSend;
extern StorageActionBlockType ReplicationQueue; extern StorageActionBlockType ReplicationQueue;
extern StorageActionBlockType DistributedSend;
} }
@ -194,9 +197,18 @@ BlockIO InterpreterSystemQuery::execute()
case Type::START_REPLICATION_QUEUES: case Type::START_REPLICATION_QUEUES:
startStopAction(context, query, ActionLocks::ReplicationQueue, true); startStopAction(context, query, ActionLocks::ReplicationQueue, true);
break; break;
case Type::STOP_DISTRIBUTED_SENDS:
startStopAction(context, query, ActionLocks::DistributedSend, false);
break;
case Type::START_DISTRIBUTED_SENDS:
startStopAction(context, query, ActionLocks::DistributedSend, true);
break;
case Type::SYNC_REPLICA: case Type::SYNC_REPLICA:
syncReplica(query); syncReplica(query);
break; break;
case Type::FLUSH_DISTRIBUTED:
flushDistributed(query);
break;
case Type::RESTART_REPLICAS: case Type::RESTART_REPLICAS:
restartReplicas(system_context); restartReplicas(system_context);
break; break;
@ -303,11 +315,21 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query)
StoragePtr table = context.getTable(database_name, table_name); StoragePtr table = context.getTable(database_name, table_name);
auto table_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get()); if (auto storage_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get()))
if (!table_replicated) storage_replicated->waitForShrinkingQueueSize(0, context.getSettingsRef().receive_timeout.value.milliseconds());
else
throw Exception("Table " + database_name + "." + table_name + " is not replicated", ErrorCodes::BAD_ARGUMENTS); throw Exception("Table " + database_name + "." + table_name + " is not replicated", ErrorCodes::BAD_ARGUMENTS);
}
table_replicated->waitForShrinkingQueueSize(0, context.getSettingsRef().receive_timeout.value.milliseconds()); void InterpreterSystemQuery::flushDistributed(ASTSystemQuery & query)
{
String database_name = !query.target_database.empty() ? query.target_database : context.getCurrentDatabase();
String & table_name = query.target_table;
if (auto storage_distributed = dynamic_cast<StorageDistributed *>(context.getTable(database_name, table_name).get()))
storage_distributed->flushClusterNodesAllData();
else
throw Exception("Table " + database_name + "." + table_name + " is not distributed", ErrorCodes::BAD_ARGUMENTS);
} }

View File

@ -31,6 +31,7 @@ private:
void restartReplicas(Context & system_context); void restartReplicas(Context & system_context);
void syncReplica(ASTSystemQuery & query); void syncReplica(ASTSystemQuery & query);
void flushDistributed(ASTSystemQuery & query);
}; };

View File

@ -474,6 +474,14 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
} }
} }
[[noreturn]] static void throwSyntaxException(const String & msg)
{
throw Exception("Invalid expression for JOIN ON. " + msg + " Supported syntax: JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) "
"[AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]", ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
};
/// Parse JOIN ON expression and collect ASTs for joined columns. /// Parse JOIN ON expression and collect ASTs for joined columns.
void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTTableJoin & table_join) void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTTableJoin & table_join)
{ {
@ -528,13 +536,6 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTT
return table_belonging; return table_belonging;
}; };
const auto supported_syntax = " Supported syntax: JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) "
"[AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]";
auto throwSyntaxException = [&](const String & msg)
{
throw Exception("Invalid expression for JOIN ON. " + msg + supported_syntax, ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
};
/// For equal expression find out corresponding table for each part, translate qualified names and add asts to join keys. /// For equal expression find out corresponding table for each part, translate qualified names and add asts to join keys.
auto add_columns_from_equals_expr = [&](const ASTPtr & expr) auto add_columns_from_equals_expr = [&](const ASTPtr & expr)
{ {

View File

@ -23,7 +23,7 @@
struct SmallStringRef struct SmallStringRef
{ {
UInt32 size; UInt32 size = 0;
union union
{ {

View File

@ -41,6 +41,8 @@ const char * ASTSystemQuery::typeToString(Type type)
return "RESTART REPLICA"; return "RESTART REPLICA";
case Type::SYNC_REPLICA: case Type::SYNC_REPLICA:
return "SYNC REPLICA"; return "SYNC REPLICA";
case Type::FLUSH_DISTRIBUTED:
return "FLUSH DISTRIBUTED";
case Type::RELOAD_DICTIONARY: case Type::RELOAD_DICTIONARY:
return "RELOAD DICTIONARY"; return "RELOAD DICTIONARY";
case Type::RELOAD_DICTIONARIES: case Type::RELOAD_DICTIONARIES:
@ -65,6 +67,10 @@ const char * ASTSystemQuery::typeToString(Type type)
return "STOP REPLICATION QUEUES"; return "STOP REPLICATION QUEUES";
case Type::START_REPLICATION_QUEUES: case Type::START_REPLICATION_QUEUES:
return "START REPLICATION QUEUES"; return "START REPLICATION QUEUES";
case Type::STOP_DISTRIBUTED_SENDS:
return "STOP DISTRIBUTED SENDS";
case Type::START_DISTRIBUTED_SENDS:
return "START DISTRIBUTED SENDS";
case Type::FLUSH_LOGS: case Type::FLUSH_LOGS:
return "FLUSH LOGS"; return "FLUSH LOGS";
default: default:
@ -99,12 +105,14 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|| type == Type::STOP_REPLICATED_SENDS || type == Type::STOP_REPLICATED_SENDS
|| type == Type::START_REPLICATED_SENDS || type == Type::START_REPLICATED_SENDS
|| type == Type::STOP_REPLICATION_QUEUES || type == Type::STOP_REPLICATION_QUEUES
|| type == Type::START_REPLICATION_QUEUES) || type == Type::START_REPLICATION_QUEUES
|| type == Type::STOP_DISTRIBUTED_SENDS
|| type == Type::START_DISTRIBUTED_SENDS)
{ {
if (!target_table.empty()) if (!target_table.empty())
print_database_table(); print_database_table();
} }
else if (type == Type::RESTART_REPLICA || type == Type::SYNC_REPLICA) else if (type == Type::RESTART_REPLICA || type == Type::SYNC_REPLICA || type == Type::FLUSH_DISTRIBUTED)
{ {
print_database_table(); print_database_table();
} }

View File

@ -40,6 +40,9 @@ public:
STOP_REPLICATION_QUEUES, STOP_REPLICATION_QUEUES,
START_REPLICATION_QUEUES, START_REPLICATION_QUEUES,
FLUSH_LOGS, FLUSH_LOGS,
FLUSH_DISTRIBUTED,
STOP_DISTRIBUTED_SENDS,
START_DISTRIBUTED_SENDS,
END END
}; };

View File

@ -49,6 +49,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::RESTART_REPLICA: case Type::RESTART_REPLICA:
case Type::SYNC_REPLICA: case Type::SYNC_REPLICA:
case Type::FLUSH_DISTRIBUTED:
if (!parseDatabaseAndTableName(pos, expected, res->target_database, res->target_table)) if (!parseDatabaseAndTableName(pos, expected, res->target_database, res->target_table))
return false; return false;
break; break;
@ -61,6 +62,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::START_REPLICATED_SENDS: case Type::START_REPLICATED_SENDS:
case Type::STOP_REPLICATION_QUEUES: case Type::STOP_REPLICATION_QUEUES:
case Type::START_REPLICATION_QUEUES: case Type::START_REPLICATION_QUEUES:
case Type::STOP_DISTRIBUTED_SENDS:
case Type::START_DISTRIBUTED_SENDS:
parseDatabaseAndTableName(pos, expected, res->target_database, res->target_table); parseDatabaseAndTableName(pos, expected, res->target_database, res->target_table);
break; break;

View File

@ -11,6 +11,7 @@
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h> #include <IO/WriteBufferFromFile.h>
#include <Compression/CompressedReadBuffer.h> #include <Compression/CompressedReadBuffer.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <boost/algorithm/string/find_iterator.hpp> #include <boost/algorithm/string/find_iterator.hpp>
@ -29,6 +30,7 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int ABORTED;
extern const int INCORRECT_FILE_NAME; extern const int INCORRECT_FILE_NAME;
extern const int CHECKSUM_DOESNT_MATCH; extern const int CHECKSUM_DOESNT_MATCH;
extern const int TOO_LARGE_SIZE_COMPRESSED; extern const int TOO_LARGE_SIZE_COMPRESSED;
@ -57,12 +59,14 @@ namespace
} }
StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor(StorageDistributed & storage, const std::string & name, const ConnectionPoolPtr & pool) StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor(
StorageDistributed & storage, const std::string & name, const ConnectionPoolPtr & pool, ActionBlocker & monitor_blocker)
: storage(storage), pool{pool}, path{storage.path + name + '/'} : storage(storage), pool{pool}, path{storage.path + name + '/'}
, current_batch_file_path{path + "current_batch.txt"} , current_batch_file_path{path + "current_batch.txt"}
, default_sleep_time{storage.global_context.getSettingsRef().distributed_directory_monitor_sleep_time_ms.totalMilliseconds()} , default_sleep_time{storage.global_context.getSettingsRef().distributed_directory_monitor_sleep_time_ms.totalMilliseconds()}
, sleep_time{default_sleep_time} , sleep_time{default_sleep_time}
, log{&Logger::get(getLoggerName())} , log{&Logger::get(getLoggerName())}
, monitor_blocker(monitor_blocker)
{ {
const Settings & settings = storage.global_context.getSettingsRef(); const Settings & settings = storage.global_context.getSettingsRef();
should_batch_inserts = settings.distributed_directory_monitor_batch_inserts; should_batch_inserts = settings.distributed_directory_monitor_batch_inserts;
@ -84,6 +88,14 @@ StorageDistributedDirectoryMonitor::~StorageDistributedDirectoryMonitor()
} }
} }
void StorageDistributedDirectoryMonitor::flushAllData()
{
if (!quit)
{
std::unique_lock lock{mutex};
processFiles();
}
}
void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() void StorageDistributedDirectoryMonitor::shutdownAndDropAllData()
{ {
@ -113,9 +125,11 @@ void StorageDistributedDirectoryMonitor::run()
{ {
auto do_sleep = true; auto do_sleep = true;
if (!monitor_blocker.isCancelled())
{
try try
{ {
do_sleep = !findFiles(); do_sleep = !processFiles();
} }
catch (...) catch (...)
{ {
@ -126,6 +140,11 @@ void StorageDistributedDirectoryMonitor::run()
std::chrono::milliseconds{max_sleep_time}); std::chrono::milliseconds{max_sleep_time});
tryLogCurrentException(getLoggerName().data()); tryLogCurrentException(getLoggerName().data());
} }
}
else
{
LOG_DEBUG(log, "Skipping send data over distributed table.");
}
if (do_sleep) if (do_sleep)
cond.wait_for(lock, sleep_time, quit_requested); cond.wait_for(lock, sleep_time, quit_requested);
@ -142,8 +161,7 @@ void StorageDistributedDirectoryMonitor::run()
ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::string & name, const StorageDistributed & storage) ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::string & name, const StorageDistributed & storage)
{ {
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.global_context.getSettingsRef()); const auto pool_factory = [&storage] (const Cluster::Address & address) -> ConnectionPoolPtr
const auto pool_factory = [&storage, &timeouts] (const Cluster::Address & address) -> ConnectionPoolPtr
{ {
const auto & cluster = storage.getCluster(); const auto & cluster = storage.getCluster();
const auto & shards_info = cluster->getShardsInfo(); const auto & shards_info = cluster->getShardsInfo();
@ -164,7 +182,7 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
} }
return std::make_shared<ConnectionPool>( return std::make_shared<ConnectionPool>(
1, address.host_name, address.port, address.default_database, address.user, address.password, timeouts, 1, address.host_name, address.port, address.default_database, address.user, address.password,
storage.getName() + '_' + address.user, Protocol::Compression::Enable, address.secure); storage.getName() + '_' + address.user, Protocol::Compression::Enable, address.secure);
}; };
@ -174,7 +192,7 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
} }
bool StorageDistributedDirectoryMonitor::findFiles() bool StorageDistributedDirectoryMonitor::processFiles()
{ {
std::map<UInt64, std::string> files; std::map<UInt64, std::string> files;
@ -212,7 +230,8 @@ bool StorageDistributedDirectoryMonitor::findFiles()
void StorageDistributedDirectoryMonitor::processFile(const std::string & file_path) void StorageDistributedDirectoryMonitor::processFile(const std::string & file_path)
{ {
LOG_TRACE(log, "Started processing `" << file_path << '`'); LOG_TRACE(log, "Started processing `" << file_path << '`');
auto connection = pool->get(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.global_context.getSettingsRef());
auto connection = pool->get(timeouts);
try try
{ {
@ -224,7 +243,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa
std::string insert_query; std::string insert_query;
readQueryAndSettings(in, insert_settings, insert_query); readQueryAndSettings(in, insert_settings, insert_query);
RemoteBlockOutputStream remote{*connection, insert_query, &insert_settings}; RemoteBlockOutputStream remote{*connection, timeouts, insert_query, &insert_settings};
remote.writePrefix(); remote.writePrefix();
remote.writePrepared(in); remote.writePrepared(in);
@ -334,8 +353,8 @@ struct StorageDistributedDirectoryMonitor::Batch
WriteBufferFromFile out{parent.current_batch_file_path}; WriteBufferFromFile out{parent.current_batch_file_path};
writeText(out); writeText(out);
} }
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(parent.storage.global_context.getSettingsRef());
auto connection = parent.pool->get(); auto connection = parent.pool->get(timeouts);
bool batch_broken = false; bool batch_broken = false;
try try
@ -361,7 +380,7 @@ struct StorageDistributedDirectoryMonitor::Batch
if (first) if (first)
{ {
first = false; first = false;
remote = std::make_unique<RemoteBlockOutputStream>(*connection, insert_query, &insert_settings); remote = std::make_unique<RemoteBlockOutputStream>(*connection, timeouts, insert_query, &insert_settings);
remote->writePrefix(); remote->writePrefix();
} }

View File

@ -19,15 +19,19 @@ namespace DB
class StorageDistributedDirectoryMonitor class StorageDistributedDirectoryMonitor
{ {
public: public:
StorageDistributedDirectoryMonitor(StorageDistributed & storage, const std::string & name, const ConnectionPoolPtr & pool); StorageDistributedDirectoryMonitor(
StorageDistributed & storage, const std::string & name, const ConnectionPoolPtr & pool, ActionBlocker & monitor_blocker);
~StorageDistributedDirectoryMonitor(); ~StorageDistributedDirectoryMonitor();
static ConnectionPoolPtr createPool(const std::string & name, const StorageDistributed & storage); static ConnectionPoolPtr createPool(const std::string & name, const StorageDistributed & storage);
void flushAllData();
void shutdownAndDropAllData(); void shutdownAndDropAllData();
private: private:
void run(); void run();
bool findFiles(); bool processFiles();
void processFile(const std::string & file_path); void processFile(const std::string & file_path);
void processFilesWithBatching(const std::map<UInt64, std::string> & files); void processFilesWithBatching(const std::map<UInt64, std::string> & files);
@ -57,6 +61,7 @@ private:
std::mutex mutex; std::mutex mutex;
std::condition_variable cond; std::condition_variable cond;
Logger * log; Logger * log;
ActionBlocker & monitor_blocker;
ThreadFromGlobalPool thread{&StorageDistributedDirectoryMonitor::run, this}; ThreadFromGlobalPool thread{&StorageDistributedDirectoryMonitor::run, this};
/// Read insert query and insert settings for backward compatible. /// Read insert query and insert settings for backward compatible.

View File

@ -242,6 +242,8 @@ ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutp
{ {
if (!job.stream) if (!job.stream)
{ {
const Settings & settings = context.getSettingsRef();
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings);
if (shard_info.hasInternalReplication()) if (shard_info.hasInternalReplication())
{ {
/// Skip replica_index in case of internal replication /// Skip replica_index in case of internal replication
@ -249,7 +251,7 @@ ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutp
throw Exception("There are several writing job for an automatically replicated shard", ErrorCodes::LOGICAL_ERROR); throw Exception("There are several writing job for an automatically replicated shard", ErrorCodes::LOGICAL_ERROR);
/// TODO: it make sense to rewrite skip_unavailable_shards and max_parallel_replicas here /// TODO: it make sense to rewrite skip_unavailable_shards and max_parallel_replicas here
auto connections = shard_info.pool->getMany(&context.getSettingsRef(), PoolMode::GET_ONE); auto connections = shard_info.pool->getMany(timeouts, &settings, PoolMode::GET_ONE);
if (connections.empty() || connections.front().isNull()) if (connections.empty() || connections.front().isNull())
throw Exception("Expected exactly one connection for shard " + toString(job.shard_index), ErrorCodes::LOGICAL_ERROR); throw Exception("Expected exactly one connection for shard " + toString(job.shard_index), ErrorCodes::LOGICAL_ERROR);
@ -263,7 +265,7 @@ ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutp
if (!connection_pool) if (!connection_pool)
throw Exception("Connection pool for replica " + replica.readableString() + " does not exist", ErrorCodes::LOGICAL_ERROR); throw Exception("Connection pool for replica " + replica.readableString() + " does not exist", ErrorCodes::LOGICAL_ERROR);
job.connection_entry = connection_pool->get(&context.getSettingsRef()); job.connection_entry = connection_pool->get(timeouts, &settings);
if (job.connection_entry.isNull()) if (job.connection_entry.isNull())
throw Exception("Got empty connection for replica" + replica.readableString(), ErrorCodes::LOGICAL_ERROR); throw Exception("Got empty connection for replica" + replica.readableString(), ErrorCodes::LOGICAL_ERROR);
} }
@ -271,7 +273,7 @@ ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutp
if (throttler) if (throttler)
job.connection_entry->setThrottler(throttler); job.connection_entry->setThrottler(throttler);
job.stream = std::make_shared<RemoteBlockOutputStream>(*job.connection_entry, query_string, &context.getSettingsRef()); job.stream = std::make_shared<RemoteBlockOutputStream>(*job.connection_entry, timeouts, query_string, &settings);
job.stream->writePrefix(); job.stream->writePrefix();
} }

View File

@ -20,7 +20,7 @@ NameSet injectRequiredColumns(const MergeTreeData & storage, const MergeTreeData
const auto & column_name = columns[i]; const auto & column_name = columns[i];
/// column has files and hence does not require evaluation /// column has files and hence does not require evaluation
if (part->hasColumnFiles(column_name)) if (part->hasColumnFiles(column_name, *storage.getColumn(column_name).type))
{ {
all_column_files_missing = false; all_column_files_missing = false;
continue; continue;

View File

@ -1112,6 +1112,10 @@ void MergeTreeData::dropAllData()
LOG_TRACE(log, "dropAllData: removing data from filesystem."); LOG_TRACE(log, "dropAllData: removing data from filesystem.");
/// Removing of each data part before recursive removal of directory is to speed-up removal, because there will be less number of syscalls.
for (DataPartPtr part : data_parts_by_info) /// a copy intended
part->remove();
Poco::File(full_path).remove(true); Poco::File(full_path).remove(true);
LOG_TRACE(log, "dropAllData: done."); LOG_TRACE(log, "dropAllData: done.");
@ -1323,7 +1327,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name
if (!new_types.count(column.name)) if (!new_types.count(column.name))
{ {
/// The column was deleted. /// The column was deleted.
if (!part || part->hasColumnFiles(column.name)) if (!part || part->hasColumnFiles(column.name, *column.type))
{ {
column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
{ {
@ -1345,7 +1349,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name
const String new_type_name = new_type->getName(); const String new_type_name = new_type->getName();
const auto * old_type = column.type.get(); const auto * old_type = column.type.get();
if (!new_type->equals(*old_type) && (!part || part->hasColumnFiles(column.name))) if (!new_type->equals(*old_type) && (!part || part->hasColumnFiles(column.name, *column.type)))
{ {
if (isMetadataOnlyConversion(old_type, new_type)) if (isMetadataOnlyConversion(old_type, new_type))
{ {

View File

@ -34,6 +34,7 @@ namespace ErrorCodes
extern const int NOT_FOUND_EXPECTED_DATA_PART; extern const int NOT_FOUND_EXPECTED_DATA_PART;
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART; extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
extern const int BAD_TTL_FILE; extern const int BAD_TTL_FILE;
extern const int CANNOT_UNLINK;
} }
@ -216,7 +217,7 @@ String MergeTreeDataPart::getColumnNameWithMinumumCompressedSize() const
for (const auto & column : storage_columns) for (const auto & column : storage_columns)
{ {
if (!hasColumnFiles(column.name)) if (!hasColumnFiles(column.name, *column.type))
continue; continue;
const auto size = getColumnSize(column.name, *column.type).data_compressed; const auto size = getColumnSize(column.name, *column.type).data_compressed;
@ -395,8 +396,37 @@ void MergeTreeDataPart::remove() const
return; return;
} }
try
{
/// Remove each expected file in directory, then remove directory itself.
for (const auto & [file, _] : checksums.files)
{
String path_to_remove = to + "/" + file;
if (0 != unlink(path_to_remove.c_str()))
throwFromErrno("Cannot unlink file " + path_to_remove, ErrorCodes::CANNOT_UNLINK);
}
for (const auto & file : {"checksums.txt", "columns.txt"})
{
String path_to_remove = to + "/" + file;
if (0 != unlink(path_to_remove.c_str()))
throwFromErrno("Cannot unlink file " + path_to_remove, ErrorCodes::CANNOT_UNLINK);
}
if (0 != rmdir(to.c_str()))
throwFromErrno("Cannot rmdir file " + to, ErrorCodes::CANNOT_UNLINK);
}
catch (...)
{
/// Recursive directory removal does many excessive "stat" syscalls under the hood.
LOG_ERROR(storage.log, "Cannot quickly remove directory " << to << " by removing files; fallback to recursive removal. Reason: "
<< getCurrentExceptionMessage(false));
to_dir.remove(true); to_dir.remove(true);
} }
}
void MergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const void MergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const
@ -858,16 +888,22 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata)
} }
} }
bool MergeTreeDataPart::hasColumnFiles(const String & column) const bool MergeTreeDataPart::hasColumnFiles(const String & column_name, const IDataType & type) const
{ {
/// NOTE: For multi-streams columns we check that just first file exist. bool res = true;
/// That's Ok under assumption that files exist either for all or for no streams.
String prefix = getFullPath(); type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
{
String file_name = IDataType::getFileNameForStream(column_name, substream_path);
String escaped_column = escapeForFileName(column); auto bin_checksum = checksums.files.find(file_name + ".bin");
return Poco::File(prefix + escaped_column + ".bin").exists() auto mrk_checksum = checksums.files.find(file_name + storage.index_granularity_info.marks_file_extension);
&& Poco::File(prefix + escaped_column + storage.index_granularity_info.marks_file_extension).exists();
if (bin_checksum == checksums.files.end() || mrk_checksum == checksums.files.end())
res = false;
}, {});
return res;
} }

View File

@ -274,7 +274,7 @@ struct MergeTreeDataPart
void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency); void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency);
/// Checks that .bin and .mrk files exist /// Checks that .bin and .mrk files exist
bool hasColumnFiles(const String & column) const; bool hasColumnFiles(const String & column, const IDataType & type) const;
/// For data in RAM ('index') /// For data in RAM ('index')
UInt64 getIndexSizeInBytes() const; UInt64 getIndexSizeInBytes() const;

View File

@ -129,12 +129,14 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read,
prewhere_info && prewhere_info->remove_prewhere_column, per_part_should_reorder[part_idx], std::move(curr_task_size_predictor)); prewhere_info && prewhere_info->remove_prewhere_column, per_part_should_reorder[part_idx], std::move(curr_task_size_predictor));
} }
MarkRanges MergeTreeReadPool::getRestMarks(const std::string & part_path, const MarkRange & from) const MarkRanges MergeTreeReadPool::getRestMarks(const MergeTreeDataPart & part, const MarkRange & from) const
{ {
MarkRanges all_part_ranges; MarkRanges all_part_ranges;
/// Inefficient in presence of large number of data parts.
for (const auto & part_ranges : parts_ranges) for (const auto & part_ranges : parts_ranges)
{ {
if (part_ranges.data_part->getFullPath() == part_path) if (part_ranges.data_part.get() == &part)
{ {
all_part_ranges = part_ranges.ranges; all_part_ranges = part_ranges.ranges;
break; break;
@ -142,7 +144,7 @@ MarkRanges MergeTreeReadPool::getRestMarks(const std::string & part_path, const
} }
if (all_part_ranges.empty()) if (all_part_ranges.empty())
throw Exception("Trying to read marks range [" + std::to_string(from.begin) + ", " + std::to_string(from.end) + "] from part '" throw Exception("Trying to read marks range [" + std::to_string(from.begin) + ", " + std::to_string(from.end) + "] from part '"
+ part_path + "' which has no ranges in this query", ErrorCodes::LOGICAL_ERROR); + part.getFullPath() + "' which has no ranges in this query", ErrorCodes::LOGICAL_ERROR);
auto begin = std::lower_bound(all_part_ranges.begin(), all_part_ranges.end(), from, [] (const auto & f, const auto & s) { return f.begin < s.begin; }); auto begin = std::lower_bound(all_part_ranges.begin(), all_part_ranges.end(), from, [] (const auto & f, const auto & s) { return f.begin < s.begin; });
if (begin == all_part_ranges.end()) if (begin == all_part_ranges.end())

View File

@ -81,7 +81,7 @@ public:
void profileFeedback(const ReadBufferFromFileBase::ProfileInfo info); void profileFeedback(const ReadBufferFromFileBase::ProfileInfo info);
/// This method tells which mark ranges we have to read if we start from @from mark range /// This method tells which mark ranges we have to read if we start from @from mark range
MarkRanges getRestMarks(const std::string & part_path, const MarkRange & from) const; MarkRanges getRestMarks(const MergeTreeDataPart & part, const MarkRange & from) const;
Block getHeader() const; Block getHeader() const;

View File

@ -44,9 +44,6 @@ MergeTreeReader::MergeTreeReader(const String & path,
{ {
try try
{ {
if (!Poco::File(path).exists())
throw Exception("Part " + path + " is missing", ErrorCodes::NOT_FOUND_EXPECTED_DATA_PART);
for (const NameAndTypePair & column : columns) for (const NameAndTypePair & column : columns)
addStreams(column.name, *column.type, profile_callback, clock_type); addStreams(column.name, *column.type, profile_callback, clock_type);
} }
@ -163,7 +160,7 @@ void MergeTreeReader::addStreams(const String & name, const IDataType & type,
if (streams.count(stream_name)) if (streams.count(stream_name))
return; return;
bool data_file_exists = Poco::File(path + stream_name + DATA_FILE_EXTENSION).exists(); bool data_file_exists = data_part->checksums.files.count(stream_name + DATA_FILE_EXTENSION);
/** If data file is missing then we will not try to open it. /** If data file is missing then we will not try to open it.
* It is necessary since it allows to add new column to structure of the table without creating new files for old parts. * It is necessary since it allows to add new column to structure of the table without creating new files for old parts.

View File

@ -74,7 +74,7 @@ bool MergeTreeThreadSelectBlockInputStream::getNewTask()
if (!reader) if (!reader)
{ {
auto rest_mark_ranges = pool->getRestMarks(path, task->mark_ranges[0]); auto rest_mark_ranges = pool->getRestMarks(*task->data_part, task->mark_ranges[0]);
if (use_uncompressed_cache) if (use_uncompressed_cache)
owned_uncompressed_cache = storage.global_context.getUncompressedCache(); owned_uncompressed_cache = storage.global_context.getUncompressedCache();
@ -95,7 +95,7 @@ bool MergeTreeThreadSelectBlockInputStream::getNewTask()
/// in other case we can reuse readers, anyway they will be "seeked" to required mark /// in other case we can reuse readers, anyway they will be "seeked" to required mark
if (path != last_readed_part_path) if (path != last_readed_part_path)
{ {
auto rest_mark_ranges = pool->getRestMarks(path, task->mark_ranges[0]); auto rest_mark_ranges = pool->getRestMarks(*task->data_part, task->mark_ranges[0]);
/// retain avg_value_size_hints /// retain avg_value_size_hints
reader = std::make_unique<MergeTreeReader>( reader = std::make_unique<MergeTreeReader>(
path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), save_marks_in_cache, path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), save_marks_in_cache,

View File

@ -20,7 +20,7 @@ namespace DB
struct ReplicatedMergeTreeQuorumEntry struct ReplicatedMergeTreeQuorumEntry
{ {
String part_name; String part_name;
size_t required_number_of_replicas; size_t required_number_of_replicas{};
std::set<String> replicas; std::set<String> replicas;
ReplicatedMergeTreeQuorumEntry() {} ReplicatedMergeTreeQuorumEntry() {}

View File

@ -65,6 +65,10 @@ namespace ErrorCodes
extern const int TOO_MANY_ROWS; extern const int TOO_MANY_ROWS;
} }
namespace ActionLocks
{
extern const StorageActionBlockType DistributedSend;
}
namespace namespace
{ {
@ -427,7 +431,7 @@ void StorageDistributed::createDirectoryMonitors()
void StorageDistributed::requireDirectoryMonitor(const std::string & name) void StorageDistributed::requireDirectoryMonitor(const std::string & name)
{ {
std::lock_guard lock(cluster_nodes_mutex); std::lock_guard lock(cluster_nodes_mutex);
cluster_nodes_data[name].requireDirectoryMonitor(name, *this); cluster_nodes_data[name].requireDirectoryMonitor(name, *this, monitors_blocker);
} }
ConnectionPoolPtr StorageDistributed::requireConnectionPool(const std::string & name) ConnectionPoolPtr StorageDistributed::requireConnectionPool(const std::string & name)
@ -454,11 +458,17 @@ void StorageDistributed::ClusterNodeData::requireConnectionPool(const std::strin
conneciton_pool = StorageDistributedDirectoryMonitor::createPool(name, storage); conneciton_pool = StorageDistributedDirectoryMonitor::createPool(name, storage);
} }
void StorageDistributed::ClusterNodeData::requireDirectoryMonitor(const std::string & name, StorageDistributed & storage) void StorageDistributed::ClusterNodeData::requireDirectoryMonitor(
const std::string & name, StorageDistributed & storage, ActionBlocker & monitor_blocker)
{ {
requireConnectionPool(name, storage); requireConnectionPool(name, storage);
if (!directory_monitor) if (!directory_monitor)
directory_monitor = std::make_unique<StorageDistributedDirectoryMonitor>(storage, name, conneciton_pool); directory_monitor = std::make_unique<StorageDistributedDirectoryMonitor>(storage, name, conneciton_pool, monitor_blocker);
}
void StorageDistributed::ClusterNodeData::flushAllData()
{
directory_monitor->flushAllData();
} }
void StorageDistributed::ClusterNodeData::shutdownAndDropAllData() void StorageDistributed::ClusterNodeData::shutdownAndDropAllData()
@ -499,6 +509,22 @@ ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const Select
return cluster->getClusterWithMultipleShards({shards.begin(), shards.end()}); return cluster->getClusterWithMultipleShards({shards.begin(), shards.end()});
} }
ActionLock StorageDistributed::getActionLock(StorageActionBlockType type)
{
if (type == ActionLocks::DistributedSend)
return monitors_blocker.cancel();
return {};
}
void StorageDistributed::flushClusterNodesAllData()
{
std::lock_guard lock(cluster_nodes_mutex);
/// TODO: Maybe it should be executed in parallel
for (auto it = cluster_nodes_data.begin(); it != cluster_nodes_data.end(); ++it)
it->second.flushAllData();
}
void registerStorageDistributed(StorageFactory & factory) void registerStorageDistributed(StorageFactory & factory)
{ {

View File

@ -11,6 +11,7 @@
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Common/ActionBlocker.h>
namespace DB namespace DB
@ -105,8 +106,11 @@ public:
/// ensure connection pool creation and return it /// ensure connection pool creation and return it
ConnectionPoolPtr requireConnectionPool(const std::string & name); ConnectionPoolPtr requireConnectionPool(const std::string & name);
void flushClusterNodesAllData();
ClusterPtr getCluster() const; ClusterPtr getCluster() const;
ActionLock getActionLock(StorageActionBlockType type) override;
String table_name; String table_name;
String remote_database; String remote_database;
@ -135,7 +139,9 @@ public:
/// Creates connection_pool if not exists. /// Creates connection_pool if not exists.
void requireConnectionPool(const std::string & name, const StorageDistributed & storage); void requireConnectionPool(const std::string & name, const StorageDistributed & storage);
/// Creates directory_monitor if not exists. /// Creates directory_monitor if not exists.
void requireDirectoryMonitor(const std::string & name, StorageDistributed & storage); void requireDirectoryMonitor(const std::string & name, StorageDistributed & storage, ActionBlocker & monitor_blocker);
void flushAllData();
void shutdownAndDropAllData(); void shutdownAndDropAllData();
}; };
@ -145,6 +151,8 @@ public:
/// Used for global monotonic ordering of files to send. /// Used for global monotonic ordering of files to send.
SimpleIncrement file_names_increment; SimpleIncrement file_names_increment;
ActionBlocker monitors_blocker;
protected: protected:
StorageDistributed( StorageDistributed(
const String & database_name, const String & database_name,

View File

@ -370,6 +370,11 @@ void StorageMaterializedView::checkPartitionCanBeDropped(const ASTPtr & partitio
target_table->checkPartitionCanBeDropped(partition); target_table->checkPartitionCanBeDropped(partition);
} }
ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type)
{
return has_inner_table ? getTargetTable()->getActionLock(type) : ActionLock{};
}
void registerStorageMaterializedView(StorageFactory & factory) void registerStorageMaterializedView(StorageFactory & factory)
{ {
factory.registerStorage("MaterializedView", [](const StorageFactory::Arguments & args) factory.registerStorage("MaterializedView", [](const StorageFactory::Arguments & args)

View File

@ -51,6 +51,8 @@ public:
StoragePtr getTargetTable() const; StoragePtr getTargetTable() const;
StoragePtr tryGetTargetTable() const; StoragePtr tryGetTargetTable() const;
ActionLock getActionLock(StorageActionBlockType type) override;
BlockInputStreams read( BlockInputStreams read(
const Names & column_names, const Names & column_names,
const SelectQueryInfo & query_info, const SelectQueryInfo & query_info,

View File

@ -3986,8 +3986,6 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query
else else
throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED); throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED);
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(global_context.getSettingsRef());
const auto & query_settings = query_context.getSettingsRef(); const auto & query_settings = query_context.getSettingsRef();
const auto & query_client_info = query_context.getClientInfo(); const auto & query_client_info = query_context.getClientInfo();
String user = query_client_info.current_user; String user = query_client_info.current_user;
@ -4003,7 +4001,7 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query
leader_address.host, leader_address.host,
leader_address.queries_port, leader_address.queries_port,
leader_address.database, leader_address.database,
user, password, timeouts, "Follower replica"); user, password, "Follower replica");
std::stringstream new_query_ss; std::stringstream new_query_ss;
formatAST(*new_query, new_query_ss, false, true); formatAST(*new_query, new_query_ss, false, true);

View File

@ -14,7 +14,7 @@ Don't use Docker from your system repository.
* [pip](https://pypi.python.org/pypi/pip). To install: `sudo apt-get install python-pip` * [pip](https://pypi.python.org/pypi/pip). To install: `sudo apt-get install python-pip`
* [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest` * [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest`
* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python` * [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python protobuf`
(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka` (highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka`

View File

@ -0,0 +1,18 @@
<yandex>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</yandex>

View File

@ -0,0 +1,35 @@
<yandex>
<profiles>
<default>
<connections_with_failover_max_tries>3</connections_with_failover_max_tries>
<connect_timeout_with_failover_ms>1000</connect_timeout_with_failover_ms>
<min_insert_block_size_rows>1</min_insert_block_size_rows>
</default>
<delays>
<connections_with_failover_max_tries>5</connections_with_failover_max_tries>
<connect_timeout_with_failover_ms>3000</connect_timeout_with_failover_ms>
<min_insert_block_size_rows>1</min_insert_block_size_rows>
</delays>
</profiles>
<users>
<default>
<password></password>
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</default>
<ready_to_wait>
<password></password>
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<profile>delays</profile>
<quota>default</quota>
</ready_to_wait>
</users>
<quotas><default></default></quotas>
</yandex>

View File

@ -0,0 +1,141 @@
import itertools
import timeit
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
NODES = {'node' + str(i): cluster.add_instance(
'node' + str(i),
main_configs=['configs/remote_servers.xml'],
user_configs=['configs/set_distributed_defaults.xml'],
) for i in (1, 2)}
CREATE_TABLES_SQL = '''
CREATE DATABASE test;
CREATE TABLE base_table(
node String
)
ENGINE = MergeTree
PARTITION BY node
ORDER BY node;
CREATE TABLE distributed_table
ENGINE = Distributed(test_cluster, default, base_table) AS base_table;
'''
INSERT_SQL_TEMPLATE = "INSERT INTO base_table VALUES ('{node_id}')"
SELECTS_SQL = {
'distributed': 'SELECT node FROM distributed_table ORDER BY node',
'remote': ("SELECT node FROM remote('node1,node2', default.base_table) "
"ORDER BY node"),
}
EXCEPTION_NETWORK = 'e.displayText() = DB::NetException: '
EXCEPTION_TIMEOUT = 'Timeout exceeded while reading from socket ('
EXCEPTION_CONNECT = 'Timeout: connect timed out: '
TIMEOUT_MEASUREMENT_EPS = 0.01
EXPECTED_BEHAVIOR = {
'default': {
'times': 3,
'timeout': 1,
},
'ready_to_wait': {
'times': 5,
'timeout': 3,
},
}
def _check_exception(exception, expected_tries=3):
lines = exception.split('\n')
assert len(lines) > 4, "Unexpected exception (expected: timeout info)"
assert lines[0].startswith('Received exception from server (version')
assert lines[1].startswith('Code: 279')
assert lines[1].endswith('All connection tries failed. Log: ')
assert lines[2] == '', "Unexpected exception text (expected: empty line)"
for i, line in enumerate(lines[3:3 + expected_tries]):
expected_lines = (
'Code: 209, ' + EXCEPTION_NETWORK + EXCEPTION_TIMEOUT,
'Code: 209, ' + EXCEPTION_NETWORK + EXCEPTION_CONNECT,
)
assert any(line.startswith(expected) for expected in expected_lines), \
'Unexpected exception at one of the connection attempts'
assert lines[3 + expected_tries] == '', 'Wrong number of connect attempts'
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
for node_id, node in NODES.items():
node.query(CREATE_TABLES_SQL)
node.query(INSERT_SQL_TEMPLATE.format(node_id=node_id))
yield cluster
finally:
cluster.shutdown()
def _check_timeout_and_exception(node, user, query_base):
repeats = EXPECTED_BEHAVIOR[user]['times']
expected_timeout = EXPECTED_BEHAVIOR[user]['timeout'] * repeats
start = timeit.default_timer()
exception = node.query_and_get_error(SELECTS_SQL[query_base], user=user)
# And it should timeout no faster than:
measured_timeout = timeit.default_timer() - start
assert measured_timeout >= expected_timeout - TIMEOUT_MEASUREMENT_EPS
# And exception should reflect connection attempts:
_check_exception(exception, repeats)
@pytest.mark.parametrize(
('first_user', 'node_name', 'query_base'),
tuple(itertools.product(EXPECTED_BEHAVIOR, NODES, SELECTS_SQL)),
)
def test_reconnect(started_cluster, node_name, first_user, query_base):
node = NODES[node_name]
# Everything is up, select should work:
assert TSV(node.query(SELECTS_SQL[query_base],
user=first_user)) == TSV('node1\nnode2')
with PartitionManager() as pm:
# Break the connection.
pm.partition_instances(*NODES.values())
# Now it shouldn't:
_check_timeout_and_exception(node, first_user, query_base)
# Other user should have different timeout and exception
_check_timeout_and_exception(
node,
'default' if first_user != 'default' else 'ready_to_wait',
query_base,
)
# select should work again:
assert TSV(node.query(SELECTS_SQL[query_base],
user=first_user)) == TSV('node1\nnode2')

View File

@ -0,0 +1,18 @@
<yandex>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</yandex>

View File

@ -0,0 +1,41 @@
from contextlib import contextmanager
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'])
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'])
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
for node in (node1, node2):
node.query('''CREATE TABLE local_table(id UInt32, val String) ENGINE = MergeTree ORDER BY id;''')
node1.query('''CREATE TABLE distributed_table(id UInt32, val String) ENGINE = Distributed(test_cluster, default, local_table, id);''')
yield cluster
finally:
cluster.shutdown()
def test_start_and_stop_replica_send(started_cluster):
node1.query("SYSTEM STOP DISTRIBUTED SENDS distributed_table;")
node1.query("INSERT INTO distributed_table VALUES (0, 'node1')")
node1.query("INSERT INTO distributed_table VALUES (1, 'node2')")
# Write only to this node when stop distributed sends
assert node1.query("SELECT COUNT() FROM distributed_table").rstrip() == '1'
node1.query("SYSTEM START DISTRIBUTED SENDS distributed_table;")
node1.query("SYSTEM FLUSH DISTRIBUTED distributed_table;")
assert node1.query("SELECT COUNT() FROM distributed_table").rstrip() == '2'

View File

@ -51,7 +51,7 @@ def test_mysql_client(mysql_client, server_address):
-e "SELECT 'тест' as b;" -e "SELECT 'тест' as b;"
'''.format(host=server_address, port=server_port), demux=True) '''.format(host=server_address, port=server_port), demux=True)
assert stdout == 'a\n1\nb\nтест\n' assert stdout == '\n'.join(['a', '1', 'b', 'тест', ''])
code, (stdout, stderr) = mysql_client.exec_run(''' code, (stdout, stderr) = mysql_client.exec_run('''
mysql --protocol tcp -h {host} -P {port} default -u default --password=abc -e "select 1 as a;" mysql --protocol tcp -h {host} -P {port} default -u default --password=abc -e "select 1 as a;"
@ -75,14 +75,17 @@ def test_mysql_client(mysql_client, server_address):
mysql --protocol tcp -h {host} -P {port} default -u default --password=123 mysql --protocol tcp -h {host} -P {port} default -u default --password=123
-e "CREATE DATABASE x;" -e "CREATE DATABASE x;"
-e "USE x;" -e "USE x;"
-e "CREATE TABLE table1 (a UInt32) ENGINE = Memory;" -e "CREATE TABLE table1 (column UInt32) ENGINE = Memory;"
-e "INSERT INTO table1 VALUES (0), (1), (5);" -e "INSERT INTO table1 VALUES (0), (1), (5);"
-e "INSERT INTO table1 VALUES (0), (1), (5);" -e "INSERT INTO table1 VALUES (0), (1), (5);"
-e "SELECT * FROM table1 ORDER BY a;" -e "SELECT * FROM table1 ORDER BY column;"
-e "DROP DATABASE x;" -e "DROP DATABASE x;"
-e "CREATE TEMPORARY TABLE tmp (tmp_column UInt32);"
-e "INSERT INTO tmp VALUES (0), (1);"
-e "SELECT * FROM tmp ORDER BY tmp_column;"
'''.format(host=server_address, port=server_port), demux=True) '''.format(host=server_address, port=server_port), demux=True)
assert stdout == 'a\n0\n0\n1\n1\n5\n5\n' assert stdout == '\n'.join(['column', '0', '0', '1', '1', '5', '5', 'tmp_column', '0', '1', ''])
def test_python_client(server_address): def test_python_client(server_address):

View File

@ -0,0 +1,30 @@
<test>
<name>merge_tree_many_partitions</name>
<type>loop</type>
<create_query>CREATE TABLE bad_partitions (x UInt64) ENGINE = MergeTree PARTITION BY x ORDER BY x</create_query>
<fill_query>INSERT INTO bad_partitions SELECT * FROM numbers(10000)</fill_query>
<stop_conditions>
<all_of>
<iterations>5</iterations>
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
</all_of>
<any_of>
<iterations>100</iterations>
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<settings>
<max_partitions_per_insert_block>0</max_partitions_per_insert_block>
</settings>
<query>SELECT count() FROM bad_partitions</query>
<drop_query>DROP TABLE IF EXISTS bad_partitions</drop_query>
</test>

View File

@ -0,0 +1,30 @@
<test>
<name>merge_tree_many_partitions_2</name>
<type>loop</type>
<create_query>CREATE TABLE bad_partitions (a UInt64, b UInt64, c UInt64, d UInt64, e UInt64, f UInt64, g UInt64, h UInt64, i UInt64, j UInt64, k UInt64, l UInt64, m UInt64, n UInt64, o UInt64, p UInt64, q UInt64, r UInt64, s UInt64, t UInt64, u UInt64, v UInt64, w UInt64, x UInt64, y UInt64, z UInt64) ENGINE = MergeTree PARTITION BY x ORDER BY x</create_query>
<fill_query>INSERT INTO bad_partitions (x) SELECT * FROM numbers(10000)</fill_query>
<stop_conditions>
<all_of>
<iterations>5</iterations>
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
</all_of>
<any_of>
<iterations>100</iterations>
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<settings>
<max_partitions_per_insert_block>0</max_partitions_per_insert_block>
</settings>
<query>SELECT sum(ignore(*)) FROM bad_partitions</query>
<drop_query>DROP TABLE IF EXISTS bad_partitions</drop_query>
</test>

View File

@ -18,9 +18,9 @@ SELECT * FROM merge(currentDatabase(), 'test_local_1');
SELECT *, _table FROM merge(currentDatabase(), 'test_local_1') ORDER BY _table; SELECT *, _table FROM merge(currentDatabase(), 'test_local_1') ORDER BY _table;
SELECT sum(value), _table FROM merge(currentDatabase(), 'test_local_1') GROUP BY _table ORDER BY _table; SELECT sum(value), _table FROM merge(currentDatabase(), 'test_local_1') GROUP BY _table ORDER BY _table;
SELECT * FROM merge(currentDatabase(), 'test_local_1') WHERE _table = 'test_local_1'; SELECT * FROM merge(currentDatabase(), 'test_local_1') WHERE _table = 'test_local_1';
SELECT * FROM merge(currentDatabase(), 'test_local_1') PREWHERE _table = 'test_local_1'; -- { serverError 8 } SELECT * FROM merge(currentDatabase(), 'test_local_1') PREWHERE _table = 'test_local_1'; -- { serverError 16 }
SELECT * FROM merge(currentDatabase(), 'test_local_1') WHERE _table in ('test_local_1', 'test_local_2'); SELECT * FROM merge(currentDatabase(), 'test_local_1') WHERE _table in ('test_local_1', 'test_local_2');
SELECT * FROM merge(currentDatabase(), 'test_local_1') PREWHERE _table in ('test_local_1', 'test_local_2'); -- { serverError 8 } SELECT * FROM merge(currentDatabase(), 'test_local_1') PREWHERE _table in ('test_local_1', 'test_local_2'); -- { serverError 16 }
SELECT '--------------Single Distributed------------'; SELECT '--------------Single Distributed------------';
SELECT * FROM merge(currentDatabase(), 'test_distributed_1'); SELECT * FROM merge(currentDatabase(), 'test_distributed_1');
@ -36,9 +36,9 @@ SELECT * FROM merge(currentDatabase(), 'test_local_1|test_local_2') ORDER BY _ta
SELECT *, _table FROM merge(currentDatabase(), 'test_local_1|test_local_2') ORDER BY _table; SELECT *, _table FROM merge(currentDatabase(), 'test_local_1|test_local_2') ORDER BY _table;
SELECT sum(value), _table FROM merge(currentDatabase(), 'test_local_1|test_local_2') GROUP BY _table ORDER BY _table; SELECT sum(value), _table FROM merge(currentDatabase(), 'test_local_1|test_local_2') GROUP BY _table ORDER BY _table;
SELECT * FROM merge(currentDatabase(), 'test_local_1|test_local_2') WHERE _table = 'test_local_1'; SELECT * FROM merge(currentDatabase(), 'test_local_1|test_local_2') WHERE _table = 'test_local_1';
SELECT * FROM merge(currentDatabase(), 'test_local_1|test_local_2') PREWHERE _table = 'test_local_1'; -- {serverError 8} SELECT * FROM merge(currentDatabase(), 'test_local_1|test_local_2') PREWHERE _table = 'test_local_1'; -- { serverError 16 }
SELECT * FROM merge(currentDatabase(), 'test_local_1|test_local_2') WHERE _table in ('test_local_1', 'test_local_2') ORDER BY value; SELECT * FROM merge(currentDatabase(), 'test_local_1|test_local_2') WHERE _table in ('test_local_1', 'test_local_2') ORDER BY value;
SELECT * FROM merge(currentDatabase(), 'test_local_1|test_local_2') PREWHERE _table in ('test_local_1', 'test_local_2') ORDER BY value; -- {serverError 8} SELECT * FROM merge(currentDatabase(), 'test_local_1|test_local_2') PREWHERE _table in ('test_local_1', 'test_local_2') ORDER BY value; -- { serverError 16 }
SELECT '--------------Local Merge Distributed------------'; SELECT '--------------Local Merge Distributed------------';
SELECT * FROM merge(currentDatabase(), 'test_local_1|test_distributed_2') ORDER BY _table; SELECT * FROM merge(currentDatabase(), 'test_local_1|test_distributed_2') ORDER BY _table;

View File

@ -0,0 +1,7 @@
0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0
1 1 1 1 1 1 1 1
42 42 42 42 42 42 42 42
42 42 42 42 42 42 42 42
42 42 42 42 42 42 42 42
42 42 42 42 42 42 42 42

View File

@ -0,0 +1,101 @@
DROP TABLE IF EXISTS test.t64;
CREATE TABLE test.t64
(
u8 UInt8,
t_u8 UInt8 Codec(T64, ZSTD),
u16 UInt16,
t_u16 UInt16 Codec(T64, ZSTD),
u32 UInt32,
t_u32 UInt32 Codec(T64, ZSTD),
u64 UInt64,
t_u64 UInt64 Codec(T64, ZSTD)
) ENGINE MergeTree() ORDER BY tuple();
INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(1);
INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(2);
INSERT INTO test.t64 SELECT 42 AS x, x, x, x, x, x, x, x FROM numbers(4);
SELECT * FROM test.t64 ORDER BY u64;
INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(8));
INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(9));
SELECT * FROM test.t64 WHERE u8 != t_u8;
SELECT * FROM test.t64 WHERE u16 != t_u16;
SELECT * FROM test.t64 WHERE u32 != t_u32;
SELECT * FROM test.t64 WHERE u64 != t_u64;
INSERT INTO test.t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11);
INSERT INTO test.t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64);
INSERT INTO test.t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65);
INSERT INTO test.t64 SELECT (intExp2(16) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65);
SELECT * FROM test.t64 WHERE u8 != t_u8;
SELECT * FROM test.t64 WHERE u16 != t_u16;
SELECT * FROM test.t64 WHERE u32 != t_u32;
SELECT * FROM test.t64 WHERE u64 != t_u64;
INSERT INTO test.t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11);
INSERT INTO test.t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(128);
INSERT INTO test.t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(129);
INSERT INTO test.t64 SELECT (intExp2(24) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(129);
SELECT * FROM test.t64 WHERE u8 != t_u8;
SELECT * FROM test.t64 WHERE u16 != t_u16;
SELECT * FROM test.t64 WHERE u32 != t_u32;
SELECT * FROM test.t64 WHERE u64 != t_u64;
INSERT INTO test.t64 SELECT (intExp2(32) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(32) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
INSERT INTO test.t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(256);
INSERT INTO test.t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(257);
INSERT INTO test.t64 SELECT (intExp2(32) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(257);
SELECT * FROM test.t64 WHERE u8 != t_u8;
SELECT * FROM test.t64 WHERE u16 != t_u16;
SELECT * FROM test.t64 WHERE u32 != t_u32;
SELECT * FROM test.t64 WHERE u64 != t_u64;
INSERT INTO test.t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
INSERT INTO test.t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(512);
INSERT INTO test.t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(513);
INSERT INTO test.t64 SELECT (intExp2(40) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(513);
SELECT * FROM test.t64 WHERE u8 != t_u8;
SELECT * FROM test.t64 WHERE u16 != t_u16;
SELECT * FROM test.t64 WHERE u32 != t_u32;
SELECT * FROM test.t64 WHERE u64 != t_u64;
INSERT INTO test.t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
INSERT INTO test.t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1024);
INSERT INTO test.t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025);
INSERT INTO test.t64 SELECT (intExp2(48) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025);
SELECT * FROM test.t64 WHERE u8 != t_u8;
SELECT * FROM test.t64 WHERE u16 != t_u16;
SELECT * FROM test.t64 WHERE u32 != t_u32;
SELECT * FROM test.t64 WHERE u64 != t_u64;
INSERT INTO test.t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
INSERT INTO test.t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2048);
INSERT INTO test.t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049);
INSERT INTO test.t64 SELECT (intExp2(56) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049);
SELECT * FROM test.t64 WHERE u8 != t_u8;
SELECT * FROM test.t64 WHERE u16 != t_u16;
SELECT * FROM test.t64 WHERE u32 != t_u32;
SELECT * FROM test.t64 WHERE u64 != t_u64;
INSERT INTO test.t64 SELECT (intExp2(63) + number * intExp2(62)) AS x, x, x, x, x, x, x, x FROM numbers(10);
SELECT * FROM test.t64 WHERE u8 != t_u8;
SELECT * FROM test.t64 WHERE u16 != t_u16;
SELECT * FROM test.t64 WHERE u32 != t_u32;
SELECT * FROM test.t64 WHERE u64 != t_u64;
OPTIMIZE TABLE test.t64 FINAL;
SELECT * FROM test.t64 WHERE u8 != t_u8;
SELECT * FROM test.t64 WHERE u16 != t_u16;
SELECT * FROM test.t64 WHERE u32 != t_u32;
SELECT * FROM test.t64 WHERE u64 != t_u64;
DROP TABLE test.t64;

View File

@ -0,0 +1,9 @@
-1 -1 -1 -1 -1 -1 -1 -1
-1 -1 -1 -1 -1 -1 -1 -1
0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0
1 1 1 1 1 1 1 1
42 42 42 42 42 42 42 42
42 42 42 42 42 42 42 42
42 42 42 42 42 42 42 42
42 42 42 42 42 42 42 42

View File

@ -0,0 +1,128 @@
DROP TABLE IF EXISTS test.t64;
CREATE TABLE test.t64
(
i8 Int8,
t_i8 Int8 Codec(T64, LZ4),
i16 Int16,
t_i16 Int16 Codec(T64, LZ4),
i32 Int32,
t_i32 Int32 Codec(T64, LZ4),
i64 Int64,
t_i64 Int64 Codec(T64, LZ4)
) ENGINE MergeTree() ORDER BY tuple();
INSERT INTO test.t64 SELECT toInt32(number)-1 AS x, x, x, x, x, x, x, x FROM numbers(2);
INSERT INTO test.t64 SELECT toInt32(number)-1 AS x, x, x, x, x, x, x, x FROM numbers(3);
INSERT INTO test.t64 SELECT 42 AS x, x, x, x, x, x, x, x FROM numbers(4);
SELECT * FROM test.t64 ORDER BY i64;
INSERT INTO test.t64 SELECT (intExp2(8) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
SELECT i8, t_i8 FROM test.t64 WHERE i8 != t_i8;
INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(8));
INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(9));
SELECT * FROM test.t64 WHERE i8 != t_i8;
SELECT * FROM test.t64 WHERE i16 != t_i16;
SELECT * FROM test.t64 WHERE i32 != t_i32;
SELECT * FROM test.t64 WHERE i64 != t_i64;
INSERT INTO test.t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11);
INSERT INTO test.t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64);
INSERT INTO test.t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65);
INSERT INTO test.t64 SELECT (intExp2(16) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(11);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(64);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65);
INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65);
SELECT * FROM test.t64 WHERE i8 != t_i8;
SELECT * FROM test.t64 WHERE i16 != t_i16;
SELECT * FROM test.t64 WHERE i32 != t_i32;
SELECT * FROM test.t64 WHERE i64 != t_i64;
INSERT INTO test.t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11);
INSERT INTO test.t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(128);
INSERT INTO test.t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(129);
INSERT INTO test.t64 SELECT (intExp2(24) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(129);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(11);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(128);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(129);
INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(129);
SELECT * FROM test.t64 WHERE i8 != t_i8;
SELECT * FROM test.t64 WHERE i16 != t_i16;
SELECT * FROM test.t64 WHERE i32 != t_i32;
SELECT * FROM test.t64 WHERE i64 != t_i64;
INSERT INTO test.t64 SELECT (intExp2(32) - 2 + number) AS x, x, x, x, x, x, x, x FROM numbers(2);
INSERT INTO test.t64 SELECT (intExp2(32) - 2 + number) AS x, x, x, x, x, x, x, x FROM numbers(3);
INSERT INTO test.t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64);
INSERT INTO test.t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65);
INSERT INTO test.t64 SELECT (intExp2(32) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(11);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(64);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65);
INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65);
SELECT * FROM test.t64 WHERE i8 != t_i8;
SELECT * FROM test.t64 WHERE i16 != t_i16;
SELECT * FROM test.t64 WHERE i32 != t_i32;
SELECT * FROM test.t64 WHERE i64 != t_i64;
INSERT INTO test.t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
INSERT INTO test.t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(512);
INSERT INTO test.t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(513);
INSERT INTO test.t64 SELECT (intExp2(40) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(513);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(512);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(513);
INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(513);
SELECT * FROM test.t64 WHERE i8 != t_i8;
SELECT * FROM test.t64 WHERE i16 != t_i16;
SELECT * FROM test.t64 WHERE i32 != t_i32;
SELECT * FROM test.t64 WHERE i64 != t_i64;
INSERT INTO test.t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
INSERT INTO test.t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1024);
INSERT INTO test.t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025);
INSERT INTO test.t64 SELECT (intExp2(48) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(1024);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(1025);
INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(1025);
SELECT * FROM test.t64 WHERE i8 != t_i8;
SELECT * FROM test.t64 WHERE i16 != t_i16;
SELECT * FROM test.t64 WHERE i32 != t_i32;
SELECT * FROM test.t64 WHERE i64 != t_i64;
INSERT INTO test.t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
INSERT INTO test.t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2048);
INSERT INTO test.t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049);
INSERT INTO test.t64 SELECT (intExp2(56) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10);
INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(20);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(2048);
INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(2049);
INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(2049);
SELECT * FROM test.t64 WHERE i8 != t_i8;
SELECT * FROM test.t64 WHERE i16 != t_i16;
SELECT * FROM test.t64 WHERE i32 != t_i32;
SELECT * FROM test.t64 WHERE i64 != t_i64;
OPTIMIZE TABLE test.t64 FINAL;
SELECT * FROM test.t64 WHERE i8 != t_i8;
SELECT * FROM test.t64 WHERE i16 != t_i16;
SELECT * FROM test.t64 WHERE i32 != t_i32;
SELECT * FROM test.t64 WHERE i64 != t_i64;
DROP TABLE test.t64;

View File

@ -163,31 +163,54 @@ clickhouse-client --query "select count(*) from datasets.ontime"
Q0. Q0.
``` sql ``` sql
select avg(c1) from (select Year, Month, count(*) as c1 from ontime group by Year, Month); SELECT avg(c1)
FROM
(
SELECT Year, Month, count(*) AS c1
FROM ontime
GROUP BY Year, Month
);
``` ```
Q1. The number of flights per day from the year 2000 to 2008 Q1. The number of flights per day from the year 2000 to 2008
``` sql ``` sql
SELECT DayOfWeek, count(*) AS c FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC; SELECT DayOfWeek, count(*) AS c
FROM ontime
WHERE Year>=2000 AND Year<=2008
GROUP BY DayOfWeek
ORDER BY c DESC;
``` ```
Q2. The number of flights delayed by more than 10 minutes, grouped by the day of the week, for 2000-2008 Q2. The number of flights delayed by more than 10 minutes, grouped by the day of the week, for 2000-2008
``` sql ``` sql
SELECT DayOfWeek, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC SELECT DayOfWeek, count(*) AS c
FROM ontime
WHERE DepDelay>10 AND Year>=2000 AND Year<=2008
GROUP BY DayOfWeek
ORDER BY c DESC;
``` ```
Q3. The number of delays by airport for 2000-2008 Q3. The number of delays by airport for 2000-2008
``` sql ``` sql
SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY Origin ORDER BY c DESC LIMIT 10 SELECT Origin, count(*) AS c
FROM ontime
WHERE DepDelay>10 AND Year>=2000 AND Year<=2008
GROUP BY Origin
ORDER BY c DESC
LIMIT 10;
``` ```
Q4. The number of delays by carrier for 2007 Q4. The number of delays by carrier for 2007
``` sql ``` sql
SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC SELECT Carrier, count(*)
FROM ontime
WHERE DepDelay>10 AND Year=2007
GROUP BY Carrier
ORDER BY count(*) DESC;
``` ```
Q5. The percentage of delays by carrier for 2007 Q5. The percentage of delays by carrier for 2007
@ -219,7 +242,11 @@ ORDER BY c3 DESC;
Better version of the same query: Better version of the same query:
``` sql ``` sql
SELECT Carrier, avg(DepDelay > 10) * 100 AS c3 FROM ontime WHERE Year = 2007 GROUP BY Carrier ORDER BY Carrier SELECT Carrier, avg(DepDelay>10)*100 AS c3
FROM ontime
WHERE Year=2007
GROUP BY Carrier
ORDER BY Carrier
``` ```
Q6. The previous request for a broader range of years, 2000-2008 Q6. The previous request for a broader range of years, 2000-2008
@ -251,7 +278,11 @@ ORDER BY c3 DESC;
Better version of the same query: Better version of the same query:
``` sql ``` sql
SELECT Carrier, avg(DepDelay > 10) * 100 AS c3 FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY Carrier ORDER BY Carrier SELECT Carrier, avg(DepDelay>10)*100 AS c3
FROM ontime
WHERE Year>=2000 AND Year<=2008
GROUP BY Carrier
ORDER BY Carrier;
``` ```
Q7. Percentage of flights delayed for more than 10 minutes, by year Q7. Percentage of flights delayed for more than 10 minutes, by year
@ -275,39 +306,49 @@ ANY INNER JOIN
from ontime from ontime
GROUP BY Year GROUP BY Year
) USING (Year) ) USING (Year)
ORDER BY Year ORDER BY Year;
``` ```
Better version of the same query: Better version of the same query:
``` sql ``` sql
SELECT Year, avg(DepDelay > 10) FROM ontime GROUP BY Year ORDER BY Year SELECT Year, avg(DepDelay>10)
FROM ontime
GROUP BY Year
ORDER BY Year;
``` ```
Q8. The most popular destinations by the number of directly connected cities for various year ranges Q8. The most popular destinations by the number of directly connected cities for various year ranges
``` sql ``` sql
SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime WHERE Year >= 2000 and Year <= 2010 GROUP BY DestCityName ORDER BY u DESC LIMIT 10; SELECT DestCityName, uniqExact(OriginCityName) AS u F
ROM ontime
WHERE Year>=2000 and Year<=2010
GROUP BY DestCityName
ORDER BY u DESC
LIMIT 10;
``` ```
Q9. Q9.
``` sql ``` sql
select Year, count(*) as c1 from ontime group by Year; SELECT Year, count(*) AS c1
FROM ontime
GROUP BY Year;
``` ```
Q10. Q10.
``` sql ``` sql
select SELECT
min(Year), max(Year), Carrier, count(*) as cnt, min(Year), max(Year), Carrier, count(*) AS cnt,
sum(ArrDelayMinutes>30) as flights_delayed, sum(ArrDelayMinutes>30) AS flights_delayed,
round(sum(ArrDelayMinutes>30)/count(*),2) as rate round(sum(ArrDelayMinutes>30)/count(*),2) AS rate
FROM ontime FROM ontime
WHERE WHERE
DayOfWeek not in (6,7) and OriginState not in ('AK', 'HI', 'PR', 'VI') DayOfWeek NOT IN (6,7) AND OriginState NOT IN ('AK', 'HI', 'PR', 'VI')
and DestState not in ('AK', 'HI', 'PR', 'VI') AND DestState NOT IN ('AK', 'HI', 'PR', 'VI')
and FlightDate < '2010-01-01' AND FlightDate < '2010-01-01'
GROUP by Carrier GROUP by Carrier
HAVING cnt>100000 and max(Year)>1990 HAVING cnt>100000 and max(Year)>1990
ORDER by rate DESC ORDER by rate DESC
@ -317,15 +358,39 @@ LIMIT 1000;
Bonus: Bonus:
``` sql ``` sql
SELECT avg(cnt) FROM (SELECT Year,Month,count(*) AS cnt FROM ontime WHERE DepDel15=1 GROUP BY Year,Month) SELECT avg(cnt)
FROM
(
SELECT Year,Month,count(*) AS cnt
FROM ontime
WHERE DepDel15=1
GROUP BY Year,Month
);
select avg(c1) from (select Year,Month,count(*) as c1 from ontime group by Year,Month) SELECT avg(c1) FROM
(
SELECT Year,Month,count(*) AS c1
FROM ontime
GROUP BY Year,Month
);
SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime GROUP BY DestCityName ORDER BY u DESC LIMIT 10; SELECT DestCityName, uniqExact(OriginCityName) AS u
FROM ontime
GROUP BY DestCityName
ORDER BY u DESC
LIMIT 10;
SELECT OriginCityName, DestCityName, count() AS c FROM ontime GROUP BY OriginCityName, DestCityName ORDER BY c DESC LIMIT 10; SELECT OriginCityName, DestCityName, count() AS c
FROM ontime
GROUP BY OriginCityName, DestCityName
ORDER BY c DESC
LIMIT 10;
SELECT OriginCityName, count() AS c FROM ontime GROUP BY OriginCityName ORDER BY c DESC LIMIT 10; SELECT OriginCityName, count() AS c
FROM ontime
GROUP BY OriginCityName
ORDER BY c DESC
LIMIT 10;
``` ```
This performance test was created by Vadim Tkachenko. See: This performance test was created by Vadim Tkachenko. See:

View File

@ -753,8 +753,8 @@ Default value: 0.
Enables or disables: Enables or disables:
1. Rewriting of queries with multiple [JOIN clauses](../../query_language/select.md#select-join) from the syntax with commas to the `JOIN ON/USING` syntax. If the setting value is 0, ClickHouse doesn't process queries with the syntax with commas, and throws an exception. 1. Rewriting queries for join from the syntax with commas to the `JOIN ON/USING` syntax. If the setting value is 0, ClickHouse doesn't process queries with syntax that uses commas, and throws an exception.
2. Converting of `CROSS JOIN` into `INNER JOIN` if conditions of join allow it. 2. Converting `CROSS JOIN` to `INNER JOIN` if `WHERE` conditions allow it.
Possible values: Possible values:
@ -763,6 +763,11 @@ Possible values:
Default value: 1. Default value: 1.
**See Also**
- [Multiple JOIN](../../query_language/select.md#select-join)
## count_distinct_implementation {#settings-count_distinct_implementation} ## count_distinct_implementation {#settings-count_distinct_implementation}
Specifies which of the `uniq*` functions should be used for performing the [COUNT(DISTINCT ...)](../../query_language/agg_functions/reference.md#agg_function-count) construction. Specifies which of the `uniq*` functions should be used for performing the [COUNT(DISTINCT ...)](../../query_language/agg_functions/reference.md#agg_function-count) construction.

View File

@ -2,7 +2,7 @@
Allows ClickHouse to connect to external databases via [ODBC](https://en.wikipedia.org/wiki/Open_Database_Connectivity). Allows ClickHouse to connect to external databases via [ODBC](https://en.wikipedia.org/wiki/Open_Database_Connectivity).
To implement ODBC connection safely, ClickHouse uses the separate program `clickhouse-odbc-bridge`. If the ODBC driver is loaded directly from the `clickhouse-server` program, the problems in the driver can crash the ClickHouse server. ClickHouse starts the `clickhouse-odbc-bridge` program automatically when it is required. The ODBC bridge program is installed by the same package as the `clickhouse-server`. To safely implement ODBC connections, ClickHouse uses a separate program `clickhouse-odbc-bridge`. If the ODBC driver is loaded directly from `clickhouse-server`, driver problems can crash the ClickHouse server. ClickHouse automatically starts `clickhouse-odbc-bridge` when it is required. The ODBC bridge program is installed from the same package as the `clickhouse-server`.
This engine supports the [Nullable](../../data_types/nullable.md) data type. This engine supports the [Nullable](../../data_types/nullable.md) data type.
@ -18,12 +18,12 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
ENGINE = ODBC(connection_settings, external_database, external_table) ENGINE = ODBC(connection_settings, external_database, external_table)
``` ```
See the detailed description of the [CREATE TABLE](../../query_language/create.md#create-table-query) query. See a detailed description of the [CREATE TABLE](../../query_language/create.md#create-table-query) query.
The table structure can be not the same as the source table structure: The table structure can differ from the source table structure:
- Names of columns should be the same as in the source table, but you can use just some of these columns in any order. - Column names should be the same as in the source table, but you can use just some of these columns and in any order.
- Types of columns may differ from the types in the source table. ClickHouse tries to [cast](../../query_language/functions/type_conversion_functions.md#type_conversion_function-cast) values into the ClickHouse data types. - Column types may differ from those in the source table. ClickHouse tries to [cast](../../query_language/functions/type_conversion_functions.md#type_conversion_function-cast) values to the ClickHouse data types.
**Engine Parameters** **Engine Parameters**
@ -33,13 +33,13 @@ The table structure can be not the same as the source table structure:
## Usage Example ## Usage Example
**Getting data from the local MySQL installation via ODBC** **Retrieving data from the local MySQL installation via ODBC**
This example is for linux Ubuntu 18.04 and MySQL server 5.7. This example is checked for Ubuntu Linux 18.04 and MySQL server 5.7.
Ensure that there are unixODBC and MySQL Connector are installed. Ensure that unixODBC and MySQL Connector are installed.
By default (if installed from packages) ClickHouse starts on behalf of the user `clickhouse`. Thus, you need to create and configure this user at MySQL server. By default (if installed from packages), ClickHouse starts as user `clickhouse`. Thus, you need to create and configure this user in the MySQL server.
``` ```
sudo mysql sudo mysql
@ -60,7 +60,7 @@ USERNAME = clickhouse
PASSWORD = clickhouse PASSWORD = clickhouse
``` ```
You can check the connection by the `isql` utility from the unixODBC installation. You can check the connection using the `isql` utility from the unixODBC installation.
``` ```
isql -v mysqlconn isql -v mysqlconn
@ -93,7 +93,7 @@ mysql> select * from test;
1 row in set (0,00 sec) 1 row in set (0,00 sec)
``` ```
Table in ClickHouse, getting data from the MySQL table: Table in ClickHouse, retrieving data from the MySQL table:
```sql ```sql
CREATE TABLE odbc_t CREATE TABLE odbc_t
@ -115,6 +115,6 @@ SELECT * FROM odbc_t
## See Also ## See Also
- [ODBC external dictionaries](../../query_language/dicts/external_dicts_dict_sources.md#dicts-external_dicts_dict_sources-odbc) - [ODBC external dictionaries](../../query_language/dicts/external_dicts_dict_sources.md#dicts-external_dicts_dict_sources-odbc)
- [ODBC table function](../../query_language/table_functions/odbc.md). - [ODBC table function](../../query_language/table_functions/odbc.md)
[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/jdbc/) <!--hide--> [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/odbc/) <!--hide-->

View File

@ -839,12 +839,12 @@ simpleLinearRegression(x, y)
Parameters: Parameters:
- `x` — Column with values of dependent variable. - `x` — Column with dependent variable values.
- `y` — Column with explanatory variable. - `y` — Column with explanatory variable values.
Returned values: Returned values:
Parameters `(a, b)` of the resulting line `y = a*x + b`. Constants `(a, b)` of the resulting line `y = a*x + b`.
**Examples** **Examples**

View File

@ -105,12 +105,14 @@ Configuration example:
Configuration fields: Configuration fields:
- `name` The column name. Tag | Description | Required
- `type` The column type. Sets the method for interpreting data in the source. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the source table, but it can be uploaded as `String`. ----|-------------|---------
- `null_value` The default value for a non-existing element. In the example, it is an empty string. `name`| Column name. | Yes
- `expression` The attribute can be an expression. The tag is not required. `type`| Column type.<br/>Sets the method for interpreting data in the source. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the source table, but it can be uploaded as `String`. | Yes
- `hierarchical` Hierarchical support. Mirrored to the parent identifier. By default, ` false`. `null_value` | Default value for a non-existing element.<br/>In the example, it is an empty string. | Yes
- `injective` Whether the `id -> attribute` image is injective. If ` true`, then you can optimize the ` GROUP BY` clause. By default, `false`. `expression` | [Expression](../syntax.md#syntax-expressions) that ClickHouse executes on the value.<br/>The expression can be a column name in the remote SQL database. Thus, you can use it for creating an alias for the remote column.<br/><br/>Default value: no expression. | No
- `is_object_id` Whether the query is executed for a MongoDB document by `ObjectID`. `hierarchical` | Hierarchical support. Mirrored to the parent identifier.<br/><br/>Default value: `false`. | No
`injective` | Flag that shows whether the `id -> attribute` image is injective.<br/>If `true`, then you can optimize the `GROUP BY` clause.<br/><br/>Default value: `false`. | No
`is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.<br/><br/>Default value: `false`. | No
[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_structure/) <!--hide--> [Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_structure/) <!--hide-->

View File

@ -449,23 +449,25 @@ See the standard [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) descriptio
#### Multiple JOIN #### Multiple JOIN
Performing queries, ClickHouse rewrites multiple joins into the sequence of two-table joins. If there are four tables for join ClickHouse joins the first and the second, then joins the result with the third table, and at the last step, it joins the fourth one. Performing queries, ClickHouse rewrites multi-table joins into the sequence of two-table joins. For example, if there are four tables for join ClickHouse joins the first and the second, then joins the result with the third table, and at the last step, it joins the fourth one.
If a query contains `WHERE` clause, ClickHouse tries to push down filters from this clause into the intermediate join. If it cannot apply the filter to each intermediate join, ClickHouse applies the filters after all joins are completed. If a query contains the `WHERE` clause, ClickHouse tries to pushdown filters from this clause through the intermediate join. If it cannot apply the filter to each intermediate join, ClickHouse applies the filters after all joins are completed.
We recommend the `JOIN ON` or `JOIN USING` syntax for creating a query. For example: We recommend the `JOIN ON` or `JOIN USING` syntax for creating queries. For example:
``` ```
SELECT * FROM t1 JOIN t2 ON t1.a = t2.a JOIN t3 ON t1.a = t3.a SELECT * FROM t1 JOIN t2 ON t1.a = t2.a JOIN t3 ON t1.a = t3.a
``` ```
Also, you can use comma separated list of tables for join. Works only with the [allow_experimental_cross_to_join_conversion = 1](../operations/settings/settings.md#settings-allow_experimental_cross_to_join_conversion) setting. You can use comma-separated lists of tables in the `FROM` clause. This works only with the [allow_experimental_cross_to_join_conversion = 1](../operations/settings/settings.md#settings-allow_experimental_cross_to_join_conversion) setting. For example:
For example, `SELECT * FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a` ```
SELECT * FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a
```
Don't mix these syntaxes. Don't mix these syntaxes.
ClickHouse doesn't support the syntax with commas directly, so we don't recommend to use it. The algorithm tries to rewrite the query in terms of `CROSS` and `INNER` `JOIN` clauses and then proceeds the query processing. When rewriting the query, ClickHouse tries to optimize performance and memory consumption. By default, ClickHouse treats comma as an `INNER JOIN` clause and converts it to `CROSS JOIN` when the algorithm cannot guaranty that `INNER JOIN` returns required data. ClickHouse doesn't directly support syntax with commas, so we don't recommend using them. The algorithm tries to rewrite the query in terms of `CROSS JOIN` and `INNER JOIN` clauses and then proceeds to query processing. When rewriting the query, ClickHouse tries to optimize performance and memory consumption. By default, ClickHouse treats commas as an `INNER JOIN` clause and converts `INNER JOIN` to `CROSS JOIN` when the algorithm cannot guarantee that `INNER JOIN` returns the required data.
#### ANY or ALL Strictness #### ANY or ALL Strictness
@ -558,14 +560,15 @@ If the `JOIN` keys are [Nullable](../data_types/nullable.md) fields, the rows wh
#### Syntax Limitations #### Syntax Limitations
For multiple `JOIN` clauses in the single `SELECT` query: For multiple `JOIN` clauses in a single `SELECT` query:
- Taking all the columns via `*` is available only if tables are joined, not subqueries. - Taking all the columns via `*` is available only if tables are joined, not subqueries.
- The `PREWHERE` clause is not available. - The `PREWHERE` clause is not available.
For `ON`, `WHERE` and `GROUP BY` clauses: For `ON`, `WHERE`, and `GROUP BY` clauses:
- Arbitrary expressions cannot be used in `ON`, `WHERE`, and `GROUP BY` clauses, but you can define an expression in a `SELECT` clause and then use it in these clauses via an alias.
- Arbitrary expressions cannot be used in `ON`, `WHERE` and `GROUP BY` clauses, but you can define an expression in `SELECT` clause and then use it via alias in these clauses.
### WHERE Clause ### WHERE Clause

View File

@ -12,7 +12,7 @@ Parameters:
- `external_database` — Name of a database in an external DBMS. - `external_database` — Name of a database in an external DBMS.
- `external_table` — Name of a table in the `external_database`. - `external_table` — Name of a table in the `external_database`.
To implement ODBC connection safely, ClickHouse uses the separate program `clickhouse-odbc-bridge`. If the ODBC driver is loaded directly from the `clickhouse-server` program, the problems in the driver can crash the ClickHouse server. ClickHouse starts the `clickhouse-odbc-bridge` program automatically when it is required. The ODBC bridge program is installed by the same package as the `clickhouse-server`. To safely implement ODBC connections, ClickHouse uses a separate program `clickhouse-odbc-bridge`. If the ODBC driver is loaded directly from `clickhouse-server`, driver problems can crash the ClickHouse server. ClickHouse automatically starts `clickhouse-odbc-bridge` when it is required. The ODBC bridge program is installed from the same package as the `clickhouse-server`.
The fields with the `NULL` values from the external table are converted into the default values for the base data type. For example, if a remote MySQL table field has the `INT NULL` type it is converted to 0 (the default value for ClickHouse `Int32` data type). The fields with the `NULL` values from the external table are converted into the default values for the base data type. For example, if a remote MySQL table field has the `INT NULL` type it is converted to 0 (the default value for ClickHouse `Int32` data type).
@ -20,11 +20,11 @@ The fields with the `NULL` values from the external table are converted into the
**Getting data from the local MySQL installation via ODBC** **Getting data from the local MySQL installation via ODBC**
This example is for linux Ubuntu 18.04 and MySQL server 5.7. This example is checked for Ubuntu Linux 18.04 and MySQL server 5.7.
Ensure that there are unixODBC and MySQL Connector are installed. Ensure that unixODBC and MySQL Connector are installed.
By default (if installed from packages) ClickHouse starts on behalf of the user `clickhouse`. Thus you need to create and configure this user at MySQL server. By default (if installed from packages), ClickHouse starts as user `clickhouse`. Thus you need to create and configure this user in the MySQL server.
``` ```
sudo mysql sudo mysql
@ -45,7 +45,7 @@ USERNAME = clickhouse
PASSWORD = clickhouse PASSWORD = clickhouse
``` ```
You can check the connection by the `isql` utility from the unixODBC installation. You can check the connection using the `isql` utility from the unixODBC installation.
``` ```
isql -v mysqlconn isql -v mysqlconn
@ -78,7 +78,7 @@ mysql> select * from test;
1 row in set (0,00 sec) 1 row in set (0,00 sec)
``` ```
Getting data from the MySQL table: Retrieving data from the MySQL table in ClickHouse:
```sql ```sql
SELECT * FROM odbc('DSN=mysqlconn', 'test', 'test') SELECT * FROM odbc('DSN=mysqlconn', 'test', 'test')

View File

@ -163,31 +163,54 @@ clickhouse-client --query "SELECT COUNT(*) FROM datasets.ontime"
Q0. Q0.
``` sql ``` sql
select avg(c1) from (select Year, Month, count(*) as c1 from ontime group by Year, Month); SELECT avg(c1)
FROM
(
SELECT Year, Month, count(*) AS c1
FROM ontime
GROUP BY Year, Month
);
``` ```
Q1. Количество полетов в день с 2000 по 2008 года Q1. Количество полетов в день с 2000 по 2008 года
``` sql ``` sql
SELECT DayOfWeek, count(*) AS c FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC; SELECT DayOfWeek, count(*) AS c
FROM ontime
WHERE Year>=2000 AND Year<=2008
GROUP BY DayOfWeek
ORDER BY c DESC;
``` ```
Q2. Количество полетов, задержанных более чем на 10 минут, с группировкой по дням неделе, за 2000-2008 года Q2. Количество полетов, задержанных более чем на 10 минут, с группировкой по дням неделе, за 2000-2008 года
``` sql ``` sql
SELECT DayOfWeek, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC SELECT DayOfWeek, count(*) AS c
FROM ontime
WHERE DepDelay>10 AND Year>=2000 AND Year<=2008
GROUP BY DayOfWeek
ORDER BY c DESC;
``` ```
Q3. Количество задержек по аэропортам за 2000-2008 Q3. Количество задержек по аэропортам за 2000-2008
``` sql ``` sql
SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY Origin ORDER BY c DESC LIMIT 10 SELECT Origin, count(*) AS c
FROM ontime
WHERE DepDelay>10 AND Year>=2000 AND Year<=2008
GROUP BY Origin
ORDER BY c DESC
LIMIT 10;
``` ```
Q4. Количество задержек по перевозчикам за 2007 год Q4. Количество задержек по перевозчикам за 2007 год
``` sql ``` sql
SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC SELECT Carrier, count(*)
FROM ontime
WHERE DepDelay>10 AND Year=2007
GROUP BY Carrier
ORDER BY count(*) DESC;
``` ```
Q5. Процент задержек по перевозчикам за 2007 год Q5. Процент задержек по перевозчикам за 2007 год
@ -219,7 +242,11 @@ ORDER BY c3 DESC;
Более оптимальная версия того же запроса: Более оптимальная версия того же запроса:
``` sql ``` sql
SELECT Carrier, avg(DepDelay > 10) * 100 AS c3 FROM ontime WHERE Year = 2007 GROUP BY Carrier ORDER BY Carrier SELECT Carrier, avg(DepDelay>10)*100 AS c3
FROM ontime
WHERE Year=2007
GROUP BY Carrier
ORDER BY Carrier
``` ```
Q6. Предыдущий запрос за более широкий диапазон лет, 2000-2008 Q6. Предыдущий запрос за более широкий диапазон лет, 2000-2008
@ -251,7 +278,11 @@ ORDER BY c3 DESC;
Более оптимальная версия того же запроса: Более оптимальная версия того же запроса:
``` sql ``` sql
SELECT Carrier, avg(DepDelay > 10) * 100 AS c3 FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY Carrier ORDER BY Carrier SELECT Carrier, avg(DepDelay>10)*100 AS c3
FROM ontime
WHERE Year>=2000 AND Year<=2008
GROUP BY Carrier
ORDER BY Carrier;
``` ```
Q7. Процент полетов, задержанных на более 10 минут, в разбивке по годам Q7. Процент полетов, задержанных на более 10 минут, в разбивке по годам
@ -275,39 +306,49 @@ ANY INNER JOIN
from ontime from ontime
GROUP BY Year GROUP BY Year
) USING (Year) ) USING (Year)
ORDER BY Year ORDER BY Year;
``` ```
Более оптимальная версия того же запроса: Более оптимальная версия того же запроса:
``` sql ``` sql
SELECT Year, avg(DepDelay > 10) FROM ontime GROUP BY Year ORDER BY Year SELECT Year, avg(DepDelay>10)
FROM ontime
GROUP BY Year
ORDER BY Year;
``` ```
Q8. Самые популярные направления по количеству напрямую соединенных городов для различных диапазонов лет Q8. Самые популярные направления по количеству напрямую соединенных городов для различных диапазонов лет
``` sql ``` sql
SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime WHERE Year >= 2000 and Year <= 2010 GROUP BY DestCityName ORDER BY u DESC LIMIT 10; SELECT DestCityName, uniqExact(OriginCityName) AS u F
ROM ontime
WHERE Year>=2000 and Year<=2010
GROUP BY DestCityName
ORDER BY u DESC
LIMIT 10;
``` ```
Q9. Q9.
``` sql ``` sql
select Year, count(*) as c1 from ontime group by Year; SELECT Year, count(*) AS c1
FROM ontime
GROUP BY Year;
``` ```
Q10. Q10.
``` sql ``` sql
select SELECT
min(Year), max(Year), Carrier, count(*) as cnt, min(Year), max(Year), Carrier, count(*) AS cnt,
sum(ArrDelayMinutes>30) as flights_delayed, sum(ArrDelayMinutes>30) AS flights_delayed,
round(sum(ArrDelayMinutes>30)/count(*),2) as rate round(sum(ArrDelayMinutes>30)/count(*),2) AS rate
FROM ontime FROM ontime
WHERE WHERE
DayOfWeek not in (6,7) and OriginState not in ('AK', 'HI', 'PR', 'VI') DayOfWeek NOT IN (6,7) AND OriginState NOT IN ('AK', 'HI', 'PR', 'VI')
and DestState not in ('AK', 'HI', 'PR', 'VI') AND DestState NOT IN ('AK', 'HI', 'PR', 'VI')
and FlightDate < '2010-01-01' AND FlightDate < '2010-01-01'
GROUP by Carrier GROUP by Carrier
HAVING cnt>100000 and max(Year)>1990 HAVING cnt>100000 and max(Year)>1990
ORDER by rate DESC ORDER by rate DESC
@ -317,15 +358,39 @@ LIMIT 1000;
Бонус: Бонус:
``` sql ``` sql
SELECT avg(cnt) FROM (SELECT Year,Month,count(*) AS cnt FROM ontime WHERE DepDel15=1 GROUP BY Year,Month) SELECT avg(cnt)
FROM
(
SELECT Year,Month,count(*) AS cnt
FROM ontime
WHERE DepDel15=1
GROUP BY Year,Month
);
select avg(c1) from (select Year,Month,count(*) as c1 from ontime group by Year,Month) SELECT avg(c1) FROM
(
SELECT Year,Month,count(*) AS c1
FROM ontime
GROUP BY Year,Month
);
SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime GROUP BY DestCityName ORDER BY u DESC LIMIT 10; SELECT DestCityName, uniqExact(OriginCityName) AS u
FROM ontime
GROUP BY DestCityName
ORDER BY u DESC
LIMIT 10;
SELECT OriginCityName, DestCityName, count() AS c FROM ontime GROUP BY OriginCityName, DestCityName ORDER BY c DESC LIMIT 10; SELECT OriginCityName, DestCityName, count() AS c
FROM ontime
GROUP BY OriginCityName, DestCityName
ORDER BY c DESC
LIMIT 10;
SELECT OriginCityName, count() AS c FROM ontime GROUP BY OriginCityName ORDER BY c DESC LIMIT 10; SELECT OriginCityName, count() AS c
FROM ontime
GROUP BY OriginCityName
ORDER BY c DESC
LIMIT 10;
``` ```
Данный тест производительности был создан Вадимом Ткаченко, статьи по теме: Данный тест производительности был создан Вадимом Ткаченко, статьи по теме:

View File

@ -667,4 +667,22 @@ load_balancing = first_or_random
- [insert_quorum](#settings-insert_quorum) - [insert_quorum](#settings-insert_quorum)
- [insert_quorum_timeout](#settings-insert_quorum_timeout) - [insert_quorum_timeout](#settings-insert_quorum_timeout)
## allow_experimental_cross_to_join_conversion {#settings-allow_experimental_cross_to_join_conversion}
Включает или отключает:
1. Перезапись запросов из синтаксиса с запятыми в синтаксис `JOIN ON/USING`. Если значение параметра равно 0, ClickHouse не обрабатывает запросы с синтаксисом, использующим запятые, и генерирует исключение.
2. Преобразование `CROSS JOIN` в `INNER JOIN`, если условия объединения таблиц это позволяют.
Возможные значения:
- 0 — выключена.
- 1 — включена.
Значение по умолчанию: 1.
**Смотрите также**
- [Множественный JOIN](../../query_language/select.md#select-join)
[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/settings/settings/) <!--hide--> [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/settings/settings/) <!--hide-->

View File

@ -1,29 +1,122 @@
# ODBC # ODBC {#table_engine-odbc}
Allows ClickHouse to connect to external databases via [ODBC](https://en.wikipedia.org/wiki/Open_Database_Connectivity). Позволяет ClickHouse подключаться к внешним базам данных с помощью [ODBC](https://en.wikipedia.org/wiki/Open_Database_Connectivity).
To implement ODBC connection, ClickHouse uses the separate program `clickhouse-odbc-bridge`. ClickHouse starts this program automatically when it is required. The ODBC bridge program is installed by the same package with the ClickHouse server. Чтобы использование ODBC было безопасным, ClickHouse использует отдельную программу `clickhouse-odbc-bridge`. Если драйвер ODBC подгружать непосредственно из `clickhouse-server`, то проблемы с драйвером могут привести к аварийной остановке сервера ClickHouse. ClickHouse автоматически запускает `clickhouse-odbc-bridge` по мере необходимости. Программа устанавливается из того же пакета, что и `clickhouse-server`.
This engine supports the [Nullable](../../data_types/nullable.md) data type. Движок поддерживает тип данных [Nullable](../../data_types/nullable.md).
## Creating a Table ## Создание таблицы
``` ```
CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = ODBC(connection_settings, external_database, external_table) CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
(
name1 [type1],
name2 [type2],
...
)
ENGINE = ODBC(connection_settings, external_database, external_table)
``` ```
**Engine Parameters** Смотрите подробное описание запроса [CREATE TABLE](../../query_language/create.md#create-table-query).
- `connection_settings` — Name of the section with connection settings in the `odbc.ini` file. Структура таблицы может отличаться от структуры исходной таблицы в удалённой СУБД:
- `external_database` — Database in an external DBMS.
- `external_table` — A name of the table in `external_database`.
## Usage Example - Имена столбцов должны быть такими же, как в исходной таблице, но вы можете использовать только некоторые из этих столбцов и в любом порядке.
- Типы столбцов могут отличаться от типов аналогичных столбцов в исходной таблице. ClickHouse пытается [приводить](../../query_language/functions/type_conversion_functions.md#type_conversion_function-cast) значения к типам данных ClickHouse.
Some examples of how to use external dictionaries through ODBC you can find in the [ODBC](../../query_language/dicts/external_dicts_dict_sources.md#dicts-external_dicts_dict_sources-odbc) section of external dictionaries configuration. **Параметры движка**
## See Also - `connection_settings` — название секции с настройками соединения в файле `odbc.ini`.
- `external_database` — имя базы данных во внешней СУБД.
- `external_table` — имя таблицы в `external_database`.
- [ODBC table function](../../query_language/table_functions/odbc.md). ## Пример использования
[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/jdbc/) <!--hide--> **Извлечение данных из локальной установки MySQL через ODBC**
Этот пример проверялся в Ubuntu Linux 18.04 для MySQL server 5.7.
Убедитесь, что unixODBC и MySQL Connector установлены.
По умолчанию (если установлен из пакетов) ClickHouse запускается от имени пользователя `clickhouse`. Таким образом, вам нужно создать и настроить этого пользователя на сервере MySQL.
```
sudo mysql
mysql> CREATE USER 'clickhouse'@'localhost' IDENTIFIED BY 'clickhouse';
mysql> GRANT ALL PRIVILEGES ON *.* TO 'clickhouse'@'clickhouse' WITH GRANT OPTION;
```
Теперь настроим соединение в `/etc/odbc.ini`.
```
$ cat /etc/odbc.ini
[mysqlconn]
DRIVER = /usr/local/lib/libmyodbc5w.so
SERVER = 127.0.0.1
PORT = 3306
DATABASE = test
USERNAME = clickhouse
PASSWORD = clickhouse
```
Вы можете проверить соединение с помощью утилиты `isql` из установки unixODBC.
```
isql -v mysqlconn
+---------------------------------------+
| Connected! |
| |
...
```
Таблица в MySQL:
```
mysql> CREATE TABLE `test`.`test` (
-> `int_id` INT NOT NULL AUTO_INCREMENT,
-> `int_nullable` INT NULL DEFAULT NULL,
-> `float` FLOAT NOT NULL,
-> `float_nullable` FLOAT NULL DEFAULT NULL,
-> PRIMARY KEY (`int_id`));
Query OK, 0 rows affected (0,09 sec)
mysql> insert into test (`int_id`, `float`) VALUES (1,2);
Query OK, 1 row affected (0,00 sec)
mysql> select * from test;
+--------+--------------+-------+----------------+
| int_id | int_nullable | float | float_nullable |
+--------+--------------+-------+----------------+
| 1 | NULL | 2 | NULL |
+--------+--------------+-------+----------------+
1 row in set (0,00 sec)
```
Таблица в ClickHouse, которая получает данные из таблицы MySQL:
```sql
CREATE TABLE odbc_t
(
`int_id` Int32,
`float_nullable` Nullable(Float32)
)
ENGINE = ODBC('DSN=mysqlconn', 'test', 'test')
```
```sql
SELECT * FROM odbc_t
```
```text
┌─int_id─┬─float_nullable─┐
│ 1 │ ᴺᵁᴸᴸ │
└────────┴────────────────┘
```
## Смотрите также
- [Внешние словари ODBC](../../query_language/dicts/external_dicts_dict_sources.md#dicts-external_dicts_dict_sources-odbc)
- [Табличная функция odbc](../../query_language/table_functions/odbc.md)
[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/odbc/) <!--hide-->

View File

@ -597,12 +597,12 @@ simpleLinearRegression(x, y)
Параметры: Параметры:
- `x`Столбец со значениями зависимой переменной. - `x`столбец со значениями зависимой переменной.
- `y` — столбец со значениями наблюдаемой переменной. - `y` — столбец со значениями наблюдаемой переменной.
Возвращаемые значения: Возвращаемые значения:
Параметры `(a, b)` результирующей прямой `x = a*y + b`. Константы `(a, b)` результирующей прямой `y = a*x + b`.
**Примеры** **Примеры**

View File

@ -431,7 +431,7 @@ ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num;
Соединяет данные в привычном для [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) смысле. Соединяет данные в привычном для [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) смысле.
!!! info "Примечание" !!! info "Примечание"
Не связана с функциональностью с [ARRAY JOIN](#select-array-join-clause). Не связана с функциональностью [ARRAY JOIN](#select-array-join-clause).
```sql ```sql
SELECT <expr_list> SELECT <expr_list>
@ -452,7 +452,29 @@ FROM <left_subquery>
Смотрите описание стандартного [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)). Смотрите описание стандартного [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)).
**Строгость ANY или ALL** #### Множественный JOIN
При выполнении запросов, ClickHouse перезаписывает множественный `JOIN` как комбинацию двух-табличных объединений и обрабатывает их последовательно. Например, если необходимо объединить четыре таблицы, ClickHouse объединяет первую и вторую таблицы, затем соединяет результат с третьей, а затем с четвертой.
Если запрос содержит секцию `WHERE`, ClickHouse пытается пробросить фильтры из этой секции в промежуточный `JOIN`. Если он не может пробросить фильтр в каждый промежуточный `JOIN`, ClickHouse применяет фильтры после того, как все `JOIN` будут выполнены.
Для создания запросов мы рекомендуем использоват синтаксис `JOIN ON` или `JOIN USING`. Например:
```
SELECT * FROM t1 JOIN t2 ON t1.a = t2.a JOIN t3 ON t1.a = t3.a
```
В секции `FROM` вы можете использовать разделенные запятыми списки таблиц для объединения. Этот синтаксис работает только при включённой настройке [allow_experimental_cross_to_join_conversion = 1](../operations/settings/settings.md#settings-allow_experimental_cross_to_join_conversion). Например:
```
SELECT * FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a
```
Не смешивайте синтаксисы.
ClickHouse не поддерживает синтаксис с запятыми напрямую и мы не рекомендуем его использовать. Алгоритм пытается переписать запрос с помощью секций `CROSS JOIN` и `INNER JOIN` и затем продолжает его выполнение. При переписывании запроса, ClickHouse пытается оптимизировать производительность и потребление памяти. По умолчанию, ClickHouse трактует запятые как `INNER JOIN` и конвертирует их в `CROSS JOIN` когда не может гарантировать, что `INNER JOIN` возвращает запрошенные данные.
#### ANY или ALL - строгость:
Если указано `ALL`, то при наличии в "правой" таблице нескольких соответствующих строк, данные будут размножены по количеству этих строк. Это нормальное поведение `JOIN` как в стандартном SQL. Если указано `ALL`, то при наличии в "правой" таблице нескольких соответствующих строк, данные будут размножены по количеству этих строк. Это нормальное поведение `JOIN` как в стандартном SQL.
Если указано `ANY`, то при наличии в "правой" таблице нескольких соответствующих строк, будет присоединена только первая попавшаяся. Если известно, что в "правой" таблице есть не более одной подходящей строки, то результаты `ANY` и `ALL` совпадают. Если указано `ANY`, то при наличии в "правой" таблице нескольких соответствующих строк, будет присоединена только первая попавшаяся. Если известно, что в "правой" таблице есть не более одной подходящей строки, то результаты `ANY` и `ALL` совпадают.
@ -534,6 +556,17 @@ LIMIT 10
Если ключами `JOIN` выступают поля типа [Nullable](../data_types/nullable.md), то строки, где хотя бы один из ключей имеет значение [NULL](syntax.md#null-literal), не соединяются. Если ключами `JOIN` выступают поля типа [Nullable](../data_types/nullable.md), то строки, где хотя бы один из ключей имеет значение [NULL](syntax.md#null-literal), не соединяются.
#### Ограничения синтаксиса
Для множественных секций `JOIN` в одном запросе `SELECT`:
- Получение всех столбцов через `*` возможно только при объединении таблиц, но не подзапросов.
- Секция `PREWHERE` недоступна.
Для секций `ON`, `WHERE` и `GROUP BY`:
- Нельзя использовать произвольные выражения в секциях `ON`, `WHERE`, и `GROUP BY`, однако можно определить выражение в секции `SELECT` и затем использовать его через алиас в других секциях.
### Секция WHERE {#select-where} ### Секция WHERE {#select-where}
Позволяет задать выражение, которое ClickHouse использует для фильтрации данных перед всеми другими действиями в запросе кроме выражений, содержащихся в секции [PREWHERE](#select-prewhere). Обычно, это выражение с логическими операторами. Позволяет задать выражение, которое ClickHouse использует для фильтрации данных перед всеми другими действиями в запросе кроме выражений, содержащихся в секции [PREWHERE](#select-prewhere). Обычно, это выражение с логическими операторами.

View File

@ -1,27 +1,98 @@
# odbc # odbc {#table_functions-odbc}
Returns table that is connected via ODBC driver. Возвращает таблицу, подключенную через [ODBC](https://en.wikipedia.org/wiki/Open_Database_Connectivity).
``` ```
odbc(connection_settings, external_database, external_table) odbc(connection_settings, external_database, external_table)
``` ```
**Function Parameters** Параметры:
- `connection_settings`Name of the section with connection settings in the `odbc.ini` file. - `connection_settings`название секции с настройками соединения в файле `odbc.ini`.
- `external_database`Database in an external DBMS. - `external_database`имя базы данных во внешней СУБД.
- `external_table`A name of the table in `external_database`. - `external_table`имя таблицы в `external_database`.
To implement ODBC connection, ClickHouse uses the separate program `clickhouse-odbc-bridge`. ClickHouse starts this program automatically when it is required. The ODBC bridge program is installed by the same package with the ClickHouse server. Чтобы использование ODBC было безопасным, ClickHouse использует отдельную программу `clickhouse-odbc-bridge`. Если драйвер ODBC подгружать непосредственно из `clickhouse-server`, то проблемы с драйвером могут привести к аварийной остановке сервера ClickHouse. ClickHouse автоматически запускает `clickhouse-odbc-bridge` по мере необходимости. Программа устанавливается из того же пакета, что и `clickhouse-server`.
This function supports the [Nullable](../../data_types/nullable.md) data type (based on DDL of remote table that is queried). Поля из внешней таблицы со значениями `NULL` получают значение по умолчанию для базового типа данных. Например, если поле в удалённой таблице MySQL имеет тип `INT NULL` оно сконвертируется в 0 (значение по умолчанию для типа данных ClickHouse `Int32`).
**Examples** ## Пример использования
**Получение данных из локальной установки MySQL через ODBC**
Этот пример проверялся в Ubuntu Linux 18.04 для MySQL server 5.7.
Убедитесь, что unixODBC и MySQL Connector установлены.
По умолчанию (если установлен из пакетов) ClickHouse запускается от имени пользователя `clickhouse`. Таким образом, вам нужно создать и настроить этого пользователя на сервере MySQL.
``` ```
select * from odbc('DSN=connection_settings_name', 'external_database_name', 'external_table_name') sudo mysql
mysql> CREATE USER 'clickhouse'@'localhost' IDENTIFIED BY 'clickhouse';
mysql> GRANT ALL PRIVILEGES ON *.* TO 'clickhouse'@'clickhouse' WITH GRANT OPTION;
``` ```
Some examples of how to use external dictionaries through ODBC you can find in the [ODBC](../../query_language/dicts/external_dicts_dict_sources.md#dicts-external_dicts_dict_sources-odbc) section of external dictionaries configuration. Теперь настроим соединение в `/etc/odbc.ini`.
[Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/jdbc/) <!--hide--> ```
$ cat /etc/odbc.ini
[mysqlconn]
DRIVER = /usr/local/lib/libmyodbc5w.so
SERVER = 127.0.0.1
PORT = 3306
DATABASE = test
USERNAME = clickhouse
PASSWORD = clickhouse
```
Вы можете проверить соединение с помощью утилиты `isql` из установки unixODBC.
```
isql -v mysqlconn
+---------------------------------------+
| Connected! |
| |
...
```
Таблица в MySQL:
```
mysql> CREATE TABLE `test`.`test` (
-> `int_id` INT NOT NULL AUTO_INCREMENT,
-> `int_nullable` INT NULL DEFAULT NULL,
-> `float` FLOAT NOT NULL,
-> `float_nullable` FLOAT NULL DEFAULT NULL,
-> PRIMARY KEY (`int_id`));
Query OK, 0 rows affected (0,09 sec)
mysql> insert into test (`int_id`, `float`) VALUES (1,2);
Query OK, 1 row affected (0,00 sec)
mysql> select * from test;
+--------+--------------+-------+----------------+
| int_id | int_nullable | float | float_nullable |
+--------+--------------+-------+----------------+
| 1 | NULL | 2 | NULL |
+--------+--------------+-------+----------------+
1 row in set (0,00 sec)
```
Получение данных из таблицы MySQL в ClickHouse:
```sql
SELECT * FROM odbc('DSN=mysqlconn', 'test', 'test')
```
```text
┌─int_id─┬─int_nullable─┬─float─┬─float_nullable─┐
│ 1 │ 0 │ 2 │ 0 │
└────────┴──────────────┴───────┴────────────────┘
```
## Смотрите также
- [Внешние словари ODBC](../../query_language/dicts/external_dicts_dict_sources.md#dicts-external_dicts_dict_sources-odbc)
- [Движок таблиц ODBC](../../operations/table_engines/odbc.md).
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/jdbc/) <!--hide-->

View File

@ -94,7 +94,7 @@
</div> </div>
<div id="announcement" class="colored-block"> <div id="announcement" class="colored-block">
<div class="page"> <div class="page">
Upcoming ClickHouse Meetup: <a class="announcement-link" href="https://www.huodongxing.com/event/3483759917300" rel="external nofollow" target="_blank">Shenzhen</a> on October 20 Upcoming Meetups: <a class="announcement-link" href="https://events.yandex.ru/events/ClickHouse/26-June-2019/" rel="external nofollow" target="_blank">Novosibirsk</a> on June 26 and <a class="announcement-link" href="https://www.huodongxing.com/event/3483759917300" rel="external nofollow" target="_blank">Shenzhen</a> on October 20
</div> </div>
</div> </div>
<div class="page"> <div class="page">