mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge branch 'master' into yandex-to-clickhouse-in-configs
This commit is contained in:
commit
fb7ce28198
@ -34,7 +34,6 @@
|
||||
* New functions `currentProfiles()`, `enabledProfiles()`, `defaultProfiles()`. [#26714](https://github.com/ClickHouse/ClickHouse/pull/26714) ([Vitaly Baranov](https://github.com/vitlibar)).
|
||||
* Add functions that return (initial_)query_id of the current query. This closes [#23682](https://github.com/ClickHouse/ClickHouse/issues/23682). [#26410](https://github.com/ClickHouse/ClickHouse/pull/26410) ([Alexey Boykov](https://github.com/mathalex)).
|
||||
* Add `REPLACE GRANT` feature. [#26384](https://github.com/ClickHouse/ClickHouse/pull/26384) ([Caspian](https://github.com/Cas-pian)).
|
||||
* Implement window function `nth_value(expr, N)` that returns the value of the Nth row of the window frame. [#26334](https://github.com/ClickHouse/ClickHouse/pull/26334) ([Zuo, RuoYu](https://github.com/ryzuo)).
|
||||
* `EXPLAIN` query now has `EXPLAIN ESTIMATE ...` mode that will show information about read rows, marks and parts from MergeTree tables. Closes [#23941](https://github.com/ClickHouse/ClickHouse/issues/23941). [#26131](https://github.com/ClickHouse/ClickHouse/pull/26131) ([fastio](https://github.com/fastio)).
|
||||
* Added `system.zookeeper_log` table. All actions of ZooKeeper client are logged into this table. Implements [#25449](https://github.com/ClickHouse/ClickHouse/issues/25449). [#26129](https://github.com/ClickHouse/ClickHouse/pull/26129) ([tavplubix](https://github.com/tavplubix)).
|
||||
* Zero-copy replication for `ReplicatedMergeTree` over `HDFS` storage. [#25918](https://github.com/ClickHouse/ClickHouse/pull/25918) ([Zhichang Yu](https://github.com/yuzhichang)).
|
||||
|
@ -18,9 +18,12 @@
|
||||
<!-- One NUMA node w/o hyperthreading -->
|
||||
<max_threads>12</max_threads>
|
||||
|
||||
<!-- disable jit for perf tests -->
|
||||
<!-- disable JIT for perf tests -->
|
||||
<compile_expressions>0</compile_expressions>
|
||||
<compile_aggregate_expressions>0</compile_aggregate_expressions>
|
||||
|
||||
<!-- Don't fail some prewarm queries too early -->
|
||||
<timeout_before_checking_execution_speed>60</timeout_before_checking_execution_speed>
|
||||
</default>
|
||||
</profiles>
|
||||
<users>
|
||||
|
@ -20,7 +20,21 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
|
||||
bool /* created_from_ddl */)
|
||||
{
|
||||
const auto config_prefix = root_config_prefix + ".mongodb";
|
||||
auto configuration = getExternalDataSourceConfiguration(config, config_prefix, context);
|
||||
ExternalDataSourceConfiguration configuration;
|
||||
auto named_collection = getExternalDataSourceConfiguration(config, config_prefix, context);
|
||||
if (named_collection)
|
||||
{
|
||||
configuration = *named_collection;
|
||||
}
|
||||
else
|
||||
{
|
||||
configuration.host = config.getString(config_prefix + ".host", "");
|
||||
configuration.port = config.getUInt(config_prefix + ".port", 0);
|
||||
configuration.username = config.getString(config_prefix + ".user", "");
|
||||
configuration.password = config.getString(config_prefix + ".password", "");
|
||||
configuration.database = config.getString(config_prefix + ".db", "");
|
||||
}
|
||||
|
||||
return std::make_unique<MongoDBDictionarySource>(dict_struct,
|
||||
config.getString(config_prefix + ".uri", ""),
|
||||
configuration.host,
|
||||
|
@ -44,7 +44,22 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
|
||||
config.getBool(config_prefix + ".mysql.fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss);
|
||||
|
||||
auto settings_config_prefix = config_prefix + ".mysql";
|
||||
auto configuration = getExternalDataSourceConfiguration(config, settings_config_prefix, global_context);
|
||||
std::shared_ptr<mysqlxx::PoolWithFailover> pool;
|
||||
ExternalDataSourceConfiguration configuration;
|
||||
auto named_collection = created_from_ddl ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context) : std::nullopt;
|
||||
if (named_collection)
|
||||
{
|
||||
configuration = *named_collection;
|
||||
std::vector<std::pair<String, UInt16>> addresses{std::make_pair(configuration.host, configuration.port)};
|
||||
pool = std::make_shared<mysqlxx::PoolWithFailover>(configuration.database, addresses, configuration.username, configuration.password);
|
||||
}
|
||||
else
|
||||
{
|
||||
configuration.database = config.getString(settings_config_prefix + ".db", "");
|
||||
configuration.table = config.getString(settings_config_prefix + ".table", "");
|
||||
pool = std::make_shared<mysqlxx::PoolWithFailover>(mysqlxx::PoolFactory::instance().get(config, settings_config_prefix));
|
||||
}
|
||||
|
||||
auto query = config.getString(settings_config_prefix + ".query", "");
|
||||
if (query.empty() && configuration.table.empty())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL dictionary source configuration must contain table or query field");
|
||||
@ -61,15 +76,6 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
|
||||
.dont_check_update_time = config.getBool(settings_config_prefix + ".dont_check_update_time", false)
|
||||
};
|
||||
|
||||
std::shared_ptr<mysqlxx::PoolWithFailover> pool;
|
||||
if (created_from_ddl)
|
||||
{
|
||||
std::vector<std::pair<String, UInt16>> addresses{std::make_pair(configuration.host, configuration.port)};
|
||||
pool = std::make_shared<mysqlxx::PoolWithFailover>(configuration.database, addresses, configuration.username, configuration.password);
|
||||
}
|
||||
else
|
||||
pool = std::make_shared<mysqlxx::PoolWithFailover>(mysqlxx::PoolFactory::instance().get(config, settings_config_prefix));
|
||||
|
||||
return std::make_unique<MySQLDictionarySource>(dict_struct, dictionary_configuration, std::move(pool), sample_block, mysql_input_stream_settings);
|
||||
#else
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
||||
|
@ -124,7 +124,7 @@ std::optional<ExternalDataSourceConfig> getExternalDataSourceConfiguration(const
|
||||
}
|
||||
|
||||
|
||||
ExternalDataSourceConfiguration getExternalDataSourceConfiguration(
|
||||
std::optional<ExternalDataSourceConfiguration> getExternalDataSourceConfiguration(
|
||||
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context)
|
||||
{
|
||||
ExternalDataSourceConfiguration configuration;
|
||||
@ -152,25 +152,33 @@ ExternalDataSourceConfiguration getExternalDataSourceConfiguration(
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Named collection of connection parameters is missing some of the parameters and dictionary parameters are added");
|
||||
}
|
||||
return configuration;
|
||||
}
|
||||
else
|
||||
{
|
||||
configuration.host = dict_config.getString(dict_config_prefix + ".host", "");
|
||||
configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0);
|
||||
configuration.username = dict_config.getString(dict_config_prefix + ".user", "");
|
||||
configuration.password = dict_config.getString(dict_config_prefix + ".password", "");
|
||||
configuration.database = dict_config.getString(dict_config_prefix + ".db", "");
|
||||
configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), "");
|
||||
configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), "");
|
||||
}
|
||||
return configuration;
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
|
||||
ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority(
|
||||
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context)
|
||||
{
|
||||
auto common_configuration = getExternalDataSourceConfiguration(dict_config, dict_config_prefix, context);
|
||||
ExternalDataSourceConfiguration common_configuration;
|
||||
|
||||
auto named_collection = getExternalDataSourceConfiguration(dict_config, dict_config_prefix, context);
|
||||
if (named_collection)
|
||||
{
|
||||
common_configuration = *named_collection;
|
||||
}
|
||||
else
|
||||
{
|
||||
common_configuration.host = dict_config.getString(dict_config_prefix + ".host", "");
|
||||
common_configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0);
|
||||
common_configuration.username = dict_config.getString(dict_config_prefix + ".user", "");
|
||||
common_configuration.password = dict_config.getString(dict_config_prefix + ".password", "");
|
||||
common_configuration.database = dict_config.getString(dict_config_prefix + ".db", "");
|
||||
common_configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), "");
|
||||
common_configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), "");
|
||||
}
|
||||
|
||||
ExternalDataSourcesByPriority configuration
|
||||
{
|
||||
.database = common_configuration.database,
|
||||
|
@ -25,8 +25,6 @@ struct ExternalDataSourceConfiguration
|
||||
void set(const ExternalDataSourceConfiguration & conf);
|
||||
};
|
||||
|
||||
using ExternalDataSourceConfigurationPtr = std::shared_ptr<ExternalDataSourceConfiguration>;
|
||||
|
||||
|
||||
struct StoragePostgreSQLConfiguration : ExternalDataSourceConfiguration
|
||||
{
|
||||
@ -67,7 +65,7 @@ struct ExternalDataSourceConfig
|
||||
*/
|
||||
std::optional<ExternalDataSourceConfig> getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine = false);
|
||||
|
||||
ExternalDataSourceConfiguration getExternalDataSourceConfiguration(
|
||||
std::optional<ExternalDataSourceConfiguration> getExternalDataSourceConfiguration(
|
||||
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context);
|
||||
|
||||
|
||||
|
@ -1,3 +0,0 @@
|
||||
<clickhouse>
|
||||
<insert_format_max_block_size>100000</insert_format_max_block_size>
|
||||
</clickhouse>
|
@ -1,17 +0,0 @@
|
||||
<!-- Config for connecting to test server in Arcadia -->
|
||||
<clickhouse>
|
||||
<tcp_port>59000</tcp_port>
|
||||
<tcp_port_secure>59440</tcp_port_secure>
|
||||
<openSSL>
|
||||
<client>
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
<verificationMode>none</verificationMode>
|
||||
<invalidCertificateHandler>
|
||||
<name>AcceptCertificateHandler</name>
|
||||
</invalidCertificateHandler>
|
||||
</client>
|
||||
</openSSL>
|
||||
</clickhouse>
|
@ -1 +0,0 @@
|
||||
config/decimals_dictionary.xml
|
@ -1 +0,0 @@
|
||||
config/executable_pool_dictionary.xml
|
@ -58,28 +58,8 @@ def test_valid_column_family_options(start_cluster):
|
||||
DROP TABLE test;
|
||||
""")
|
||||
|
||||
def test_invalid_column_family_options():
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/num_levels/no_such_column_family_option/g' /etc/clickhouse-server/config.d/rocksdb.xml"])
|
||||
node.restart_clickhouse()
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
""")
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/no_such_column_family_option/num_levels/g' /etc/clickhouse-server/config.d/rocksdb.xml"])
|
||||
node.restart_clickhouse()
|
||||
|
||||
def test_table_valid_column_family_options():
|
||||
def test_table_valid_column_family_options(start_cluster):
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
DROP TABLE test;
|
||||
""")
|
||||
|
||||
def test_table_invalid_column_family_options():
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/max_bytes_for_level_base/no_such_table_column_family_option/g' /etc/clickhouse-server/config.d/rocksdb.xml"])
|
||||
node.restart_clickhouse()
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node.query("""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
""")
|
||||
node.exec_in_container(['bash', '-c', "sed -i 's/no_such_table_column_family_option/max_bytes_for_level_base/g' /etc/clickhouse-server/config.d/rocksdb.xml"])
|
||||
node.restart_clickhouse()
|
||||
|
@ -1 +0,0 @@
|
||||
config/ints_dictionary.xml
|
@ -1,77 +0,0 @@
|
||||
import os
|
||||
import re
|
||||
import sys
|
||||
|
||||
import pytest
|
||||
|
||||
from .server import ServerThread
|
||||
|
||||
|
||||
# Command-line arguments
|
||||
|
||||
def pytest_addoption(parser):
|
||||
parser.addoption("--builddir", action="store", default=None, help="Path to build directory to use binaries from")
|
||||
|
||||
|
||||
# HTML report hooks
|
||||
|
||||
def pytest_html_report_title(report):
|
||||
report.title = "ClickHouse Functional Stateless Tests (PyTest)"
|
||||
|
||||
|
||||
RE_TEST_NAME = re.compile(r"\[(.*)\]")
|
||||
def pytest_itemcollected(item):
|
||||
match = RE_TEST_NAME.search(item.name)
|
||||
if match:
|
||||
item._nodeid = match.group(1)
|
||||
|
||||
|
||||
# Fixtures
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
def cmdopts(request):
|
||||
return {
|
||||
'builddir': request.config.getoption("--builddir")
|
||||
}
|
||||
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
def bin_prefix(cmdopts):
|
||||
prefix = 'clickhouse'
|
||||
if cmdopts['builddir'] is not None:
|
||||
prefix = os.path.join(cmdopts['builddir'], 'programs', prefix)
|
||||
# FIXME: does this hangs the server start for some reason?
|
||||
# if not os.path.isabs(prefix):
|
||||
# prefix = os.path.abspath(prefix)
|
||||
return prefix
|
||||
|
||||
|
||||
# TODO: also support stateful queries.
|
||||
QUERIES_PATH = os.path.join(os.path.dirname(os.path.abspath(__file__)), '0_stateless')
|
||||
|
||||
@pytest.fixture(scope='module', params=[f for f in os.listdir(QUERIES_PATH) if f.endswith('.sql')])
|
||||
def sql_query(request):
|
||||
return os.path.join(QUERIES_PATH, os.path.splitext(request.param)[0])
|
||||
|
||||
|
||||
@pytest.fixture(scope='module', params=[f for f in os.listdir(QUERIES_PATH) if f.endswith('.sh')])
|
||||
def shell_query(request):
|
||||
return os.path.join(QUERIES_PATH, os.path.splitext(request.param)[0])
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def standalone_server(bin_prefix, tmp_path):
|
||||
server = ServerThread(bin_prefix, str(tmp_path))
|
||||
server.start()
|
||||
wait_result = server.wait()
|
||||
|
||||
if wait_result is not None:
|
||||
with open(os.path.join(server.log_dir, 'server', 'stdout.txt'), 'r') as f:
|
||||
print(f.read(), file=sys.stderr)
|
||||
with open(os.path.join(server.log_dir, 'server', 'stderr.txt'), 'r') as f:
|
||||
print(f.read(), file=sys.stderr)
|
||||
pytest.fail('Server died unexpectedly with code {code}'.format(code=server._proc.returncode), pytrace=False)
|
||||
|
||||
yield server
|
||||
|
||||
server.stop()
|
@ -1,2 +0,0 @@
|
||||
[pytest]
|
||||
render_collapsed = True
|
@ -1,260 +0,0 @@
|
||||
import difflib
|
||||
import os
|
||||
import random
|
||||
import string
|
||||
import subprocess
|
||||
import sys
|
||||
|
||||
import pytest
|
||||
|
||||
|
||||
SKIP_LIST = [
|
||||
# these couple of tests hangs everything
|
||||
"00600_replace_running_query",
|
||||
"00987_distributed_stack_overflow",
|
||||
"01954_clickhouse_benchmark_multiple_long",
|
||||
|
||||
# just fail
|
||||
"00133_long_shard_memory_tracker_and_exception_safety",
|
||||
"00463_long_sessions_in_http_interface",
|
||||
"00505_secure",
|
||||
"00505_shard_secure",
|
||||
"00646_url_engine",
|
||||
"00725_memory_tracking", # BROKEN
|
||||
"00738_lock_for_inner_table",
|
||||
"00821_distributed_storage_with_join_on",
|
||||
"00825_protobuf_format_array_3dim",
|
||||
"00825_protobuf_format_array_of_arrays",
|
||||
"00825_protobuf_format_enum_mapping",
|
||||
"00825_protobuf_format_nested_in_nested",
|
||||
"00825_protobuf_format_nested_optional",
|
||||
"00825_protobuf_format_no_length_delimiter",
|
||||
"00825_protobuf_format_persons",
|
||||
"00825_protobuf_format_squares",
|
||||
"00825_protobuf_format_table_default",
|
||||
"00834_cancel_http_readonly_queries_on_client_close",
|
||||
"00877_memory_limit_for_new_delete",
|
||||
"00900_parquet_load",
|
||||
"00933_test_fix_extra_seek_on_compressed_cache",
|
||||
"00965_logs_level_bugfix",
|
||||
"00965_send_logs_level_concurrent_queries",
|
||||
"00974_query_profiler",
|
||||
"00990_hasToken",
|
||||
"00990_metric_log_table_not_empty",
|
||||
"01014_lazy_database_concurrent_recreate_reattach_and_show_tables",
|
||||
"01017_uniqCombined_memory_usage",
|
||||
"01018_Distributed__shard_num",
|
||||
"01018_ip_dictionary_long",
|
||||
"01035_lc_empty_part_bug", # FLAKY
|
||||
"01050_clickhouse_dict_source_with_subquery",
|
||||
"01053_ssd_dictionary",
|
||||
"01054_cache_dictionary_overflow_cell",
|
||||
"01057_http_compression_prefer_brotli",
|
||||
"01080_check_for_error_incorrect_size_of_nested_column",
|
||||
"01083_expressions_in_engine_arguments",
|
||||
"01086_odbc_roundtrip",
|
||||
"01088_benchmark_query_id",
|
||||
"01092_memory_profiler",
|
||||
"01098_temporary_and_external_tables",
|
||||
"01099_parallel_distributed_insert_select",
|
||||
"01103_check_cpu_instructions_at_startup",
|
||||
"01107_atomic_db_detach_attach",
|
||||
"01114_database_atomic",
|
||||
"01148_zookeeper_path_macros_unfolding",
|
||||
"01152_cross_replication", # tcp port in reference
|
||||
"01175_distributed_ddl_output_mode_long",
|
||||
"01181_db_atomic_drop_on_cluster", # tcp port in reference
|
||||
"01280_ssd_complex_key_dictionary",
|
||||
"01293_client_interactive_vertical_multiline", # expect-test
|
||||
"01293_client_interactive_vertical_singleline", # expect-test
|
||||
"01293_show_clusters",
|
||||
"01293_show_settings",
|
||||
"01293_system_distribution_queue", # FLAKY
|
||||
"01294_lazy_database_concurrent_recreate_reattach_and_show_tables_long",
|
||||
"01294_system_distributed_on_cluster",
|
||||
"01300_client_save_history_when_terminated", # expect-test
|
||||
"01304_direct_io",
|
||||
"01306_benchmark_json",
|
||||
"01035_lc_empty_part_bug", # FLAKY
|
||||
"01175_distributed_ddl_output_mode_long", # tcp port in reference
|
||||
"01320_create_sync_race_condition_zookeeper",
|
||||
"01355_CSV_input_format_allow_errors",
|
||||
"01370_client_autocomplete_word_break_characters", # expect-test
|
||||
"01376_GROUP_BY_injective_elimination_dictGet",
|
||||
"01393_benchmark_secure_port",
|
||||
"01418_custom_settings",
|
||||
"01451_wrong_error_long_query",
|
||||
"01455_opentelemetry_distributed",
|
||||
"01473_event_time_microseconds",
|
||||
"01474_executable_dictionary",
|
||||
"01507_clickhouse_server_start_with_embedded_config",
|
||||
"01514_distributed_cancel_query_on_error",
|
||||
"01520_client_print_query_id", # expect-test
|
||||
"01526_client_start_and_exit", # expect-test
|
||||
"01526_max_untracked_memory",
|
||||
"01527_dist_sharding_key_dictGet_reload",
|
||||
"01528_play",
|
||||
"01545_url_file_format_settings",
|
||||
"01553_datetime64_comparison",
|
||||
"01555_system_distribution_queue_mask",
|
||||
"01558_ttest_scipy",
|
||||
"01561_mann_whitney_scipy",
|
||||
"01582_distinct_optimization",
|
||||
"01591_window_functions",
|
||||
"01594_too_low_memory_limits",
|
||||
"01599_multiline_input_and_singleline_comments", # expect-test
|
||||
"01601_custom_tld",
|
||||
"01606_git_import",
|
||||
"01610_client_spawn_editor", # expect-test
|
||||
"01654_test_writer_block_sequence", # No module named 'pandas'
|
||||
"01658_read_file_to_stringcolumn",
|
||||
"01666_merge_tree_max_query_limit",
|
||||
"01674_unicode_asan",
|
||||
"01676_clickhouse_client_autocomplete", # expect-test (partially)
|
||||
"01676_long_clickhouse_client_autocomplete",
|
||||
"01683_text_log_deadlock", # secure tcp
|
||||
"01684_ssd_cache_dictionary_simple_key",
|
||||
"01685_ssd_cache_dictionary_complex_key",
|
||||
"01737_clickhouse_server_wait_server_pool_long",
|
||||
"01746_executable_pool_dictionary",
|
||||
"01747_executable_pool_dictionary_implicit_key.sql",
|
||||
"01747_join_view_filter_dictionary",
|
||||
"01748_dictionary_table_dot",
|
||||
"01754_cluster_all_replicas_shard_num",
|
||||
"01759_optimize_skip_unused_shards_zero_shards",
|
||||
"01763_max_distributed_depth", # BROKEN
|
||||
"01780_clickhouse_dictionary_source_loop",
|
||||
"01801_s3_cluster",
|
||||
"01802_test_postgresql_protocol_with_row_policy",
|
||||
"01804_dictionary_decimal256_type", # hardcoded path
|
||||
"01848_http_insert_segfault",
|
||||
"01875_ssd_cache_dictionary_decimal256_type",
|
||||
"01880_remote_ipv6",
|
||||
"01889_check_row_policy_defined_using_user_function",
|
||||
"01889_clickhouse_client_config_format",
|
||||
"01903_ssd_cache_dictionary_array_type",
|
||||
]
|
||||
|
||||
|
||||
def check_result(result, error, return_code, reference, replace_map):
|
||||
if replace_map:
|
||||
for old, new in replace_map.items():
|
||||
result = result.replace(old.encode('utf-8'), new.encode('utf-8'))
|
||||
|
||||
if return_code != 0:
|
||||
try:
|
||||
print(error.decode('utf-8'), file=sys.stderr)
|
||||
except UnicodeDecodeError:
|
||||
print(error.decode('latin1'), file=sys.stderr) # encoding with 1 symbol per 1 byte, covering all values
|
||||
pytest.fail('Client died unexpectedly with code {code}'.format(code=return_code), pytrace=False)
|
||||
elif result != reference:
|
||||
pytest.fail("Query output doesn't match reference:{eol}{diff}".format(
|
||||
eol=os.linesep,
|
||||
diff=os.linesep.join(l.strip() for l in difflib.unified_diff(reference.decode('utf-8').splitlines(),
|
||||
result.decode('utf-8').splitlines(),
|
||||
fromfile='expected', tofile='actual'))),
|
||||
pytrace=False)
|
||||
|
||||
|
||||
def run_client(bin_prefix, port, database, query, reference, replace_map=None):
|
||||
# We can't use `text=True` since some tests may return binary data
|
||||
cmd = [bin_prefix + '-client', '--port', str(port), '-d', database, '-m', '-n', '--testmode']
|
||||
client = subprocess.Popen(cmd, stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
|
||||
result, error = client.communicate(query.encode('utf-8'))
|
||||
assert client.returncode is not None, "Client should exit after processing all queries"
|
||||
|
||||
check_result(result, error, client.returncode, reference, replace_map)
|
||||
|
||||
|
||||
def run_shell(bin_prefix, server, database, path, reference, replace_map=None):
|
||||
env = {
|
||||
'CLICKHOUSE_BINARY': bin_prefix,
|
||||
'CLICKHOUSE_DATABASE': database,
|
||||
'CLICKHOUSE_PORT_TCP': str(server.tcp_port),
|
||||
'CLICKHOUSE_PORT_TCP_SECURE': str(server.tcps_port),
|
||||
'CLICKHOUSE_PORT_TCP_WITH_PROXY': str(server.proxy_port),
|
||||
'CLICKHOUSE_PORT_HTTP': str(server.http_port),
|
||||
'CLICKHOUSE_PORT_INTERSERVER': str(server.inter_port),
|
||||
'CLICKHOUSE_PORT_POSTGRESQL': str(server.postgresql_port),
|
||||
'CLICKHOUSE_TMP': server.tmp_dir,
|
||||
'CLICKHOUSE_CONFIG_CLIENT': server.client_config,
|
||||
'PROTOC_BINARY': os.path.abspath(os.path.join(os.path.dirname(bin_prefix), '..', 'contrib', 'protobuf', 'protoc')), # FIXME: adhoc solution
|
||||
}
|
||||
shell = subprocess.Popen([path], env=env, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
|
||||
result, error = shell.communicate()
|
||||
assert shell.returncode is not None, "Script should exit after executing all commands"
|
||||
|
||||
check_result(result, error, shell.returncode, reference, replace_map)
|
||||
|
||||
|
||||
def random_str(length=10):
|
||||
alphabet = string.ascii_lowercase + string.digits
|
||||
random.seed(os.urandom(8))
|
||||
return ''.join(random.choice(alphabet) for _ in range(length))
|
||||
|
||||
|
||||
def test_sql_query(bin_prefix, sql_query, standalone_server):
|
||||
for test in SKIP_LIST:
|
||||
if test in sql_query:
|
||||
pytest.skip("Test matches skip-list: " + test)
|
||||
return
|
||||
|
||||
tcp_port = standalone_server.tcp_port
|
||||
|
||||
query_path = sql_query + ".sql"
|
||||
reference_path = sql_query + ".reference"
|
||||
|
||||
if not os.path.exists(reference_path):
|
||||
pytest.skip('No .reference file found')
|
||||
|
||||
with open(query_path, 'r') as file:
|
||||
query = file.read()
|
||||
with open(reference_path, 'rb') as file:
|
||||
reference = file.read()
|
||||
|
||||
random_name = 'test_{random}'.format(random=random_str())
|
||||
run_client(bin_prefix, tcp_port, 'default', 'CREATE DATABASE {random};'.format(random=random_name), b'')
|
||||
|
||||
run_client(bin_prefix, tcp_port, random_name, query, reference, {random_name: 'default'})
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);"
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED TABLES\n')
|
||||
|
||||
query = 'DROP DATABASE {random};'.format(random=random_name)
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'')
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;"
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
|
||||
|
||||
|
||||
def test_shell_query(bin_prefix, shell_query, standalone_server):
|
||||
for test in SKIP_LIST:
|
||||
if test in shell_query:
|
||||
pytest.skip("Test matches skip-list: " + test)
|
||||
return
|
||||
|
||||
tcp_port = standalone_server.tcp_port
|
||||
|
||||
shell_path = shell_query + ".sh"
|
||||
reference_path = shell_query + ".reference"
|
||||
|
||||
if not os.path.exists(reference_path):
|
||||
pytest.skip('No .reference file found')
|
||||
|
||||
with open(reference_path, 'rb') as file:
|
||||
reference = file.read()
|
||||
|
||||
random_name = 'test_{random}'.format(random=random_str())
|
||||
query = 'CREATE DATABASE {random};'.format(random=random_name)
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'')
|
||||
|
||||
run_shell(bin_prefix, standalone_server, random_name, shell_path, reference, {random_name: 'default'})
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);"
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED TABLES\n')
|
||||
|
||||
query = 'DROP DATABASE {random};'.format(random=random_name)
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'')
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;"
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
|
File diff suppressed because it is too large
Load Diff
@ -1,150 +0,0 @@
|
||||
<?xml version="1.0"?>
|
||||
<!--
|
||||
Config for test server in "Arcadia".
|
||||
This file is unused by ClickHouse continuous integration tests.
|
||||
"Arcadia" is a proprietary monorepository in Yandex, that we don't care about.
|
||||
If you are looking at this file, it is a wrong place.
|
||||
Do not edit this file.
|
||||
-->
|
||||
<clickhouse>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/tmp/clickhouse/log/clickhouse-server.log</log>
|
||||
<errorlog>/tmp/clickhouse/log/clickhouse-server.err.log</errorlog>
|
||||
<size>10M</size>
|
||||
<count>1</count>
|
||||
<compress>0</compress>
|
||||
</logger>
|
||||
<listen_host>::</listen_host>
|
||||
<listen_host>0.0.0.0</listen_host>
|
||||
<listen_try>1</listen_try>
|
||||
<http_port>58123</http_port>
|
||||
<tcp_port>59000</tcp_port>
|
||||
<https_port>58443</https_port>
|
||||
<tcp_port_secure>59440</tcp_port_secure>
|
||||
<interserver_http_port>59009</interserver_http_port>
|
||||
<max_thread_pool_size>10000</max_thread_pool_size>
|
||||
<openSSL>
|
||||
<server> <!-- Used for https server AND secure tcp port -->
|
||||
<!-- openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt -->
|
||||
<certificateFile>/tmp/clickhouse/etc/server.crt</certificateFile>
|
||||
<privateKeyFile>/tmp/clickhouse/etc/server.key</privateKeyFile>
|
||||
<!-- openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 -->
|
||||
<dhParamsFile>/tmp/clickhouse/etc/dhparam.pem</dhParamsFile>
|
||||
<verificationMode>none</verificationMode>
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
</server>
|
||||
|
||||
<client> <!-- Used for connecting to https dictionary source and secured Zookeeper communication -->
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
<verificationMode>none</verificationMode>
|
||||
<invalidCertificateHandler>
|
||||
<name>AcceptCertificateHandler</name>
|
||||
</invalidCertificateHandler>
|
||||
</client>
|
||||
</openSSL>
|
||||
|
||||
<keep_alive_timeout>3</keep_alive_timeout>
|
||||
<path>/tmp/clickhouse/data/</path>
|
||||
<tmp_path>/tmp/clickhouse/tmp/</tmp_path>
|
||||
<users_config>users.xml</users_config>
|
||||
<access_control_path>/tmp/clickhouse/data/access/</access_control_path>
|
||||
<custom_settings_prefixes>custom_</custom_settings_prefixes>
|
||||
<mark_cache_size>5368709120</mark_cache_size>
|
||||
<default_profile>default</default_profile>
|
||||
<default_database>default</default_database>
|
||||
<timezone>Europe/Moscow</timezone>
|
||||
<remote_servers incl="clickhouse_remote_servers" >
|
||||
<!-- Test only shard config for testing distributed storage -->
|
||||
<test_unavailable_shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>59000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>1</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_unavailable_shard>
|
||||
<test_shard_localhost>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>59000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_shard_localhost>
|
||||
<test_cluster_two_shards>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>127.0.0.1</host>
|
||||
<port>59000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>127.0.0.2</host>
|
||||
<port>59000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster_two_shards>
|
||||
<test_cluster_two_shards_localhost>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>59000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>59000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster_two_shards_localhost>
|
||||
<test_shard_localhost_secure>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>59440</port>
|
||||
<secure>1</secure>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_shard_localhost_secure>
|
||||
</remote_servers>
|
||||
<include_from/>
|
||||
<zookeeper incl="zookeeper-servers" optional="true" />
|
||||
<macros incl="macros" optional="true" />
|
||||
<builtin_dictionaries_reload_interval>3600</builtin_dictionaries_reload_interval>
|
||||
<max_session_timeout>3600</max_session_timeout>
|
||||
<default_session_timeout>60</default_session_timeout>
|
||||
<query_log>
|
||||
<database>system</database>
|
||||
<table>query_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</query_log>
|
||||
<dictionaries_config>*_dictionary.xml</dictionaries_config>
|
||||
<user_defined_executable_functions_config>*_function.xml</user_defined_executable_functions_config>
|
||||
<compression incl="clickhouse_compression">
|
||||
</compression>
|
||||
<distributed_ddl>
|
||||
<path>/clickhouse/task_queue/ddl</path>
|
||||
</distributed_ddl>
|
||||
<format_schema_path>/tmp/clickhouse/data/format_schemas/</format_schema_path>
|
||||
<query_masking_rules>
|
||||
<rule>
|
||||
<regexp>TOPSECRET.TOPSECRET</regexp>
|
||||
<replace>[hidden]</replace>
|
||||
</rule>
|
||||
</query_masking_rules>
|
||||
</clickhouse>
|
@ -1 +0,0 @@
|
||||
config/strings_dictionary.xml
|
@ -1,7 +0,0 @@
|
||||
<clickhouse>
|
||||
<users>
|
||||
<default>
|
||||
<access_management>1</access_management>
|
||||
</default>
|
||||
</users>
|
||||
</clickhouse>
|
@ -1 +0,0 @@
|
||||
../../programs/server/users.d/readonly.xml
|
@ -1 +0,0 @@
|
||||
../programs/server/users.xml
|
Loading…
Reference in New Issue
Block a user