Merge branch 'master' into format-settings-parsing

This commit is contained in:
Alexey Milovidov 2024-08-08 04:35:39 +02:00
commit 9cbaa28e45
19 changed files with 59 additions and 92 deletions

View File

@ -71,6 +71,7 @@ add_contrib (zlib-ng-cmake zlib-ng)
add_contrib (bzip2-cmake bzip2)
add_contrib (minizip-ng-cmake minizip-ng)
add_contrib (snappy-cmake snappy)
add_contrib (rocksdb-cmake rocksdb)
add_contrib (thrift-cmake thrift)
# parquet/arrow/orc
add_contrib (arrow-cmake arrow) # requires: snappy, thrift, double-conversion
@ -147,7 +148,6 @@ add_contrib (hive-metastore-cmake hive-metastore) # requires: thrift, avro, arro
add_contrib (cppkafka-cmake cppkafka)
add_contrib (libpqxx-cmake libpqxx)
add_contrib (libpq-cmake libpq)
add_contrib (rocksdb-cmake rocksdb) # requires: jemalloc, snappy, zlib, lz4, zstd, liburing
add_contrib (nuraft-cmake NuRaft)
add_contrib (fast_float-cmake fast_float)
add_contrib (idna-cmake idna)

2
contrib/rocksdb vendored

@ -1 +1 @@
Subproject commit 5f003e4a22d2e48e37c98d9620241237cd30dd24
Subproject commit 49ce8a1064dd1ad89117899839bf136365e49e79

View File

@ -5,38 +5,36 @@ if (NOT ENABLE_ROCKSDB OR NO_SSE3_OR_HIGHER) # assumes SSE4.2 and PCLMUL
return()
endif()
# not in original build system, otherwise xxHash.cc fails to compile with ClickHouse C++23 default
set (CMAKE_CXX_STANDARD 20)
# Always disable jemalloc for rocksdb by default because it introduces non-standard jemalloc APIs
option(WITH_JEMALLOC "build with JeMalloc" OFF)
option(WITH_LIBURING "build with liburing" OFF) # TODO could try to enable this conditionally, depending on ClickHouse's ENABLE_LIBURING
# ClickHouse cannot be compiled without snappy, lz4, zlib, zstd
option(WITH_SNAPPY "build with SNAPPY" ON)
option(WITH_LZ4 "build with lz4" ON)
option(WITH_ZLIB "build with zlib" ON)
option(WITH_ZSTD "build with zstd" ON)
if (ENABLE_JEMALLOC)
add_definitions(-DROCKSDB_JEMALLOC -DJEMALLOC_NO_DEMANGLE)
list (APPEND THIRDPARTY_LIBS ch_contrib::jemalloc)
endif ()
if (ENABLE_LIBURING)
add_definitions(-DROCKSDB_IOURING_PRESENT)
list (APPEND THIRDPARTY_LIBS ch_contrib::liburing)
endif ()
if (WITH_SNAPPY)
if(WITH_SNAPPY)
add_definitions(-DSNAPPY)
list(APPEND THIRDPARTY_LIBS ch_contrib::snappy)
endif()
if (WITH_ZLIB)
if(WITH_ZLIB)
add_definitions(-DZLIB)
list(APPEND THIRDPARTY_LIBS ch_contrib::zlib)
endif()
if (WITH_LZ4)
if(WITH_LZ4)
add_definitions(-DLZ4)
list(APPEND THIRDPARTY_LIBS ch_contrib::lz4)
endif()
if (WITH_ZSTD)
if(WITH_ZSTD)
add_definitions(-DZSTD)
list(APPEND THIRDPARTY_LIBS ch_contrib::zstd)
endif()
@ -90,7 +88,6 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/cache/sharded_cache.cc
${ROCKSDB_SOURCE_DIR}/cache/tiered_secondary_cache.cc
${ROCKSDB_SOURCE_DIR}/db/arena_wrapped_db_iter.cc
${ROCKSDB_SOURCE_DIR}/db/attribute_group_iterator_impl.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_contents.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_fetcher.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_addition.cc
@ -107,7 +104,6 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/db/blob/prefetch_buffer_collection.cc
${ROCKSDB_SOURCE_DIR}/db/builder.cc
${ROCKSDB_SOURCE_DIR}/db/c.cc
${ROCKSDB_SOURCE_DIR}/db/coalescing_iterator.cc
${ROCKSDB_SOURCE_DIR}/db/column_family.cc
${ROCKSDB_SOURCE_DIR}/db/compaction/compaction.cc
${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_iterator.cc
@ -128,7 +124,6 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_write.cc
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_compaction_flush.cc
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_files.cc
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_follower.cc
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_open.cc
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_debug.cc
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_experimental.cc
@ -186,7 +181,6 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/env/env_encryption.cc
${ROCKSDB_SOURCE_DIR}/env/file_system.cc
${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc
${ROCKSDB_SOURCE_DIR}/env/fs_on_demand.cc
${ROCKSDB_SOURCE_DIR}/env/fs_remap.cc
${ROCKSDB_SOURCE_DIR}/env/mock_env.cc
${ROCKSDB_SOURCE_DIR}/env/unique_id_gen.cc
@ -374,7 +368,6 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/volatile_tier_impl.cc
${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/cache_simulator.cc
${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/sim_cache.cc
${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_for_tiering_collector.cc
${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_on_deletion_collector.cc
${ROCKSDB_SOURCE_DIR}/utilities/trace/file_trace_reader_writer.cc
${ROCKSDB_SOURCE_DIR}/utilities/trace/replayer_impl.cc
@ -395,7 +388,6 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn_db.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn_db.cc
${ROCKSDB_SOURCE_DIR}/utilities/types_util.cc
${ROCKSDB_SOURCE_DIR}/utilities/ttl/db_ttl_impl.cc
${ROCKSDB_SOURCE_DIR}/utilities/wal_filter.cc
${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index.cc
@ -426,18 +418,14 @@ if(HAS_ARMV8_CRC)
endif(HAS_ARMV8_CRC)
list(APPEND SOURCES
${ROCKSDB_SOURCE_DIR}/port/port_posix.cc
${ROCKSDB_SOURCE_DIR}/env/env_posix.cc
${ROCKSDB_SOURCE_DIR}/env/fs_posix.cc
${ROCKSDB_SOURCE_DIR}/env/io_posix.cc)
"${ROCKSDB_SOURCE_DIR}/port/port_posix.cc"
"${ROCKSDB_SOURCE_DIR}/env/env_posix.cc"
"${ROCKSDB_SOURCE_DIR}/env/fs_posix.cc"
"${ROCKSDB_SOURCE_DIR}/env/io_posix.cc")
add_library(_rocksdb ${SOURCES})
add_library(ch_contrib::rocksdb ALIAS _rocksdb)
target_link_libraries(_rocksdb PRIVATE ${THIRDPARTY_LIBS} ${SYSTEM_LIBS})
# Not in the native build system but useful anyways:
# Make all functions in xxHash.h inline. Beneficial for performance: https://github.com/Cyan4973/xxHash/tree/v0.8.2#build-modifiers
target_compile_definitions (_rocksdb PRIVATE XXH_INLINE_ALL)
# SYSTEM is required to overcome some issues
target_include_directories(_rocksdb SYSTEM BEFORE INTERFACE "${ROCKSDB_SOURCE_DIR}/include")

View File

@ -836,6 +836,7 @@ class SettingsRandomizer:
"cross_join_min_bytes_to_compress": lambda: random.choice([0, 1, 100000000]),
"min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]),
"max_parsing_threads": lambda: random.choice([0, 1, 10]),
"trace_profile_events": lambda: random.randint(0, 1),
"optimize_functions_to_subcolumns": lambda: random.randint(0, 1),
}

View File

@ -80,5 +80,3 @@ EOF""",
instance.query(f"SHOW GRANTS FOR `{user_id}`")
== f"GRANT SELECT ON mydb.* TO `{user_id}`\n"
)
instance.stop_clickhouse()
instance.start_clickhouse()

View File

@ -37,16 +37,6 @@ def get_status(dictionary_name):
).rstrip("\n")
def get_status_retry(dictionary_name, expect, retry_count=10, sleep_time=0.5):
for _ in range(retry_count):
res = get_status(dictionary_name)
if res == expect:
return res
time.sleep(sleep_time)
raise Exception(f'Expected result "{expect}" did not occur')
def get_last_exception(dictionary_name):
return (
instance.query(
@ -263,13 +253,7 @@ def test_reload_after_fail_by_timer(started_cluster):
# on sanitizers builds it can return 'FAILED_AND_RELOADING' which is not quite right
# add retry for these builds
if (
instance.is_built_with_sanitizer()
and get_status("no_file_2") == "FAILED_AND_RELOADING"
):
get_status_retry("no_file_2", expect="FAILED")
assert get_status("no_file_2") == "FAILED"
assert get_status("no_file_2") in ["FAILED", "FAILED_AND_RELOADING"]
# Creating the file source makes the dictionary able to load.
instance.copy_file_to_container(
@ -284,7 +268,7 @@ def test_reload_after_fail_by_timer(started_cluster):
)
instance.query("SYSTEM RELOAD DICTIONARY no_file_2")
instance.query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file_2") == "LOADED"
assert get_status("no_file_2") in ["LOADED", "LOADED_AND_RELOADING"]
# Removing the file source should not spoil the loaded dictionary.
instance.exec_in_container(
@ -292,7 +276,7 @@ def test_reload_after_fail_by_timer(started_cluster):
)
time.sleep(6)
instance.query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file_2") == "LOADED"
assert get_status("no_file_2") in ["LOADED", "LOADED_AND_RELOADING"]
def test_reload_after_fail_in_cache_dictionary(started_cluster):

View File

@ -2,7 +2,9 @@
-- Tag no-fasttest: Not sure why fail even in sequential mode. Disabled for now to make some progress.
SET allow_introspection_functions = 1;
SET trace_profile_events = 0; -- This can inhibit profiler from working, because it prevents sending samples from different profilers concurrently.
SET query_profiler_cpu_time_period_ns = 0;
SET query_profiler_real_time_period_ns = 100000000;
SET log_queries = 1;
SELECT sleep(0.5), ignore('test real time query profiler');

View File

@ -14,7 +14,7 @@ min_trace_entries=2
# do not use _, they should be escaped for LIKE
query_id_tcp_prefix="01526-tcp-memory-tracking-$RANDOM-$$"
${CLICKHOUSE_CLIENT} --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 -q "with '$query_id_tcp_prefix' as __id $query FORMAT Null"
${CLICKHOUSE_CLIENT} --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --trace_profile_events 0 -q "with '$query_id_tcp_prefix' as __id $query FORMAT Null"
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
query_id_tcp="$(${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT query_id FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE '%$query_id_tcp_prefix%'")"
${CLICKHOUSE_CLIENT} -q "SELECT count()>=$min_trace_entries FROM system.trace_log WHERE query_id = '$query_id_tcp' AND abs(size) < 4e6 AND event_time >= now() - interval 1 hour"
@ -23,7 +23,7 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count()>=$min_trace_entries FROM system.trace_lo
# query_id cannot be longer then 28 bytes
query_id_http="01526_http_${RANDOM}_$$"
echo "$query" | ${CLICKHOUSE_CURL} -sSg -o /dev/null "${CLICKHOUSE_URL}&query_id=$query_id_http&max_untracked_memory=0&memory_profiler_sample_probability=1&max_threads=1" -d @-
echo "$query" | ${CLICKHOUSE_CURL} -sSg -o /dev/null "${CLICKHOUSE_URL}&query_id=$query_id_http&max_untracked_memory=0&memory_profiler_sample_probability=1&max_threads=1&trace_profile_events=0" -d @-
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
# at least 2, one allocation, one deallocation
# (but actually even more)

View File

@ -1,5 +1,9 @@
-- Tags: no-parallel, no-fasttest
-- ^ because query_thread_log is not guaranteed to be written under high load
-- (when the queue is full, events are silently dropped)
-- enforce some defaults to be sure that the env settings will not affect the test
SET max_threads=5, async_socket_for_remote=1, prefer_localhost_replica=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread', remote_filesystem_read_method='read';
SET max_threads=5, async_socket_for_remote=1, prefer_localhost_replica=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread', remote_filesystem_read_method='read', trace_profile_events=0;
-- we use query_thread_log to check peak thread usage
-- after https://github.com/ClickHouse/ClickHouse/issues/53417 there is a simpler way to check it

View File

@ -21,9 +21,9 @@ $CLICKHOUSE_CLIENT -nm -q "
drop table if exists rmt_master;
drop table if exists rmt_slave;
create table rmt_master (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'master') order by key settings always_fetch_merged_part=0;
create table rmt_master (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'master') order by key settings always_fetch_merged_part=0, old_parts_lifetime=600;
-- always_fetch_merged_part=1, consider this table as a 'slave'
create table rmt_slave (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'slave') order by key settings always_fetch_merged_part=1;
create table rmt_slave (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'slave') order by key settings always_fetch_merged_part=1, old_parts_lifetime=600;
insert into rmt_master values (1);

View File

@ -3,14 +3,11 @@
SELECT * FROM loop(numbers(3)) LIMIT 10;
SELECT * FROM loop (numbers(3)) LIMIT 10 settings max_block_size = 1;
DROP DATABASE IF EXISTS 03147_db;
CREATE DATABASE IF NOT EXISTS 03147_db;
CREATE TABLE 03147_db.t (n Int8) ENGINE=MergeTree ORDER BY n;
INSERT INTO 03147_db.t SELECT * FROM numbers(10);
USE 03147_db;
CREATE TABLE t (n Int8) ENGINE=MergeTree ORDER BY n;
INSERT INTO t SELECT * FROM numbers(10);
SELECT * FROM loop(03147_db.t) LIMIT 15;
SELECT * FROM loop({CLICKHOUSE_DATABASE:Identifier}.t) LIMIT 15;
SELECT * FROM loop(t) LIMIT 15;
SELECT * FROM loop(03147_db, t) LIMIT 15;
SELECT * FROM loop({CLICKHOUSE_DATABASE:Identifier}, t) LIMIT 15;
SELECT * FROM loop('', '') -- { serverError UNKNOWN_TABLE }

View File

@ -1,5 +1,4 @@
#!/usr/bin/env bash
# Tags: no-parallel
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,11 +1,10 @@
#!/usr/bin/env bash
# Tags: no-parallel, no-ordinary-database
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
SQL_FILE_NAME=$"03156_default_multiquery_split_$$.sql"
SQL_FILE_NAME=$"03156_default_multiquery_split_${CLICKHOUSE_DATABASE}.sql"
# The old multiquery implementation uses '\n' to split INSERT query segmentation
# this case is mainly to test the following situations

View File

@ -1,4 +1,4 @@
-- Tags: no-fasttest, no-parallel, no-random-settings
-- Tags: no-fasttest, no-random-settings
set max_insert_threads=1;
@ -22,4 +22,4 @@ CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, c
INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000);
SELECT max(blockSize()) FROM test_parquet;
DROP TABLE IF EXISTS test_parquet;
DROP TABLE IF EXISTS test_parquet;

View File

@ -1,4 +1,3 @@
1
3168 8613
[] ['SELECT(a, b) ON default.d_03168_query_log']
[] []

View File

@ -1,32 +1,28 @@
#!/usr/bin/env bash
# Tags: no-parallel
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
user_name="u_03168_query_log"
table_name="default.d_03168_query_log"
user_name="u_03168_query_log_${CLICKHOUSE_DATABASE}"
table_name="d_03168_query_log"
test_query="select a, b from ${table_name}"
${CLICKHOUSE_CLIENT_BINARY} --query "drop user if exists ${user_name}"
${CLICKHOUSE_CLIENT_BINARY} --query "create user ${user_name}"
${CLICKHOUSE_CLIENT_BINARY} --query "drop table if exists ${table_name}"
${CLICKHOUSE_CLIENT_BINARY} --query "create table ${table_name} (a UInt64, b UInt64) order by a"
${CLICKHOUSE_CLIENT} --query "drop user if exists ${user_name}"
${CLICKHOUSE_CLIENT} --query "create user ${user_name}"
${CLICKHOUSE_CLIENT} --query "drop table if exists ${table_name}"
${CLICKHOUSE_CLIENT} --query "create table ${table_name} (a UInt64, b UInt64) order by a"
${CLICKHOUSE_CLIENT} --query "insert into table ${table_name} values (3168, 8613)"
${CLICKHOUSE_CLIENT_BINARY} --query "insert into table ${table_name} values (3168, 8613)"
${CLICKHOUSE_CLIENT} --user ${user_name} --query "${test_query}" 2>&1 >/dev/null | (grep -q "ACCESS_DENIED" || echo "Expected ACCESS_DENIED error not found")
error="$(${CLICKHOUSE_CLIENT_BINARY} --user ${user_name} --query "${test_query}" 2>&1 >/dev/null)"
echo "${error}" | grep -Fc "ACCESS_DENIED"
${CLICKHOUSE_CLIENT} --query "grant select(a, b) on ${table_name} to ${user_name}"
${CLICKHOUSE_CLIENT} --user ${user_name} --query "${test_query}"
${CLICKHOUSE_CLIENT_BINARY} --query "grant select(a, b) on ${table_name} to ${user_name}"
${CLICKHOUSE_CLIENT} --query "system flush logs"
${CLICKHOUSE_CLIENT} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'ExceptionBeforeStart' and current_database = currentDatabase() order by event_time desc limit 1"
${CLICKHOUSE_CLIENT} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryStart' and current_database = currentDatabase() order by event_time desc limit 1"
${CLICKHOUSE_CLIENT} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryFinish' and current_database = currentDatabase() order by event_time desc limit 1"
${CLICKHOUSE_CLIENT_BINARY} --user ${user_name} --query "${test_query}"
${CLICKHOUSE_CLIENT_BINARY} --query "system flush logs"
${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'ExceptionBeforeStart' and current_database = currentDatabase() order by event_time desc limit 1"
${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryStart' and current_database = currentDatabase() order by event_time desc limit 1"
${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryFinish' and current_database = currentDatabase() order by event_time desc limit 1"
${CLICKHOUSE_CLIENT_BINARY} --query "drop table ${table_name}"
${CLICKHOUSE_CLIENT_BINARY} --query "drop user ${user_name}"
${CLICKHOUSE_CLIENT} --query "drop table ${table_name}"
${CLICKHOUSE_CLIENT} --query "drop user ${user_name}"

View File

@ -1,5 +1,3 @@
-- Tags: no-parallel
CREATE TABLE x ( hash_id UInt64, user_result Decimal(3, 2) ) ENGINE = Memory();
CREATE TABLE y ( hash_id UInt64, user_result DECIMAL(18, 6) ) ENGINE = Memory();

View File

@ -1,4 +1,5 @@
-- Tags: no-random-settings, no-object-storage, no-parallel
-- no-parallel: Running `DROP MARK CACHE` can have a big impact on other concurrent tests
-- Tag no-object-storage: this test relies on the number of opened files in MergeTree that can differ in object storages
SET allow_experimental_dynamic_type = 1;

View File

@ -1,5 +1,6 @@
#!/usr/bin/env bash
# Tags: no-parallel, no-fasttest
# Tags: no-fasttest
# no-fasttest: Requires libraries
set -e