Merge branch 'master' into named-collections-in-clickhouse-local

This commit is contained in:
Alexey Milovidov 2024-07-11 21:50:25 +02:00
commit ec0acf65c4
38 changed files with 491 additions and 253 deletions

2
contrib/azure vendored

@ -1 +1 @@
Subproject commit 92c94d7f37a43cc8fc4d466884a95f610c0593bf
Subproject commit ea3e19a7be08519134c643177d56c7484dfec884

View File

@ -179,12 +179,19 @@ endif ()
target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_PROF=1)
# jemalloc provides support for two different libunwind flavors: the original HP libunwind and the one coming with gcc / g++ / libstdc++.
# The latter is identified by `JEMALLOC_PROF_LIBGCC` and uses `_Unwind_Backtrace` method instead of `unw_backtrace`.
# At the time ClickHouse uses LLVM libunwind which follows libgcc's way of backtracking.
# jemalloc provides support two unwind flavors:
# - JEMALLOC_PROF_LIBUNWIND - unw_backtrace() - gnu libunwind (compatible with llvm libunwind)
# - JEMALLOC_PROF_LIBGCC - _Unwind_Backtrace() - the original HP libunwind and the one coming with gcc / g++ / libstdc++.
#
# ClickHouse has to provide `unw_backtrace` method by the means of [commit 8e2b31e](https://github.com/ClickHouse/libunwind/commit/8e2b31e766dd502f6df74909e04a7dbdf5182eb1).
target_compile_definitions (_jemalloc PRIVATE -DJEMALLOC_PROF_LIBGCC=1)
# But for JEMALLOC_PROF_LIBGCC it also calls _Unwind_Backtrace() during
# bootstraping of jemalloc, which may lead to deadlock, if the dlsym will do
# allocations somewhere (like glibc does prio 2.34, see [1]).
#
# [1]: https://sourceware.org/git/?p=glibc.git;a=commit;h=fada9018199c21c469ff0e731ef75c6020074ac9
#
# And since ClickHouse unwind already supports unw_backtrace() we can safely
# switch to it to avoid this deadlock.
target_compile_definitions (_jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1)
target_link_libraries (_jemalloc PRIVATE unwind)
# for RTLD_NEXT

2
contrib/pocketfft vendored

@ -1 +1 @@
Subproject commit 9efd4da52cf8d28d14531d14e43ad9d913807546
Subproject commit f4c1aa8aa9ce79ad39e80f2c9c41b92ead90fda3

2
contrib/rocksdb vendored

@ -1 +1 @@
Subproject commit 078fa5638690004e1f744076d1bdcc4e93767304
Subproject commit be366233921293bd07a84dc4ea6991858665f202

View File

@ -5,20 +5,13 @@ if (NOT ENABLE_ROCKSDB)
return()
endif()
## this file is extracted from `contrib/rocksdb/CMakeLists.txt`
set(ROCKSDB_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb")
list(APPEND CMAKE_MODULE_PATH "${ROCKSDB_SOURCE_DIR}/cmake/modules/")
set(PORTABLE ON)
## always disable jemalloc for rocksdb by default
## because it introduces non-standard jemalloc APIs
# Always disable jemalloc for rocksdb by default because it introduces non-standard jemalloc APIs
option(WITH_JEMALLOC "build with JeMalloc" OFF)
set(USE_SNAPPY OFF)
if (TARGET ch_contrib::snappy)
set(USE_SNAPPY ON)
endif()
option(WITH_SNAPPY "build with SNAPPY" ${USE_SNAPPY})
## lz4, zlib, zstd is enabled in ClickHouse by default
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)
@ -26,78 +19,46 @@ option(WITH_ZSTD "build with zstd" ON)
# third-party/folly is only validated to work on Linux and Windows for now.
# So only turn it on there by default.
if(CMAKE_SYSTEM_NAME MATCHES "Linux|Windows")
if(MSVC AND MSVC_VERSION LESS 1910)
# Folly does not compile with MSVC older than VS2017
option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" OFF)
else()
option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" ON)
endif()
option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" ON)
else()
option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" OFF)
endif()
if( NOT DEFINED CMAKE_CXX_STANDARD )
set(CMAKE_CXX_STANDARD 11)
if(WITH_SNAPPY)
add_definitions(-DSNAPPY)
list(APPEND THIRDPARTY_LIBS ch_contrib::snappy)
endif()
if(MSVC)
option(WITH_XPRESS "build with windows built in compression" OFF)
include("${ROCKSDB_SOURCE_DIR}/thirdparty.inc")
else()
if(CMAKE_SYSTEM_NAME MATCHES "FreeBSD" AND NOT CMAKE_SYSTEM_NAME MATCHES "kFreeBSD")
# FreeBSD has jemalloc as default malloc
# but it does not have all the jemalloc files in include/...
set(WITH_JEMALLOC ON)
else()
if(WITH_JEMALLOC AND TARGET ch_contrib::jemalloc)
add_definitions(-DROCKSDB_JEMALLOC -DJEMALLOC_NO_DEMANGLE)
list(APPEND THIRDPARTY_LIBS ch_contrib::jemalloc)
endif()
endif()
if(WITH_SNAPPY)
add_definitions(-DSNAPPY)
list(APPEND THIRDPARTY_LIBS ch_contrib::snappy)
endif()
if(WITH_ZLIB)
add_definitions(-DZLIB)
list(APPEND THIRDPARTY_LIBS ch_contrib::zlib)
endif()
if(WITH_LZ4)
add_definitions(-DLZ4)
list(APPEND THIRDPARTY_LIBS ch_contrib::lz4)
endif()
if(WITH_ZSTD)
add_definitions(-DZSTD)
list(APPEND THIRDPARTY_LIBS ch_contrib::zstd)
endif()
if(WITH_ZLIB)
add_definitions(-DZLIB)
list(APPEND THIRDPARTY_LIBS ch_contrib::zlib)
endif()
if(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64")
if(POWER9)
set(HAS_POWER9 1)
set(HAS_ALTIVEC 1)
else()
set(HAS_POWER8 1)
set(HAS_ALTIVEC 1)
endif(POWER9)
endif(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64")
if(WITH_LZ4)
add_definitions(-DLZ4)
list(APPEND THIRDPARTY_LIBS ch_contrib::lz4)
endif()
if(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64|arm64|ARM64")
set(HAS_ARMV8_CRC 1)
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function")
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function")
endif(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64|arm64|ARM64")
if(WITH_ZSTD)
add_definitions(-DZSTD)
list(APPEND THIRDPARTY_LIBS ch_contrib::zstd)
endif()
option(PORTABLE "build a portable binary" ON)
if(ENABLE_AVX2 AND ENABLE_PCLMULQDQ)
if(ENABLE_SSE42 AND ENABLE_PCLMULQDQ)
add_definitions(-DHAVE_SSE42)
add_definitions(-DHAVE_PCLMUL)
endif()
if(CMAKE_SYSTEM_PROCESSOR MATCHES "arm64|aarch64|AARCH64")
set (HAS_ARMV8_CRC 1)
# the original build descriptions set specific flags for ARM. These flags are already subsumed by ClickHouse's general
# ARM flags, see cmake/cpu_features.cmake
# set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function")
# set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function")
endif()
set (HAVE_THREAD_LOCAL 1)
if(HAVE_THREAD_LOCAL)
add_definitions(-DROCKSDB_SUPPORT_THREAD_LOCAL)
@ -107,8 +68,6 @@ if(CMAKE_SYSTEM_NAME MATCHES "Darwin")
add_definitions(-DOS_MACOSX)
elseif(CMAKE_SYSTEM_NAME MATCHES "Linux")
add_definitions(-DOS_LINUX)
elseif(CMAKE_SYSTEM_NAME MATCHES "SunOS")
add_definitions(-DOS_SOLARIS)
elseif(CMAKE_SYSTEM_NAME MATCHES "FreeBSD")
add_definitions(-DOS_FREEBSD)
elseif(CMAKE_SYSTEM_NAME MATCHES "Android")
@ -123,12 +82,10 @@ endif()
if (OS_LINUX)
add_definitions(-DROCKSDB_SCHED_GETCPU_PRESENT)
add_definitions(-DROCKSDB_AUXV_SYSAUXV_PRESENT)
add_definitions(-DROCKSDB_AUXV_GETAUXVAL_PRESENT)
elseif (OS_FREEBSD)
add_definitions(-DROCKSDB_AUXV_SYSAUXV_PRESENT)
endif()
set(ROCKSDB_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb")
include_directories(${ROCKSDB_SOURCE_DIR})
include_directories("${ROCKSDB_SOURCE_DIR}/include")
@ -136,11 +93,11 @@ if(WITH_FOLLY_DISTRIBUTED_MUTEX)
include_directories("${ROCKSDB_SOURCE_DIR}/third-party/folly")
endif()
# Main library source code
set(SOURCES
${ROCKSDB_SOURCE_DIR}/cache/cache.cc
${ROCKSDB_SOURCE_DIR}/cache/cache_entry_roles.cc
${ROCKSDB_SOURCE_DIR}/cache/cache_key.cc
${ROCKSDB_SOURCE_DIR}/cache/cache_reservation_manager.cc
${ROCKSDB_SOURCE_DIR}/cache/clock_cache.cc
${ROCKSDB_SOURCE_DIR}/cache/lru_cache.cc
${ROCKSDB_SOURCE_DIR}/cache/sharded_cache.cc
@ -156,6 +113,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_format.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_sequential_reader.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_writer.cc
${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/column_family.cc
@ -229,6 +187,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc
${ROCKSDB_SOURCE_DIR}/env/fs_remap.cc
${ROCKSDB_SOURCE_DIR}/env/mock_env.cc
${ROCKSDB_SOURCE_DIR}/env/unique_id_gen.cc
${ROCKSDB_SOURCE_DIR}/file/delete_scheduler.cc
${ROCKSDB_SOURCE_DIR}/file/file_prefetch_buffer.cc
${ROCKSDB_SOURCE_DIR}/file/file_util.cc
@ -247,6 +206,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/memory/concurrent_arena.cc
${ROCKSDB_SOURCE_DIR}/memory/jemalloc_nodump_allocator.cc
${ROCKSDB_SOURCE_DIR}/memory/memkind_kmem_allocator.cc
${ROCKSDB_SOURCE_DIR}/memory/memory_allocator.cc
${ROCKSDB_SOURCE_DIR}/memtable/alloc_tracker.cc
${ROCKSDB_SOURCE_DIR}/memtable/hash_linklist_rep.cc
${ROCKSDB_SOURCE_DIR}/memtable/hash_skiplist_rep.cc
@ -322,6 +282,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/table/table_factory.cc
${ROCKSDB_SOURCE_DIR}/table/table_properties.cc
${ROCKSDB_SOURCE_DIR}/table/two_level_iterator.cc
${ROCKSDB_SOURCE_DIR}/table/unique_id.cc
${ROCKSDB_SOURCE_DIR}/test_util/sync_point.cc
${ROCKSDB_SOURCE_DIR}/test_util/sync_point_impl.cc
${ROCKSDB_SOURCE_DIR}/test_util/testutil.cc
@ -333,9 +294,12 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/tools/ldb_tool.cc
${ROCKSDB_SOURCE_DIR}/tools/sst_dump_tool.cc
${ROCKSDB_SOURCE_DIR}/tools/trace_analyzer_tool.cc
${ROCKSDB_SOURCE_DIR}/trace_replay/trace_replay.cc
${ROCKSDB_SOURCE_DIR}/trace_replay/block_cache_tracer.cc
${ROCKSDB_SOURCE_DIR}/trace_replay/io_tracer.cc
${ROCKSDB_SOURCE_DIR}/trace_replay/trace_record_handler.cc
${ROCKSDB_SOURCE_DIR}/trace_replay/trace_record_result.cc
${ROCKSDB_SOURCE_DIR}/trace_replay/trace_record.cc
${ROCKSDB_SOURCE_DIR}/trace_replay/trace_replay.cc
${ROCKSDB_SOURCE_DIR}/util/coding.cc
${ROCKSDB_SOURCE_DIR}/util/compaction_job_stats_impl.cc
${ROCKSDB_SOURCE_DIR}/util/comparator.cc
@ -347,6 +311,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/util/murmurhash.cc
${ROCKSDB_SOURCE_DIR}/util/random.cc
${ROCKSDB_SOURCE_DIR}/util/rate_limiter.cc
${ROCKSDB_SOURCE_DIR}/util/regex.cc
${ROCKSDB_SOURCE_DIR}/util/ribbon_config.cc
${ROCKSDB_SOURCE_DIR}/util/slice.cc
${ROCKSDB_SOURCE_DIR}/util/file_checksum_helper.cc
@ -362,18 +327,23 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db_impl_filesnapshot.cc
${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_dump_tool.cc
${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_file.cc
${ROCKSDB_SOURCE_DIR}/utilities/cache_dump_load.cc
${ROCKSDB_SOURCE_DIR}/utilities/cache_dump_load_impl.cc
${ROCKSDB_SOURCE_DIR}/utilities/cassandra/cassandra_compaction_filter.cc
${ROCKSDB_SOURCE_DIR}/utilities/cassandra/format.cc
${ROCKSDB_SOURCE_DIR}/utilities/cassandra/merge_operator.cc
${ROCKSDB_SOURCE_DIR}/utilities/checkpoint/checkpoint_impl.cc
${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters.cc
${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters/remove_emptyvalue_compactionfilter.cc
${ROCKSDB_SOURCE_DIR}/utilities/debug.cc
${ROCKSDB_SOURCE_DIR}/utilities/env_mirror.cc
${ROCKSDB_SOURCE_DIR}/utilities/env_timed.cc
${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_env.cc
${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_fs.cc
${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_secondary_cache.cc
${ROCKSDB_SOURCE_DIR}/utilities/leveldb_options/leveldb_options.cc
${ROCKSDB_SOURCE_DIR}/utilities/memory/memory_util.cc
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators.cc
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/bytesxor.cc
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/max.cc
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/put.cc
@ -393,6 +363,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/sim_cache.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
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/lock_manager.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point/point_lock_tracker.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point/point_lock_manager.cc
@ -411,6 +382,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn_db.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
${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index_internal.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.cc
@ -425,7 +397,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/standalone_port.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/util/dbt.cc
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc
rocksdb_build_version.cc)
build_version.cc) # generated by hand
if(ENABLE_SSE42 AND ENABLE_PCLMULQDQ)
set_source_files_properties(
@ -462,5 +434,6 @@ endif()
add_library(_rocksdb ${SOURCES})
add_library(ch_contrib::rocksdb ALIAS _rocksdb)
target_link_libraries(_rocksdb PRIVATE ${THIRDPARTY_LIBS} ${SYSTEM_LIBS})
# SYSTEM is required to overcome some issues
target_include_directories(_rocksdb SYSTEM BEFORE INTERFACE "${ROCKSDB_SOURCE_DIR}/include")

View File

@ -16,6 +16,9 @@ dpkg -i package_folder/clickhouse-client_*.deb
ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
# shellcheck disable=SC1091
source /utils.lib
# install test configs
/usr/share/clickhouse-test/config/install.sh
@ -272,3 +275,5 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
mv /var/log/clickhouse-server/stderr1.log /test_output/ ||:
mv /var/log/clickhouse-server/stderr2.log /test_output/ ||:
fi
collect_core_dumps

View File

@ -12,8 +12,7 @@ MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 10800 : MAX_RUN_TIME))
USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0}
USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0}
# disable for now
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=1
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0
@ -310,7 +309,7 @@ function run_tests()
try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')"
set +e
clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
timeout -k 60m -s TERM --preserve-status 140m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
--no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee -a test_output/test_result.txt
@ -483,3 +482,5 @@ if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
mv /var/log/clickhouse-server/stderr1.log /test_output/ ||:
tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||:
fi
collect_core_dumps

View File

@ -1,8 +1,5 @@
#!/bin/bash
# core.COMM.PID-TID
sysctl kernel.core_pattern='core.%e.%p-%P'
OK="\tOK\t\\N\t"
FAIL="\tFAIL\t\\N\t"
@ -315,12 +312,4 @@ function collect_query_and_trace_logs()
done
}
function collect_core_dumps()
{
find . -type f -maxdepth 1 -name 'core.*' | while read -r core; do
zstd --threads=0 "$core"
mv "$core.zst" /test_output/
done
}
# vi: ft=bash

View File

@ -1,5 +1,10 @@
#!/bin/bash
# core.COMM.PID-TID
sysctl kernel.core_pattern='core.%e.%p-%P'
# ASAN doesn't work with suid_dumpable=2
sysctl fs.suid_dumpable=1
function run_with_retry()
{
if [[ $- =~ e ]]; then
@ -48,4 +53,12 @@ function timeout_with_logging() {
return $exit_code
}
function collect_core_dumps()
{
find . -type f -maxdepth 1 -name 'core.*' | while read -r core; do
zstd --threads=0 "$core"
mv "$core.zst" /test_output/
done
}
# vi: ft=bash

View File

@ -21,6 +21,9 @@ source /attach_gdb.lib
# shellcheck source=../stateless/stress_tests.lib
source /stress_tests.lib
# shellcheck disable=SC1091
source /utils.lib
install_packages package_folder
# Thread Fuzzer allows to check more permutations of possible thread scheduling

View File

@ -235,7 +235,7 @@ bool NamedCollectionFactory::loadIfNot(std::lock_guard<std::mutex> & lock)
loadFromConfig(context->getConfigRef(), lock);
loadFromSQL(lock);
if (metadata_storage->supportsPeriodicUpdate())
if (metadata_storage->isReplicated())
{
update_task = context->getSchedulePool().createTask("NamedCollectionsMetadataStorage", [this]{ updateFunc(); });
update_task->activate();
@ -357,6 +357,13 @@ void NamedCollectionFactory::reloadFromSQL()
add(std::move(collections), lock);
}
bool NamedCollectionFactory::usesReplicatedStorage()
{
std::lock_guard lock(mutex);
loadIfNot(lock);
return metadata_storage->isReplicated();
}
void NamedCollectionFactory::updateFunc()
{
LOG_TRACE(log, "Named collections background updating thread started");

View File

@ -34,6 +34,8 @@ public:
void updateFromSQL(const ASTAlterNamedCollectionQuery & query);
bool usesReplicatedStorage();
void loadIfNot();
void shutdown();

View File

@ -67,7 +67,7 @@ public:
virtual bool removeIfExists(const std::string & path) = 0;
virtual bool supportsPeriodicUpdate() const = 0;
virtual bool isReplicated() const = 0;
virtual bool waitUpdate(size_t /* timeout */) { return false; }
};
@ -89,7 +89,7 @@ public:
~LocalStorage() override = default;
bool supportsPeriodicUpdate() const override { return false; }
bool isReplicated() const override { return false; }
std::vector<std::string> list() const override
{
@ -221,7 +221,7 @@ public:
~ZooKeeperStorage() override = default;
bool supportsPeriodicUpdate() const override { return true; }
bool isReplicated() const override { return true; }
/// Return true if children changed.
bool waitUpdate(size_t timeout) override
@ -465,14 +465,14 @@ void NamedCollectionsMetadataStorage::writeCreateQuery(const ASTCreateNamedColle
storage->write(getFileName(query.collection_name), serializeAST(*normalized_query), replace);
}
bool NamedCollectionsMetadataStorage::supportsPeriodicUpdate() const
bool NamedCollectionsMetadataStorage::isReplicated() const
{
return storage->supportsPeriodicUpdate();
return storage->isReplicated();
}
bool NamedCollectionsMetadataStorage::waitUpdate()
{
if (!storage->supportsPeriodicUpdate())
if (!storage->isReplicated())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Periodic updates are not supported");
const auto & config = Context::getGlobalContextInstance()->getConfigRef();

View File

@ -30,7 +30,7 @@ public:
/// Return true if update was made
bool waitUpdate();
bool supportsPeriodicUpdate() const;
bool isReplicated() const;
private:
class INamedCollectionsStorage;

View File

@ -346,6 +346,7 @@ class IColumn;
\
M(Bool, ignore_on_cluster_for_replicated_udf_queries, false, "Ignore ON CLUSTER clause for replicated UDF management queries.", 0) \
M(Bool, ignore_on_cluster_for_replicated_access_entities_queries, false, "Ignore ON CLUSTER clause for replicated access entities management queries.", 0) \
M(Bool, ignore_on_cluster_for_replicated_named_collections_queries, false, "Ignore ON CLUSTER clause for replicated named collections management queries.", 0) \
/** Settings for testing hedged requests */ \
M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \
M(Milliseconds, sleep_in_send_data_ms, 0, "Time to sleep in sending data in TCPHandler", 0) \

View File

@ -76,6 +76,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"},
{"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"},
{"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"},
{"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."},
{"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."},
{"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}
}},

View File

@ -4,6 +4,7 @@
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/removeOnClusterClauseIfNeeded.h>
#include <Common/NamedCollections/NamedCollectionsFactory.h>
@ -13,14 +14,16 @@ namespace DB
BlockIO InterpreterAlterNamedCollectionQuery::execute()
{
auto current_context = getContext();
const auto & query = query_ptr->as<const ASTAlterNamedCollectionQuery &>();
const auto updated_query = removeOnClusterClauseIfNeeded(query_ptr, getContext());
const auto & query = updated_query->as<const ASTAlterNamedCollectionQuery &>();
current_context->checkAccess(AccessType::ALTER_NAMED_COLLECTION, query.collection_name);
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
return executeDDLQueryOnCluster(updated_query, current_context, params);
}
NamedCollectionFactory::instance().updateFromSQL(query);

View File

@ -4,6 +4,7 @@
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/removeOnClusterClauseIfNeeded.h>
#include <Common/NamedCollections/NamedCollectionsFactory.h>
@ -13,14 +14,16 @@ namespace DB
BlockIO InterpreterCreateNamedCollectionQuery::execute()
{
auto current_context = getContext();
const auto & query = query_ptr->as<const ASTCreateNamedCollectionQuery &>();
const auto updated_query = removeOnClusterClauseIfNeeded(query_ptr, getContext());
const auto & query = updated_query->as<const ASTCreateNamedCollectionQuery &>();
current_context->checkAccess(AccessType::CREATE_NAMED_COLLECTION, query.collection_name);
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
return executeDDLQueryOnCluster(updated_query, current_context, params);
}
NamedCollectionFactory::instance().createFromSQL(query);

View File

@ -4,6 +4,7 @@
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/removeOnClusterClauseIfNeeded.h>
#include <Common/NamedCollections/NamedCollectionsFactory.h>
@ -13,14 +14,16 @@ namespace DB
BlockIO InterpreterDropNamedCollectionQuery::execute()
{
auto current_context = getContext();
const auto & query = query_ptr->as<const ASTDropNamedCollectionQuery &>();
const auto updated_query = removeOnClusterClauseIfNeeded(query_ptr, getContext());
const auto & query = updated_query->as<const ASTDropNamedCollectionQuery &>();
current_context->checkAccess(AccessType::DROP_NAMED_COLLECTION, query.collection_name);
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
return executeDDLQueryOnCluster(updated_query, current_context, params);
}
NamedCollectionFactory::instance().removeFromSQL(query);

View File

@ -15,6 +15,10 @@
#include <Parsers/Access/ASTCreateUserQuery.h>
#include <Parsers/Access/ASTDropAccessEntityQuery.h>
#include <Parsers/Access/ASTGrantQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Common/NamedCollections/NamedCollectionsFactory.h>
namespace DB
@ -38,6 +42,13 @@ static bool isAccessControlQuery(const ASTPtr & query)
|| query->as<ASTGrantQuery>();
}
static bool isNamedCollectionQuery(const ASTPtr & query)
{
return query->as<ASTCreateNamedCollectionQuery>()
|| query->as<ASTDropNamedCollectionQuery>()
|| query->as<ASTAlterNamedCollectionQuery>();
}
ASTPtr removeOnClusterClauseIfNeeded(const ASTPtr & query, ContextPtr context, const WithoutOnClusterASTRewriteParams & params)
{
auto * query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(query.get());
@ -50,7 +61,10 @@ ASTPtr removeOnClusterClauseIfNeeded(const ASTPtr & query, ContextPtr context, c
&& context->getUserDefinedSQLObjectsStorage().isReplicated())
|| (isAccessControlQuery(query)
&& context->getSettings().ignore_on_cluster_for_replicated_access_entities_queries
&& context->getAccessControl().containsStorage(ReplicatedAccessStorage::STORAGE_TYPE)))
&& context->getAccessControl().containsStorage(ReplicatedAccessStorage::STORAGE_TYPE))
|| (isNamedCollectionQuery(query)
&& context->getSettings().ignore_on_cluster_for_replicated_named_collections_queries
&& NamedCollectionFactory::instance().usesReplicatedStorage()))
{
LOG_DEBUG(getLogger("removeOnClusterClauseIfNeeded"), "ON CLUSTER clause was ignored for query {}", query->getID());
return query_on_cluster->getRewrittenASTWithoutOnCluster(params);

View File

@ -8,6 +8,7 @@
#include <Common/logger_useful.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnArray.h>
#include <Formats/FormatFactory.h>
#include <IO/ReadBufferFromFileBase.h>
@ -30,6 +31,7 @@
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/NestedUtils.h>
#include <boost/algorithm/string/case_conv.hpp>
#include <parquet/file_reader.h>
@ -111,7 +113,7 @@ struct DeltaLakeMetadataImpl
std::set<String> result_files;
NamesAndTypesList current_schema;
DataLakePartitionColumns current_partition_columns;
const auto checkpoint_version = getCheckpointIfExists(result_files);
const auto checkpoint_version = getCheckpointIfExists(result_files, current_schema, current_partition_columns);
if (checkpoint_version)
{
@ -205,9 +207,32 @@ struct DeltaLakeMetadataImpl
Poco::Dynamic::Var json = parser.parse(json_str);
Poco::JSON::Object::Ptr object = json.extract<Poco::JSON::Object::Ptr>();
// std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
// object->stringify(oss);
// LOG_TEST(log, "Metadata: {}", oss.str());
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
object->stringify(oss);
LOG_TEST(log, "Metadata: {}", oss.str());
if (object->has("metaData"))
{
const auto metadata_object = object->get("metaData").extract<Poco::JSON::Object::Ptr>();
const auto schema_object = metadata_object->getValue<String>("schemaString");
Poco::JSON::Parser p;
Poco::Dynamic::Var fields_json = parser.parse(schema_object);
const Poco::JSON::Object::Ptr & fields_object = fields_json.extract<Poco::JSON::Object::Ptr>();
auto current_schema = parseMetadata(fields_object);
if (file_schema.empty())
{
file_schema = current_schema;
}
else if (file_schema != current_schema)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Reading from files with different schema is not possible "
"({} is different from {})",
file_schema.toString(), current_schema.toString());
}
}
if (object->has("add"))
{
@ -230,7 +255,12 @@ struct DeltaLakeMetadataImpl
const auto value = partition_values->getValue<String>(partition_name);
auto name_and_type = file_schema.tryGetByName(partition_name);
if (!name_and_type)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No such column in schema: {}", partition_name);
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"No such column in schema: {} (schema: {})",
partition_name, file_schema.toNamesAndTypesDescription());
}
auto field = getFieldValue(value, name_and_type->type);
current_partition_columns.emplace_back(*name_and_type, field);
@ -246,52 +276,35 @@ struct DeltaLakeMetadataImpl
auto path = object->get("remove").extract<Poco::JSON::Object::Ptr>()->getValue<String>("path");
result.erase(fs::path(configuration->getPath()) / path);
}
if (object->has("metaData"))
{
const auto metadata_object = object->get("metaData").extract<Poco::JSON::Object::Ptr>();
const auto schema_object = metadata_object->getValue<String>("schemaString");
Poco::JSON::Parser p;
Poco::Dynamic::Var fields_json = parser.parse(schema_object);
Poco::JSON::Object::Ptr fields_object = fields_json.extract<Poco::JSON::Object::Ptr>();
const auto fields = fields_object->get("fields").extract<Poco::JSON::Array::Ptr>();
NamesAndTypesList current_schema;
for (size_t i = 0; i < fields->size(); ++i)
{
const auto field = fields->getObject(static_cast<UInt32>(i));
auto column_name = field->getValue<String>("name");
auto type = field->getValue<String>("type");
auto is_nullable = field->getValue<bool>("nullable");
std::string physical_name;
auto schema_metadata_object = field->get("metadata").extract<Poco::JSON::Object::Ptr>();
if (schema_metadata_object->has("delta.columnMapping.physicalName"))
physical_name = schema_metadata_object->getValue<String>("delta.columnMapping.physicalName");
else
physical_name = column_name;
LOG_TEST(log, "Found column: {}, type: {}, nullable: {}, physical name: {}",
column_name, type, is_nullable, physical_name);
current_schema.push_back({physical_name, getFieldType(field, "type", is_nullable)});
}
if (file_schema.empty())
{
file_schema = current_schema;
}
else if (file_schema != current_schema)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Reading from files with different schema is not possible "
"({} is different from {})",
file_schema.toString(), current_schema.toString());
}
}
}
}
NamesAndTypesList parseMetadata(const Poco::JSON::Object::Ptr & metadata_json)
{
NamesAndTypesList schema;
const auto fields = metadata_json->get("fields").extract<Poco::JSON::Array::Ptr>();
for (size_t i = 0; i < fields->size(); ++i)
{
const auto field = fields->getObject(static_cast<UInt32>(i));
auto column_name = field->getValue<String>("name");
auto type = field->getValue<String>("type");
auto is_nullable = field->getValue<bool>("nullable");
std::string physical_name;
auto schema_metadata_object = field->get("metadata").extract<Poco::JSON::Object::Ptr>();
if (schema_metadata_object->has("delta.columnMapping.physicalName"))
physical_name = schema_metadata_object->getValue<String>("delta.columnMapping.physicalName");
else
physical_name = column_name;
LOG_TEST(log, "Found column: {}, type: {}, nullable: {}, physical name: {}",
column_name, type, is_nullable, physical_name);
schema.push_back({physical_name, getFieldType(field, "type", is_nullable)});
}
return schema;
}
DataTypePtr getFieldType(const Poco::JSON::Object::Ptr & field, const String & type_key, bool is_nullable)
{
if (field->isObject(type_key))
@ -505,7 +518,10 @@ struct DeltaLakeMetadataImpl
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Arrow error: {}", _s.ToString()); \
} while (false)
size_t getCheckpointIfExists(std::set<String> & result)
size_t getCheckpointIfExists(
std::set<String> & result,
NamesAndTypesList & file_schema,
DataLakePartitionColumns & file_partition_columns)
{
const auto version = readLastCheckpointIfExists();
if (!version)
@ -526,7 +542,8 @@ struct DeltaLakeMetadataImpl
auto columns = ParquetSchemaReader(*buf, format_settings).readSchema();
/// Read only columns that we need.
columns.filterColumns(NameSet{"add", "remove"});
auto filter_column_names = NameSet{"add", "metaData"};
columns.filterColumns(filter_column_names);
Block header;
for (const auto & column : columns)
header.insert({column.type->createColumn(), column.type, column.name});
@ -540,9 +557,6 @@ struct DeltaLakeMetadataImpl
ArrowMemoryPool::instance(),
&reader));
std::shared_ptr<arrow::Schema> file_schema;
THROW_ARROW_NOT_OK(reader->GetSchema(&file_schema));
ArrowColumnToCHColumn column_reader(
header, "Parquet",
format_settings.parquet.allow_missing_columns,
@ -553,29 +567,85 @@ struct DeltaLakeMetadataImpl
std::shared_ptr<arrow::Table> table;
THROW_ARROW_NOT_OK(reader->ReadTable(&table));
Chunk res = column_reader.arrowTableToCHChunk(table, reader->parquet_reader()->metadata()->num_rows());
const auto & res_columns = res.getColumns();
Chunk chunk = column_reader.arrowTableToCHChunk(table, reader->parquet_reader()->metadata()->num_rows());
auto res_block = header.cloneWithColumns(chunk.detachColumns());
res_block = Nested::flatten(res_block);
if (res_columns.size() != 2)
{
throw Exception(
ErrorCodes::INCORRECT_DATA,
"Unexpected number of columns: {} (having: {}, expected: {})",
res_columns.size(), res.dumpStructure(), header.dumpStructure());
}
const auto * nullable_path_column = assert_cast<const ColumnNullable *>(res_block.getByName("add.path").column.get());
const auto & path_column = assert_cast<const ColumnString &>(nullable_path_column->getNestedColumn());
const auto * nullable_schema_column = assert_cast<const ColumnNullable *>(res_block.getByName("metaData.schemaString").column.get());
const auto & schema_column = assert_cast<const ColumnString &>(nullable_schema_column->getNestedColumn());
auto partition_values_column_raw = res_block.getByName("add.partitionValues").column;
const auto & partition_values_column = assert_cast<const ColumnMap &>(*partition_values_column_raw);
const auto * tuple_column = assert_cast<const ColumnTuple *>(res_columns[0].get());
const auto & nullable_column = assert_cast<const ColumnNullable &>(tuple_column->getColumn(0));
const auto & path_column = assert_cast<const ColumnString &>(nullable_column.getNestedColumn());
for (size_t i = 0; i < path_column.size(); ++i)
{
const auto filename = String(path_column.getDataAt(i));
if (filename.empty())
const auto metadata = String(schema_column.getDataAt(i));
if (!metadata.empty())
{
Poco::JSON::Parser parser;
Poco::Dynamic::Var json = parser.parse(metadata);
const Poco::JSON::Object::Ptr & object = json.extract<Poco::JSON::Object::Ptr>();
auto current_schema = parseMetadata(object);
if (file_schema.empty())
{
file_schema = current_schema;
LOG_TEST(log, "Processed schema from checkpoint: {}", file_schema.toString());
}
else if (file_schema != current_schema)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Reading from files with different schema is not possible "
"({} is different from {})",
file_schema.toString(), current_schema.toString());
}
}
}
for (size_t i = 0; i < path_column.size(); ++i)
{
const auto path = String(path_column.getDataAt(i));
if (path.empty())
continue;
LOG_TEST(log, "Adding {}", filename);
const auto [_, inserted] = result.insert(std::filesystem::path(configuration->getPath()) / filename);
auto filename = fs::path(path).filename().string();
auto it = file_partition_columns.find(filename);
if (it == file_partition_columns.end())
{
Field map;
partition_values_column.get(i, map);
auto partition_values_map = map.safeGet<Map>();
if (!partition_values_map.empty())
{
auto & current_partition_columns = file_partition_columns[filename];
for (const auto & map_value : partition_values_map)
{
const auto tuple = map_value.safeGet<Tuple>();
const auto partition_name = tuple[0].safeGet<String>();
auto name_and_type = file_schema.tryGetByName(partition_name);
if (!name_and_type)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"No such column in schema: {} (schema: {})",
partition_name, file_schema.toString());
}
const auto value = tuple[1].safeGet<String>();
auto field = getFieldValue(value, name_and_type->type);
current_partition_columns.emplace_back(std::move(name_and_type.value()), std::move(field));
LOG_TEST(log, "Partition {} value is {} (for {})", partition_name, value, filename);
}
}
}
LOG_TEST(log, "Adding {}", path);
const auto [_, inserted] = result.insert(std::filesystem::path(configuration->getPath()) / path);
if (!inserted)
throw Exception(ErrorCodes::INCORRECT_DATA, "File already exists {}", filename);
throw Exception(ErrorCodes::INCORRECT_DATA, "File already exists {}", path);
}
return version;

View File

@ -41,6 +41,7 @@ public:
auto object_storage = base_configuration->createObjectStorage(context, /* is_readonly */true);
DataLakeMetadataPtr metadata;
NamesAndTypesList schema_from_metadata;
const bool use_schema_from_metadata = columns_.empty();
if (base_configuration->format == "auto")
base_configuration->format = "Parquet";
@ -50,8 +51,9 @@ public:
try
{
metadata = DataLakeMetadata::create(object_storage, base_configuration, context);
schema_from_metadata = metadata->getTableSchema();
configuration->setPaths(metadata->getDataFiles());
if (use_schema_from_metadata)
schema_from_metadata = metadata->getTableSchema();
}
catch (...)
{
@ -66,7 +68,7 @@ public:
return std::make_shared<IStorageDataLake<DataLakeMetadata>>(
base_configuration, std::move(metadata), configuration, object_storage,
context, table_id_,
columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_,
use_schema_from_metadata ? ColumnsDescription(schema_from_metadata) : columns_,
constraints_, comment_, format_settings_);
}

View File

@ -206,23 +206,25 @@ Chunk StorageObjectStorageSource::generate()
if (!partition_columns.empty() && chunk_size && chunk.hasColumns())
{
auto partition_values = partition_columns.find(filename);
for (const auto & [name_and_type, value] : partition_values->second)
if (partition_values != partition_columns.end())
{
if (!read_from_format_info.source_header.has(name_and_type.name))
continue;
for (const auto & [name_and_type, value] : partition_values->second)
{
if (!read_from_format_info.source_header.has(name_and_type.name))
continue;
const auto column_pos = read_from_format_info.source_header.getPositionByName(name_and_type.name);
auto partition_column = name_and_type.type->createColumnConst(chunk.getNumRows(), value)->convertToFullColumnIfConst();
const auto column_pos = read_from_format_info.source_header.getPositionByName(name_and_type.name);
auto partition_column = name_and_type.type->createColumnConst(chunk.getNumRows(), value)->convertToFullColumnIfConst();
/// This column is filled with default value now, remove it.
chunk.erase(column_pos);
/// This column is filled with default value now, remove it.
chunk.erase(column_pos);
/// Add correct values.
if (chunk.hasColumns())
chunk.addColumn(column_pos, std::move(partition_column));
else
chunk.addColumn(std::move(partition_column));
/// Add correct values.
if (column_pos < chunk.getNumColumns())
chunk.addColumn(column_pos, std::move(partition_column));
else
chunk.addColumn(std::move(partition_column));
}
}
}
return chunk;

View File

@ -35,7 +35,6 @@ void registerStorageFuzzJSON(StorageFactory & factory);
void registerStorageS3(StorageFactory & factory);
void registerStorageHudi(StorageFactory & factory);
void registerStorageS3Queue(StorageFactory & factory);
void registerStorageAzureQueue(StorageFactory & factory);
#if USE_PARQUET
void registerStorageDeltaLake(StorageFactory & factory);
@ -45,6 +44,10 @@ void registerStorageIceberg(StorageFactory & factory);
#endif
#endif
#if USE_AZURE_BLOB_STORAGE
void registerStorageAzureQueue(StorageFactory & factory);
#endif
#if USE_HDFS
#if USE_HIVE
void registerStorageHive(StorageFactory & factory);

View File

@ -104,6 +104,8 @@ def get_run_command(
return (
f"docker run --volume={builds_path}:/package_folder "
# For dmesg and sysctl
"--privileged "
f"{ci_logs_args}"
f"--volume={repo_path}/tests:/usr/share/clickhouse-test "
f"{volume_with_broken_test}"

View File

@ -2501,28 +2501,13 @@ def do_run_tests(jobs, test_suite: TestSuite):
)
],
)
while not future_seq.ready():
sleep(0.1)
if server_died.is_set():
sleep(5)
break
future_seq.wait()
while not future.ready():
sleep(0.1)
if server_died.is_set():
print("== Server died ==")
sleep(5)
break
print("future wait DONE")
future.wait()
finally:
print("pool.terminate")
pool.terminate()
print("pool.close")
pool.close()
print("pool.join")
pool.join()
print("pool.join DONE")
if not args.run_sequential_tests_in_parallel:
run_tests_array(

View File

@ -9,4 +9,21 @@
<key1>value1</key1>
</collection1>
</named_collections>
<remote_servers>
<replicated_nc_nodes_cluster>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>node_with_keeper</host>
<port>9000</port>
</replica>
<replica>
<host>node_with_keeper_2</host>
<port>9000</port>
</replica>
</shard>
<allow_distributed_ddl_queries>true</allow_distributed_ddl_queries>
</replicated_nc_nodes_cluster>
</remote_servers>
</clickhouse>

View File

@ -1,4 +1,9 @@
<clickhouse>
<profiles>
<default>
<ignore_on_cluster_for_replicated_named_collections_queries>0</ignore_on_cluster_for_replicated_named_collections_queries>
</default>
</profiles>
<users>
<default>
<password></password>

View File

@ -3,6 +3,8 @@ import pytest
import os
import time
from helpers.cluster import ClickHouseCluster
from contextlib import nullcontext as does_not_raise
from helpers.client import QueryRuntimeException
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
NAMED_COLLECTIONS_CONFIG = os.path.join(
@ -761,3 +763,32 @@ def test_keeper_storage(cluster):
check_dropped(node1)
check_dropped(node2)
@pytest.mark.parametrize(
"ignore, expected_raise",
[(True, does_not_raise()), (False, pytest.raises(QueryRuntimeException))],
)
def test_keeper_storage_remove_on_cluster(cluster, ignore, expected_raise):
node = cluster.instances["node_with_keeper"]
replace_in_users_config(
node,
"ignore_on_cluster_for_replicated_named_collections_queries>.",
f"ignore_on_cluster_for_replicated_named_collections_queries>{int(ignore)}",
)
node.query("SYSTEM RELOAD CONFIG")
with expected_raise:
node.query(
"DROP NAMED COLLECTION IF EXISTS test_nc ON CLUSTER `replicated_nc_nodes_cluster`"
)
node.query(
f"CREATE NAMED COLLECTION test_nc ON CLUSTER `replicated_nc_nodes_cluster` AS key1=1, key2=2 OVERRIDABLE"
)
node.query(
f"ALTER NAMED COLLECTION test_nc ON CLUSTER `replicated_nc_nodes_cluster` SET key2=3"
)
node.query(
f"DROP NAMED COLLECTION test_nc ON CLUSTER `replicated_nc_nodes_cluster`"
)

View File

@ -161,6 +161,9 @@ def test_parallel_replicas_custom_key_replicatedmergetree(
insert_data("test_table_for_rmt", row_num, all_nodes=False)
for node in nodes:
node.query("SYSTEM SYNC REPLICA test_table_for_rmt LIGHTWEIGHT")
expected_result = ""
for i in range(4):
expected_result += f"{i}\t250\n"

View File

@ -596,19 +596,116 @@ def test_partition_columns(started_cluster):
)
assert result == 1
# instance.query(
# f"""
# DROP TABLE IF EXISTS {TABLE_NAME};
# CREATE TABLE {TABLE_NAME} (a Int32, b String, c DateTime)
# ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')"""
# )
# assert (
# int(
# instance.query(
# f"SELECT count() FROM {TABLE_NAME} WHERE c != toDateTime('2000/01/05')"
# )
# )
# == num_rows - 1
# )
# instance.query(f"SELECT a, b, c, FROM {TABLE_NAME}")
# assert False
instance.query(
f"""
DROP TABLE IF EXISTS {TABLE_NAME};
CREATE TABLE {TABLE_NAME} (a Nullable(Int32), b Nullable(String), c Nullable(Date32), d Nullable(Int32), e Nullable(Bool))
ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')"""
)
assert (
"""1 test1 2000-01-01 1 false
2 test2 2000-01-02 2 false
3 test3 2000-01-03 3 false
4 test4 2000-01-04 4 false
5 test5 2000-01-05 5 false
6 test6 2000-01-06 6 false
7 test7 2000-01-07 7 false
8 test8 2000-01-08 8 false
9 test9 2000-01-09 9 false"""
== instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY b").strip()
)
assert (
int(
instance.query(
f"SELECT count() FROM {TABLE_NAME} WHERE c == toDateTime('2000/01/05')"
)
)
== 1
)
# Subset of columns should work.
instance.query(
f"""
DROP TABLE IF EXISTS {TABLE_NAME};
CREATE TABLE {TABLE_NAME} (b Nullable(String), c Nullable(Date32), d Nullable(Int32))
ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')"""
)
assert (
"""test1 2000-01-01 1
test2 2000-01-02 2
test3 2000-01-03 3
test4 2000-01-04 4
test5 2000-01-05 5
test6 2000-01-06 6
test7 2000-01-07 7
test8 2000-01-08 8
test9 2000-01-09 9"""
== instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY b").strip()
)
for i in range(num_rows + 1, 2 * num_rows + 1):
data = [
(
i,
"test" + str(i),
datetime.strptime(f"2000-01-{i}", "%Y-%m-%d"),
i,
False,
)
]
df = spark.createDataFrame(data=data, schema=schema)
df.printSchema()
df.write.mode("append").format("delta").partitionBy(partition_columns).save(
f"/{TABLE_NAME}"
)
files = upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "")
ok = False
for file in files:
if file.endswith("last_checkpoint"):
ok = True
assert ok
result = int(
instance.query(
f"""SELECT count()
FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')
"""
)
)
assert result == num_rows * 2
assert (
"""1 test1 2000-01-01 1 false
2 test2 2000-01-02 2 false
3 test3 2000-01-03 3 false
4 test4 2000-01-04 4 false
5 test5 2000-01-05 5 false
6 test6 2000-01-06 6 false
7 test7 2000-01-07 7 false
8 test8 2000-01-08 8 false
9 test9 2000-01-09 9 false
10 test10 2000-01-10 10 false
11 test11 2000-01-11 11 false
12 test12 2000-01-12 12 false
13 test13 2000-01-13 13 false
14 test14 2000-01-14 14 false
15 test15 2000-01-15 15 false
16 test16 2000-01-16 16 false
17 test17 2000-01-17 17 false
18 test18 2000-01-18 18 false"""
== instance.query(
f"""
SELECT * FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123') ORDER BY c
"""
).strip()
)
assert (
int(
instance.query(
f"SELECT count() FROM {TABLE_NAME} WHERE c == toDateTime('2000/01/15')"
)
)
== 1
)

View File

@ -15,7 +15,7 @@ tmp_path=$(mktemp "$CURDIR/01268_procfs_metrics.XXXXXX")
trap 'rm -f $tmp_path' EXIT
truncate -s1025 "$tmp_path"
$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM file('$tmp_path', 'LineAsString') FORMAT Null" |& grep -m1 -F -o -e OSReadChars
$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events --storage_file_read_method=pread -q "SELECT * FROM file('$tmp_path', 'LineAsString') FORMAT Null" |& grep -m1 -F -o -e OSReadChars
# NOTE: that OSCPUVirtualTimeMicroseconds is in microseconds, so 1e6 is not enough.
$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM numbers(1e8) FORMAT Null" |& grep -m1 -F -o -e OSCPUVirtualTimeMicroseconds
exit 0

View File

@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
FILE_NAME=test_02240.data
DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME

View File

@ -36,19 +36,10 @@ SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS N
3 3 3 33
\N \N \N \N
-- aliases defined in the join condition are valid
SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
1 42 \N \N \N 0
2 2 2 2 1 1
3 3 3 33 1 1
\N \N 4 42 \N 0
\N \N \N \N \N 1
SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
1 42 \N \N \N 0
2 2 2 2 1 1
3 3 3 33 1 1
\N \N 4 42 \N 0
\N \N \N \N \N 0
\N \N \N \N \N 0
-- FIXME(@vdimir) broken query formatting for the following queries:
-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
-- check for non-nullable columns for which `is null` is replaced with constant
SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST;
2 2 2 2

View File

@ -36,8 +36,9 @@ SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST;
SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST;
-- aliases defined in the join condition are valid
SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
-- FIXME(@vdimir) broken query formatting for the following queries:
-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
-- check for non-nullable columns for which `is null` is replaced with constant
SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST;

View File

@ -0,0 +1,6 @@
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS table_with_materialized;
CREATE TABLE table_with_materialized (col String MATERIALIZED 'A') ENGINE = Memory;
SELECT number FROM numbers(1) AS n, table_with_materialized;
DROP TABLE table_with_materialized;