diff --git a/cmake/dbms_glob_sources.cmake b/cmake/dbms_glob_sources.cmake index 01c4a8b16e9..fbe7f96cea3 100644 --- a/cmake/dbms_glob_sources.cmake +++ b/cmake/dbms_glob_sources.cmake @@ -4,10 +4,19 @@ macro(add_glob cur_list) endmacro() macro(add_headers_and_sources prefix common_path) - add_glob(${prefix}_headers ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h) - add_glob(${prefix}_sources ${common_path}/*.cpp ${common_path}/*.c ${common_path}/*.h) + add_glob(${prefix}_headers ${common_path}/*.h) + add_glob(${prefix}_sources ${common_path}/*.cpp ${common_path}/*.c) endmacro() macro(add_headers_only prefix common_path) - add_glob(${prefix}_headers ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h) + add_glob(${prefix}_headers ${common_path}/*.h) +endmacro() + +macro(extract_into_parent_list src_list dest_list) + list(REMOVE_ITEM ${src_list} ${ARGN}) + get_filename_component(__dir_name ${CMAKE_CURRENT_SOURCE_DIR} NAME) + foreach(file IN ITEMS ${ARGN}) + list(APPEND ${dest_list} ${__dir_name}/${file}) + endforeach() + set(${dest_list} "${${dest_list}}" PARENT_SCOPE) endmacro() diff --git a/cmake/target.cmake b/cmake/target.cmake index 204a67d4357..e4a2f060f1e 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -19,6 +19,19 @@ else () message (FATAL_ERROR "Platform ${CMAKE_SYSTEM_NAME} is not supported") endif () +# Since we always use toolchain files to generate hermetic builds, cmake will +# always think it's a cross-compilation, See +# https://cmake.org/cmake/help/latest/variable/CMAKE_CROSSCOMPILING.html +# +# This will slow down cmake configuration and compilation. For instance, LLVM +# will try to configure NATIVE LLVM targets with all tests enabled (You'll see +# Building native llvm-tblgen...). +# +# Here, we set it manually by checking the system name and processor. +if (${CMAKE_SYSTEM_NAME} STREQUAL ${CMAKE_HOST_SYSTEM_NAME} AND ${CMAKE_SYSTEM_PROCESSOR} STREQUAL ${CMAKE_HOST_SYSTEM_PROCESSOR}) + set (CMAKE_CROSSCOMPILING 0) +endif () + if (CMAKE_CROSSCOMPILING) if (OS_DARWIN) # FIXME: broken dependencies diff --git a/contrib/krb5 b/contrib/krb5 index 1d5c970e936..71b06c22760 160000 --- a/contrib/krb5 +++ b/contrib/krb5 @@ -1 +1 @@ -Subproject commit 1d5c970e9369f444caf81d1d06a231a6bad8581f +Subproject commit 71b06c2276009ae649c7703019f3b4605f66fd3d diff --git a/docker/test/integration/runner/compose/docker_compose_keeper.yml b/docker/test/integration/runner/compose/docker_compose_keeper.yml index 8524823ed87..91010c4aa83 100644 --- a/docker/test/integration/runner/compose/docker_compose_keeper.yml +++ b/docker/test/integration/runner/compose/docker_compose_keeper.yml @@ -20,6 +20,9 @@ services: - type: ${keeper_fs:-tmpfs} source: ${keeper_db_dir1:-} target: /var/lib/clickhouse-keeper + - type: ${keeper_fs:-tmpfs} + source: ${keeper_db_dir1:-} + target: /var/lib/clickhouse entrypoint: "${keeper_cmd_prefix:-clickhouse keeper} --config=/etc/clickhouse-keeper/keeper_config1.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log" cap_add: - SYS_PTRACE @@ -53,6 +56,9 @@ services: - type: ${keeper_fs:-tmpfs} source: ${keeper_db_dir2:-} target: /var/lib/clickhouse-keeper + - type: ${keeper_fs:-tmpfs} + source: ${keeper_db_dir1:-} + target: /var/lib/clickhouse entrypoint: "${keeper_cmd_prefix:-clickhouse keeper} --config=/etc/clickhouse-keeper/keeper_config2.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log" cap_add: - SYS_PTRACE @@ -86,6 +92,9 @@ services: - type: ${keeper_fs:-tmpfs} source: ${keeper_db_dir3:-} target: /var/lib/clickhouse-keeper + - type: ${keeper_fs:-tmpfs} + source: ${keeper_db_dir1:-} + target: /var/lib/clickhouse entrypoint: "${keeper_cmd_prefix:-clickhouse keeper} --config=/etc/clickhouse-keeper/keeper_config3.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log" cap_add: - SYS_PTRACE diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index fe6e8e15b0c..fc2933aa2cf 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -114,7 +114,11 @@ Example of disk configuration: ## Using local cache {#using-local-cache} -It is possible to configure local cache over disks in storage configuration starting from version 22.3. For versions 22.3 - 22.7 cache is supported only for `s3` disk type. For versions >= 22.8 cache is supported for any disk type: S3, Azure, Local, Encrypted, etc. Cache uses `LRU` cache policy. +It is possible to configure local cache over disks in storage configuration starting from version 22.3. +For versions 22.3 - 22.7 cache is supported only for `s3` disk type. For versions >= 22.8 cache is supported for any disk type: S3, Azure, Local, Encrypted, etc. +For versions >= 23.5 cache is supported only for remote disk types: S3, Azure, HDFS. +Cache uses `LRU` cache policy. + Example of configuration for versions later or equal to 22.8: diff --git a/docs/en/operations/system-tables/clusters.md b/docs/en/operations/system-tables/clusters.md index deb9a0aaeb3..2659f80e338 100644 --- a/docs/en/operations/system-tables/clusters.md +++ b/docs/en/operations/system-tables/clusters.md @@ -23,6 +23,7 @@ Columns: - `database_shard_name` ([String](../../sql-reference/data-types/string.md)) — The name of the `Replicated` database shard (for clusters that belong to a `Replicated` database). - `database_replica_name` ([String](../../sql-reference/data-types/string.md)) — The name of the `Replicated` database replica (for clusters that belong to a `Replicated` database). - `is_active` ([Nullable(UInt8)](../../sql-reference/data-types/int-uint.md)) — The status of the `Replicated` database replica (for clusters that belong to a `Replicated` database): 1 means "replica is online", 0 means "replica is offline", `NULL` means "unknown". +- `name` ([String](../../sql-reference/data-types/string.md)) - An alias to cluster. **Example** diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 31288b4aa01..21471bce8a9 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -390,7 +390,10 @@ static void transformFixedString(const UInt8 * src, UInt8 * dst, size_t size, UI static void transformUUID(const UUID & src_uuid, UUID & dst_uuid, UInt64 seed) { - const UInt128 & src = src_uuid.toUnderType(); + auto src_copy = src_uuid; + transformEndianness(src_copy); + + const UInt128 & src = src_copy.toUnderType(); UInt128 & dst = dst_uuid.toUnderType(); SipHash hash; @@ -400,8 +403,9 @@ static void transformUUID(const UUID & src_uuid, UUID & dst_uuid, UInt64 seed) /// Saving version and variant from an old UUID dst = hash.get128(); - dst.items[1] = (dst.items[1] & 0x1fffffffffffffffull) | (src.items[1] & 0xe000000000000000ull); - dst.items[0] = (dst.items[0] & 0xffffffffffff0fffull) | (src.items[0] & 0x000000000000f000ull); + const UInt64 trace[2] = {0x000000000000f000ull, 0xe000000000000000ull}; + UUIDHelpers::getLowBytes(dst_uuid) = (UUIDHelpers::getLowBytes(dst_uuid) & (0xffffffffffffffffull - trace[1])) | (UUIDHelpers::getLowBytes(src_uuid) & trace[1]); + UUIDHelpers::getHighBytes(dst_uuid) = (UUIDHelpers::getHighBytes(dst_uuid) & (0xffffffffffffffffull - trace[0])) | (UUIDHelpers::getHighBytes(src_uuid) & trace[0]); } class FixedStringModel : public IModel diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index bb7d9dfd4f7..c3f12caa4a4 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -49,6 +50,7 @@ namespace md5.update(type_storage_chars, strlen(type_storage_chars)); UUID result; memcpy(&result, md5.digest().data(), md5.digestLength()); + transformEndianness(result); return result; } diff --git a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index 82e4f1122a8..c41fb551a96 100644 --- a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -100,6 +100,17 @@ void AggregateFunctionBoundingRatioData::deserialize(ReadBuffer & buf) } } +inline void writeBinary(const AggregateFunctionBoundingRatioData::Point & p, WriteBuffer & buf) +{ + writePODBinary(p, buf); +} + +inline void readBinary(AggregateFunctionBoundingRatioData::Point & p, ReadBuffer & buf) +{ + readPODBinary(p, buf); +} + + class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper { private: diff --git a/src/AggregateFunctions/CMakeLists.txt b/src/AggregateFunctions/CMakeLists.txt index a45adde1a36..cf696da3127 100644 --- a/src/AggregateFunctions/CMakeLists.txt +++ b/src/AggregateFunctions/CMakeLists.txt @@ -1,28 +1,26 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") add_headers_and_sources(clickhouse_aggregate_functions .) -list(REMOVE_ITEM clickhouse_aggregate_functions_sources +extract_into_parent_list(clickhouse_aggregate_functions_sources dbms_sources IAggregateFunction.cpp AggregateFunctionFactory.cpp AggregateFunctionCombinatorFactory.cpp - AggregateFunctionCount.cpp AggregateFunctionState.cpp + AggregateFunctionCount.cpp parseAggregateFunctionParameters.cpp - FactoryHelpers.cpp ) - -list(REMOVE_ITEM clickhouse_aggregate_functions_headers +extract_into_parent_list(clickhouse_aggregate_functions_headers dbms_headers IAggregateFunction.h IAggregateFunctionCombinator.h AggregateFunctionFactory.h AggregateFunctionCombinatorFactory.h - AggregateFunctionCount.h AggregateFunctionState.h - parseAggregateFunctionParameters.h + AggregateFunctionCount.cpp FactoryHelpers.h + parseAggregateFunctionParameters.h ) -add_library(clickhouse_aggregate_functions ${clickhouse_aggregate_functions_sources}) +add_library(clickhouse_aggregate_functions ${clickhouse_aggregate_functions_headers} ${clickhouse_aggregate_functions_sources}) target_link_libraries(clickhouse_aggregate_functions PRIVATE dbms PUBLIC ch_contrib::cityhash) if(ENABLE_EXAMPLES) diff --git a/src/AggregateFunctions/QuantileTiming.h b/src/AggregateFunctions/QuantileTiming.h index 1d73453bc67..45fbf38258f 100644 --- a/src/AggregateFunctions/QuantileTiming.h +++ b/src/AggregateFunctions/QuantileTiming.h @@ -783,6 +783,16 @@ public: for (size_t i = 0; i < size; ++i) result[i] = std::numeric_limits::quiet_NaN(); } + + friend void writeBinary(const Kind & x, WriteBuffer & buf) + { + writePODBinary(x, buf); + } + + friend void readBinary(Kind & x, ReadBuffer & buf) + { + readPODBinary(x, buf); + } }; #undef SMALL_THRESHOLD diff --git a/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/src/AggregateFunctions/ReservoirSamplerDeterministic.h index 25d3b182654..daed0b98ca3 100644 --- a/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -276,3 +276,12 @@ private: return NanLikeValueConstructor>::getValue(); } }; + +namespace DB +{ +template +void readBinary(std::pair & x, ReadBuffer & buf) +{ + readPODBinary(x, buf); +} +} diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 90e76ef9b46..139e0a11474 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -563,8 +563,13 @@ void BackupsWorker::writeBackupEntries(BackupMutablePtr backup, BackupEntries && } }; - if (always_single_threaded || !backups_thread_pool->trySchedule([job] { job(true); })) + if (always_single_threaded) + { job(false); + continue; + } + + backups_thread_pool->scheduleOrThrowOnError([job] { job(true); }); } { @@ -854,8 +859,7 @@ void BackupsWorker::restoreTablesData(const OperationID & restore_id, BackupPtr } }; - if (!thread_pool.trySchedule([job] { job(true); })) - job(false); + thread_pool.scheduleOrThrowOnError([job] { job(true); }); } { diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 5600d382746..51acb077e17 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -49,6 +49,8 @@ else() add_definitions(-DENABLE_MULTITARGET_CODE=0) endif() +set(dbms_headers) +set(dbms_sources) add_subdirectory (Access) add_subdirectory (Backups) @@ -78,10 +80,6 @@ add_subdirectory (Daemon) add_subdirectory (Loggers) add_subdirectory (Formats) - -set(dbms_headers) -set(dbms_sources) - add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) add_headers_and_sources(clickhouse_common_io IO) @@ -151,47 +149,7 @@ else() message(STATUS "StorageFileLog is only supported on Linux") endif () -list (APPEND clickhouse_common_io_sources ${CONFIG_INCLUDE_PATH}/config_version.cpp) - -list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/FunctionsLogical.cpp Functions/indexHint.cpp) -list (APPEND dbms_headers Functions/IFunction.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/FunctionsLogical.h Functions/indexHint.h) - -list (APPEND dbms_sources - AggregateFunctions/IAggregateFunction.cpp - AggregateFunctions/AggregateFunctionFactory.cpp - AggregateFunctions/AggregateFunctionCombinatorFactory.cpp - AggregateFunctions/AggregateFunctionState.cpp - AggregateFunctions/AggregateFunctionCount.cpp - AggregateFunctions/parseAggregateFunctionParameters.cpp) -list (APPEND dbms_headers - AggregateFunctions/IAggregateFunction.h - AggregateFunctions/IAggregateFunctionCombinator.h - AggregateFunctions/AggregateFunctionFactory.h - AggregateFunctions/AggregateFunctionCombinatorFactory.h - AggregateFunctions/AggregateFunctionState.h - AggregateFunctions/AggregateFunctionCount.cpp - AggregateFunctions/FactoryHelpers.h - AggregateFunctions/parseAggregateFunctionParameters.h) - -list (APPEND dbms_sources - TableFunctions/ITableFunction.cpp - TableFunctions/TableFunctionView.cpp - TableFunctions/TableFunctionFactory.cpp) -list (APPEND dbms_headers - TableFunctions/ITableFunction.h - TableFunctions/TableFunctionView.h - TableFunctions/TableFunctionFactory.h) - -list (APPEND dbms_sources - Dictionaries/DictionaryFactory.cpp - Dictionaries/DictionarySourceFactory.cpp - Dictionaries/DictionaryStructure.cpp - Dictionaries/getDictionaryConfigurationFromAST.cpp) -list (APPEND dbms_headers - Dictionaries/DictionaryFactory.h - Dictionaries/DictionarySourceFactory.h - Dictionaries/DictionaryStructure.h - Dictionaries/getDictionaryConfigurationFromAST.h) +list(APPEND clickhouse_common_io_sources ${CONFIG_INCLUDE_PATH}/config_version.cpp) if (NOT ENABLE_SSL) list (REMOVE_ITEM clickhouse_common_io_sources Common/OpenSSLHelpers.cpp) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index fcd95e5c963..4ee6bb3d586 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -865,10 +865,14 @@ ColumnPtr ColumnNullable::getNestedColumnWithDefaultOnNull() const if (next_null_index != start) res->insertRangeFrom(*nested_column, start, next_null_index - start); - if (next_null_index < end) - res->insertDefault(); + size_t next_none_null_index = next_null_index; + while (next_none_null_index < end && null_map_data[next_none_null_index]) + ++next_none_null_index; - start = next_null_index + 1; + if (next_null_index != next_none_null_index) + res->insertManyDefaults(next_none_null_index - next_null_index); + + start = next_none_null_index; } return res; } diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 4f57e04ffea..070f8d5dcad 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -33,6 +33,7 @@ static struct InitFiu #define APPLY_FOR_FAILPOINTS(ONCE, REGULAR, PAUSEABLE_ONCE, PAUSEABLE) \ ONCE(replicated_merge_tree_commit_zk_fail_after_op) \ + REGULAR(use_delayed_remote_source) \ REGULAR(dummy_failpoint) \ PAUSEABLE_ONCE(dummy_pausable_failpoint_once) \ PAUSEABLE(dummy_pausable_failpoint) diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index 87107fa9f82..49ab875297c 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -2,9 +2,10 @@ #include #include +#include +#include #include #include -#include #include @@ -406,7 +407,7 @@ struct UInt128TrivialHash struct UUIDTrivialHash { - size_t operator()(DB::UUID x) const { return x.toUnderType().items[0]; } + size_t operator()(DB::UUID x) const { return DB::UUIDHelpers::getHighBytes(x); } }; struct UInt256Hash diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 3afc96816bd..b17eceda66f 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -227,8 +228,8 @@ bool TracingContext::parseTraceparentHeader(std::string_view traceparent, String ++data; this->trace_flags = unhex2(data); - this->trace_id.toUnderType().items[0] = trace_id_higher_64; - this->trace_id.toUnderType().items[1] = trace_id_lower_64; + UUIDHelpers::getHighBytes(this->trace_id) = trace_id_higher_64; + UUIDHelpers::getLowBytes(this->trace_id) = trace_id_lower_64; this->span_id = span_id_64; return true; } @@ -239,8 +240,8 @@ String TracingContext::composeTraceparentHeader() const // parent id. return fmt::format( "00-{:016x}{:016x}-{:016x}-{:02x}", - trace_id.toUnderType().items[0], - trace_id.toUnderType().items[1], + UUIDHelpers::getHighBytes(trace_id), + UUIDHelpers::getLowBytes(trace_id), span_id, // This cast is needed because fmt is being weird and complaining that // "mixing character types is not allowed". @@ -335,8 +336,8 @@ TracingContextHolder::TracingContextHolder( while (_parent_trace_context.trace_id == UUID()) { // Make sure the random generated trace_id is not 0 which is an invalid id. - _parent_trace_context.trace_id.toUnderType().items[0] = thread_local_rng(); - _parent_trace_context.trace_id.toUnderType().items[1] = thread_local_rng(); + UUIDHelpers::getHighBytes(_parent_trace_context.trace_id) = thread_local_rng(); + UUIDHelpers::getLowBytes(_parent_trace_context.trace_id) = thread_local_rng(); } _parent_trace_context.span_id = 0; } diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 533e73c7adb..5550b68c824 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -101,6 +101,12 @@ bool ShellCommand::tryWaitProcessWithTimeout(size_t timeout_in_seconds) out.close(); err.close(); + for (auto & [_, fd] : write_fds) + fd.close(); + + for (auto & [_, fd] : read_fds) + fd.close(); + return waitForPid(pid, timeout_in_seconds); } @@ -287,6 +293,12 @@ int ShellCommand::tryWait() out.close(); err.close(); + for (auto & [_, fd] : write_fds) + fd.close(); + + for (auto & [_, fd] : read_fds) + fd.close(); + LOG_TRACE(getLogger(), "Will wait for shell command pid {}", pid); int status = 0; diff --git a/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp b/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp index fda3a6ee1c8..15ccf612b74 100644 --- a/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp +++ b/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp @@ -41,7 +41,7 @@ TEST(Common, makeRegexpPatternFromGlobs) EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..2}{1..2}"), "f(1|2)(1|2)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..1}{1..1}"), "f(1)(1)"); EXPECT_EQ(makeRegexpPatternFromGlobs("f{0..0}{0..0}"), "f(0)(0)"); - EXPECT_EQ(makeRegexpPatternFromGlobs("file{1..5}"),"file(1|2|3|4|5)"); - EXPECT_EQ(makeRegexpPatternFromGlobs("file{1,2,3}"),"file(1|2|3)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("file{1..5}"), "file(1|2|3|4|5)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("file{1,2,3}"), "file(1|2|3)"); EXPECT_EQ(makeRegexpPatternFromGlobs("{1,2,3}blabla{a.x,b.x,c.x}smth[]_else{aa,bb}?*"), "(1|2|3)blabla(a\\.x|b\\.x|c\\.x)smth\\[\\]_else(aa|bb)[^/][^/]*"); } diff --git a/src/Core/MySQL/MySQLGtid.cpp b/src/Core/MySQL/MySQLGtid.cpp index 5cbc826d0d0..2b46c3d14ad 100644 --- a/src/Core/MySQL/MySQLGtid.cpp +++ b/src/Core/MySQL/MySQLGtid.cpp @@ -174,8 +174,8 @@ String GTIDSets::toPayload() const for (const auto & set : sets) { // MySQL UUID is big-endian. - writeBinaryBigEndian(set.uuid.toUnderType().items[0], buffer); - writeBinaryBigEndian(set.uuid.toUnderType().items[1], buffer); + writeBinaryBigEndian(UUIDHelpers::getHighBytes(set.uuid), buffer); + writeBinaryBigEndian(UUIDHelpers::getLowBytes(set.uuid), buffer); UInt64 intervals_size = set.intervals.size(); buffer.write(reinterpret_cast(&intervals_size), 8); diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index dcb407daa90..3042ae44a3d 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -940,13 +940,8 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(&commit_flag), 1); // MySQL UUID is big-endian. - UInt64 high = 0UL; - UInt64 low = 0UL; - readBigEndianStrict(payload, reinterpret_cast(&low), 8); - gtid.uuid.toUnderType().items[0] = low; - - readBigEndianStrict(payload, reinterpret_cast(&high), 8); - gtid.uuid.toUnderType().items[1] = high; + readBinaryBigEndian(UUIDHelpers::getHighBytes(gtid.uuid), payload); + readBinaryBigEndian(UUIDHelpers::getLowBytes(gtid.uuid), payload); payload.readStrict(reinterpret_cast(>id.seq_no), 8); diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index e4287e8769b..1584dbd42ac 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -33,8 +33,10 @@ namespace MySQLReplication inline void readBigEndianStrict(ReadBuffer & payload, char * to, size_t n) { payload.readStrict(to, n); +#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ char *start = to, *end = to + n; std::reverse(start, end); +#endif } inline void readTimeFractionalPart(ReadBuffer & payload, UInt32 & factional, UInt16 meta) diff --git a/src/Core/UUID.cpp b/src/Core/UUID.cpp index ef1e10f5063..10350964f50 100644 --- a/src/Core/UUID.cpp +++ b/src/Core/UUID.cpp @@ -9,10 +9,11 @@ namespace UUIDHelpers { UUID generateV4() { - UInt128 res{thread_local_rng(), thread_local_rng()}; - res.items[0] = (res.items[0] & 0xffffffffffff0fffull) | 0x0000000000004000ull; - res.items[1] = (res.items[1] & 0x3fffffffffffffffull) | 0x8000000000000000ull; - return UUID{res}; + UUID uuid; + getHighBytes(uuid) = (thread_local_rng() & 0xffffffffffff0fffull) | 0x0000000000004000ull; + getLowBytes(uuid) = (thread_local_rng() & 0x3fffffffffffffffull) | 0x8000000000000000ull; + + return uuid; } } diff --git a/src/Core/UUID.h b/src/Core/UUID.h index a24dcebdc9e..2bdefe9d3fc 100644 --- a/src/Core/UUID.h +++ b/src/Core/UUID.h @@ -2,6 +2,59 @@ #include +/** + * Implementation Details + * ^^^^^^^^^^^^^^^^^^^^^^ + * The underlying implementation for a UUID has it represented as a 128-bit unsigned integer. Underlying this, a wide + * integer with a 64-bit unsigned integer as its base is utilized. This wide integer can be interfaced with as an array + * to access different components of the base. For example, on a Little Endian platform, accessing at index 0 will give + * you the 8 higher bytes, and index 1 will give you the 8 lower bytes. On a Big Endian platform, this is reversed where + * index 0 will give you the 8 lower bytes, and index 1 will give you the 8 higher bytes. + * + * uuid.toUnderType().items[0] + * + * // uint64_t uint64_t + * // [xxxxxxxx] [ ] + * + * uuid.toUnderType().items[1] + * + * // uint64_t uint64_t + * // [ ] [xxxxxxxx] + * + * The way that data is stored in the underlying wide integer treats the data as two 64-bit chunks sequenced in the + * array. On a Little Endian platform, this results in the following layout + * + * // Suppose uuid contains 61f0c404-5cb3-11e7-907b-a6006ad3dba0 + * + * uuid.toUnderType().items[0] + * + * // uint64_t as HEX + * // [E7 11 B3 5C 04 C4 F0 61] [A0 DB D3 6A 00 A6 7B 90] + * // ^^^^^^^^^^^^^^^^^^^^^^^ + * + * uuid.toUnderType().items[1] + * + * // uint64_t as HEX + * // [E7 11 B3 5C 04 C4 F0 61] [A0 DB D3 6A 00 A6 7B 90] + * // ^^^^^^^^^^^^^^^^^^^^^^^ + * + * while on a Big Endian platform this would be + * + * // Suppose uuid contains 61f0c404-5cb3-11e7-907b-a6006ad3dba0 + * + * uuid.toUnderType().items[0] + * + * // uint64_t as HEX + * // [90 7B A6 00 6A D3 DB A0] [61 F0 C4 04 5C B3 11 E7] + * // ^^^^^^^^^^^^^^^^^^^^^^^ + * + * uuid.toUnderType().items[1] + * + * // uint64_t as HEX + * // [90 7B A6 00 6A D3 DB A0] [61 F0 C4 04 5C B3 11 E7] + * // ^^^^^^^^^^^^^^^^^^^^^^^ +*/ + namespace DB { @@ -11,6 +64,29 @@ namespace UUIDHelpers /// Generate random UUID. UUID generateV4(); + constexpr size_t HighBytes = (std::endian::native == std::endian::little) ? 0 : 1; + constexpr size_t LowBytes = (std::endian::native == std::endian::little) ? 1 : 0; + + inline uint64_t getHighBytes(const UUID & uuid) + { + return uuid.toUnderType().items[HighBytes]; + } + + inline uint64_t & getHighBytes(UUID & uuid) + { + return uuid.toUnderType().items[HighBytes]; + } + + inline uint64_t getLowBytes(const UUID & uuid) + { + return uuid.toUnderType().items[LowBytes]; + } + + inline uint64_t & getLowBytes(UUID & uuid) + { + return uuid.toUnderType().items[LowBytes]; + } + const UUID Nil{}; } diff --git a/src/DataTypes/Serializations/SerializationUUID.cpp b/src/DataTypes/Serializations/SerializationUUID.cpp index 76be273d7dc..93658fd05a3 100644 --- a/src/DataTypes/Serializations/SerializationUUID.cpp +++ b/src/DataTypes/Serializations/SerializationUUID.cpp @@ -111,25 +111,25 @@ void SerializationUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr, void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { UUID x = field.get(); - writeBinary(x, ostr); + writeBinaryLittleEndian(x, ostr); } void SerializationUUID::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { UUID x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); field = NearestFieldType(x); } void SerializationUUID::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeBinary(assert_cast &>(column).getData()[row_num], ostr); + writeBinaryLittleEndian(assert_cast &>(column).getData()[row_num], ostr); } void SerializationUUID::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { UUID x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); assert_cast &>(column).getData().push_back(x); } diff --git a/src/Dictionaries/CMakeLists.txt b/src/Dictionaries/CMakeLists.txt index c9dd554a6f1..90d2fedceac 100644 --- a/src/Dictionaries/CMakeLists.txt +++ b/src/Dictionaries/CMakeLists.txt @@ -16,10 +16,20 @@ if (OMIT_HEAVY_DEBUG_SYMBOLS) PROPERTIES COMPILE_FLAGS -g0) endif() -list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp DictionarySourceFactory.cpp DictionaryStructure.cpp getDictionaryConfigurationFromAST.cpp) -list(REMOVE_ITEM clickhouse_dictionaries_headers DictionaryFactory.h DictionarySourceFactory.h DictionaryStructure.h getDictionaryConfigurationFromAST.h) +extract_into_parent_list(clickhouse_dictionaries_sources dbms_sources + DictionaryFactory.cpp + DictionarySourceFactory.cpp + DictionaryStructure.cpp + getDictionaryConfigurationFromAST.cpp +) +extract_into_parent_list(clickhouse_dictionaries_headers dbms_headers + DictionaryFactory.h + DictionarySourceFactory.h + DictionaryStructure.h + getDictionaryConfigurationFromAST.h +) -add_library(clickhouse_dictionaries ${clickhouse_dictionaries_sources}) +add_library(clickhouse_dictionaries ${clickhouse_dictionaries_headers} ${clickhouse_dictionaries_sources}) target_link_libraries(clickhouse_dictionaries PRIVATE diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 06436488050..f798e7a549c 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -3,10 +3,22 @@ add_subdirectory(divide) include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") add_headers_and_sources(clickhouse_functions .) -list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp FunctionHelpers.cpp extractTimeZoneFromFunctionArguments.cpp FunctionsLogical.cpp) -list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h FunctionHelpers.h extractTimeZoneFromFunctionArguments.h FunctionsLogical.h) +extract_into_parent_list(clickhouse_functions_sources dbms_sources + IFunction.cpp + FunctionFactory.cpp + FunctionHelpers.cpp + extractTimeZoneFromFunctionArguments.cpp + FunctionsLogical.cpp +) +extract_into_parent_list(clickhouse_functions_headers dbms_headers + IFunction.h + FunctionFactory.h + FunctionHelpers.h + extractTimeZoneFromFunctionArguments.h + FunctionsLogical.h +) -add_library(clickhouse_functions_obj OBJECT ${clickhouse_functions_sources}) +add_library(clickhouse_functions_obj OBJECT ${clickhouse_functions_headers} ${clickhouse_functions_sources}) list (APPEND OBJECT_LIBS $) diff --git a/src/Functions/FunctionsBinaryRepresentation.cpp b/src/Functions/FunctionsBinaryRepresentation.cpp index c3a8f51ee4b..0f3f8be96a7 100644 --- a/src/Functions/FunctionsBinaryRepresentation.cpp +++ b/src/Functions/FunctionsBinaryRepresentation.cpp @@ -507,8 +507,8 @@ public: // use executeOnUInt instead of using executeOneString // because the latter one outputs the string in the memory order - Impl::executeOneUIntOrInt(uuid[i].toUnderType().items[0], end, false, false); - Impl::executeOneUIntOrInt(uuid[i].toUnderType().items[1], end, false, true); + Impl::executeOneUIntOrInt(UUIDHelpers::getHighBytes(uuid[i]), end, false, false); + Impl::executeOneUIntOrInt(UUIDHelpers::getLowBytes(uuid[i]), end, false, true); pos += end - begin; out_offsets[i] = pos; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 79d17d8ac98..00e2ebcda43 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -203,18 +203,15 @@ struct ConvertImpl } } - if constexpr (std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v && std::is_same_v) { - static_assert(std::is_same_v, "UInt128 and UUID types must be same"); - if constexpr (std::endian::native == std::endian::little) - { - vec_to[i].items[1] = vec_from[i].toUnderType().items[0]; - vec_to[i].items[0] = vec_from[i].toUnderType().items[1]; - } - else - { - vec_to[i] = vec_from[i].toUnderType(); - } + static_assert( + std::is_same_v, + "UInt128 and UUID types must be same"); + + vec_to[i].items[1] = vec_from[i].toUnderType().items[0]; + vec_to[i].items[0] = vec_from[i].toUnderType().items[1]; + continue; } diff --git a/src/Functions/currentDatabase.cpp b/src/Functions/currentDatabase.cpp index b7fd6c4fecc..954899c3c2b 100644 --- a/src/Functions/currentDatabase.cpp +++ b/src/Functions/currentDatabase.cpp @@ -55,6 +55,7 @@ REGISTER_FUNCTION(CurrentDatabase) { factory.registerFunction(); factory.registerAlias("DATABASE", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("SCHEMA", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive); factory.registerAlias("current_database", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/generateUUIDv4.cpp b/src/Functions/generateUUIDv4.cpp index 1e89d9b5167..e70c2e17595 100644 --- a/src/Functions/generateUUIDv4.cpp +++ b/src/Functions/generateUUIDv4.cpp @@ -60,9 +60,8 @@ public: { /// https://tools.ietf.org/html/rfc4122#section-4.4 - UInt128 & impl = uuid.toUnderType(); - impl.items[0] = (impl.items[0] & 0xffffffffffff0fffull) | 0x0000000000004000ull; - impl.items[1] = (impl.items[1] & 0x3fffffffffffffffull) | 0x8000000000000000ull; + UUIDHelpers::getHighBytes(uuid) = (UUIDHelpers::getHighBytes(uuid) & 0xffffffffffff0fffull) | 0x0000000000004000ull; + UUIDHelpers::getLowBytes(uuid) = (UUIDHelpers::getLowBytes(uuid) & 0x3fffffffffffffffull) | 0x8000000000000000ull; } return col_res; diff --git a/src/Functions/reinterpretAs.cpp b/src/Functions/reinterpretAs.cpp index 36c944d16fd..9e86a70f877 100644 --- a/src/Functions/reinterpretAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -1,26 +1,27 @@ #include -#include #include +#include #include -#include -#include -#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include #include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include -#include +#include #include +#include #include @@ -261,8 +262,10 @@ public: memcpy(static_cast(&to[i]), static_cast(&from[i]), copy_size); else { - size_t offset_to = sizeof(To) > sizeof(From) ? sizeof(To) - sizeof(From) : 0; - memcpy(reinterpret_cast(&to[i]) + offset_to, static_cast(&from[i]), copy_size); + // Handle the cases of both 128-bit representation to 256-bit and 128-bit to 64-bit or lower. + const size_t offset_from = sizeof(From) > sizeof(To) ? sizeof(From) - sizeof(To) : 0; + const size_t offset_to = sizeof(To) > sizeof(From) ? sizeof(To) - sizeof(From) : 0; + memcpy(reinterpret_cast(&to[i]) + offset_to, reinterpret_cast(&from[i]) + offset_from, copy_size); } } @@ -315,7 +318,11 @@ private: { std::string_view data = src.getDataAt(i).toView(); +#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ memcpy(&data_to[offset], data.data(), std::min(n, data.size())); +#else + reverseMemcpy(&data_to[offset], data.data(), std::min(n, data.size())); +#endif offset += n; } } @@ -326,7 +333,11 @@ private: ColumnFixedString::Chars & data_to = dst.getChars(); data_to.resize(n * rows); +#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ memcpy(data_to.data(), src.getRawData().data(), data_to.size()); +#else + reverseMemcpy(data_to.data(), src.getRawData().data(), data_to.size()); +#endif } static void NO_INLINE executeToString(const IColumn & src, ColumnString & dst) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 72eaea20c69..bf3215d5823 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -53,36 +53,25 @@ UUID parseUUID(std::span src) { UUID uuid; const auto * src_ptr = src.data(); - auto * dst = reinterpret_cast(&uuid); const auto size = src.size(); #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ - const std::reverse_iterator dst_it(dst + sizeof(UUID)); + const std::reverse_iterator dst(reinterpret_cast(&uuid) + sizeof(UUID)); +#else + auto * dst = reinterpret_cast(&uuid); #endif if (size == 36) { -#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ - parseHex<4>(src_ptr, dst_it + 8); - parseHex<2>(src_ptr + 9, dst_it + 12); - parseHex<2>(src_ptr + 14, dst_it + 14); - parseHex<2>(src_ptr + 19, dst_it); - parseHex<6>(src_ptr + 24, dst_it + 2); -#else - parseHex<4>(src_ptr, dst); - parseHex<2>(src_ptr + 9, dst + 4); - parseHex<2>(src_ptr + 14, dst + 6); - parseHex<2>(src_ptr + 19, dst + 8); - parseHex<6>(src_ptr + 24, dst + 10); -#endif + parseHex<4>(src_ptr, dst + 8); + parseHex<2>(src_ptr + 9, dst + 12); + parseHex<2>(src_ptr + 14, dst + 14); + parseHex<2>(src_ptr + 19, dst); + parseHex<6>(src_ptr + 24, dst + 2); } else if (size == 32) { -#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ - parseHex<8>(src_ptr, dst_it + 8); - parseHex<8>(src_ptr + 16, dst_it); -#else - parseHex<16>(src_ptr, dst); -#endif + parseHex<8>(src_ptr, dst + 8); + parseHex<8>(src_ptr + 16, dst); } else throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Unexpected length when trying to parse UUID ({})", size); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index da001a6f975..f99c78fdf16 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -116,6 +116,13 @@ inline void readPODBinary(T & x, ReadBuffer & buf) buf.readStrict(reinterpret_cast(&x), sizeof(x)); /// NOLINT } +inline void readUUIDBinary(UUID & x, ReadBuffer & buf) +{ + auto & uuid = x.toUnderType(); + readPODBinary(uuid.items[0], buf); + readPODBinary(uuid.items[1], buf); +} + template inline void readIntBinary(T & x, ReadBuffer & buf) { @@ -1106,16 +1113,26 @@ inline void readBinary(Decimal64 & x, ReadBuffer & buf) { readPODBinary(x, buf); inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); } inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(UUID & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(IPv4 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(IPv6 & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(UUID & x, ReadBuffer & buf) +{ + readUUIDBinary(x, buf); +} + +inline void readBinary(CityHash_v1_0_2::uint128 & x, ReadBuffer & buf) +{ + readPODBinary(x.low64, buf); + readPODBinary(x.high64, buf); +} + inline void readBinary(StackTrace::FramePointers & x, ReadBuffer & buf) { readPODBinary(x, buf); } template inline void readBinaryEndian(T & x, ReadBuffer & buf) { - readPODBinary(x, buf); + readBinary(x, buf); transformEndianness(x); } diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index e2401ffb958..34eabe55d7f 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -23,30 +23,23 @@ void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes) std::array formatUUID(const UUID & uuid) { std::array dst; - const auto * src_ptr = reinterpret_cast(&uuid); auto * dst_ptr = dst.data(); + #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ - const std::reverse_iterator src_it(src_ptr + 16); - formatHex(src_it + 8, dst_ptr, 4); - dst[8] = '-'; - formatHex(src_it + 12, dst_ptr + 9, 2); - dst[13] = '-'; - formatHex(src_it + 14, dst_ptr + 14, 2); - dst[18] = '-'; - formatHex(src_it, dst_ptr + 19, 2); - dst[23] = '-'; - formatHex(src_it + 2, dst_ptr + 24, 6); + const auto * src_ptr = reinterpret_cast(&uuid); + const std::reverse_iterator src(src_ptr + 16); #else - formatHex(src_ptr, dst_ptr, 4); - dst[8] = '-'; - formatHex(src_ptr + 4, dst_ptr + 9, 2); - dst[13] = '-'; - formatHex(src_ptr + 6, dst_ptr + 14, 2); - dst[18] = '-'; - formatHex(src_ptr + 8, dst_ptr + 19, 2); - dst[23] = '-'; - formatHex(src_ptr + 10, dst_ptr + 24, 6); + const auto * src = reinterpret_cast(&uuid); #endif + formatHex(src + 8, dst_ptr, 4); + dst[8] = '-'; + formatHex(src + 12, dst_ptr + 9, 2); + dst[13] = '-'; + formatHex(src + 14, dst_ptr + 14, 2); + dst[18] = '-'; + formatHex(src, dst_ptr + 19, 2); + dst[23] = '-'; + formatHex(src + 2, dst_ptr + 24, 6); return dst; } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index d7a572837d7..57337e7bb96 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -88,6 +88,13 @@ inline void writePODBinary(const T & x, WriteBuffer & buf) buf.write(reinterpret_cast(&x), sizeof(x)); /// NOLINT } +inline void writeUUIDBinary(const UUID & x, WriteBuffer & buf) +{ + const auto & uuid = x.toUnderType(); + writePODBinary(uuid.items[0], buf); + writePODBinary(uuid.items[1], buf); +} + template inline void writeIntBinary(const T & x, WriteBuffer & buf) { @@ -882,10 +889,20 @@ inline void writeBinary(const Decimal128 & x, WriteBuffer & buf) { writePODBinar inline void writeBinary(const Decimal256 & x, WriteBuffer & buf) { writePODBinary(x.value, buf); } inline void writeBinary(const LocalDate & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const LocalDateTime & x, WriteBuffer & buf) { writePODBinary(x, buf); } -inline void writeBinary(const UUID & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const IPv4 & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const IPv6 & x, WriteBuffer & buf) { writePODBinary(x, buf); } +inline void writeBinary(const UUID & x, WriteBuffer & buf) +{ + writeUUIDBinary(x, buf); +} + +inline void writeBinary(const CityHash_v1_0_2::uint128 & x, WriteBuffer & buf) +{ + writePODBinary(x.low64, buf); + writePODBinary(x.high64, buf); +} + inline void writeBinary(const StackTrace::FramePointers & x, WriteBuffer & buf) { writePODBinary(x, buf); } /// Methods for outputting the value in text form for a tab-separated format. @@ -1208,7 +1225,7 @@ template inline void writeBinaryEndian(T x, WriteBuffer & buf) { transformEndianness(x); - writePODBinary(x, buf); + writeBinary(x, buf); } template diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 953e38d56cd..da716d57f88 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -35,6 +36,11 @@ namespace ErrorCodes extern const int ALL_REPLICAS_ARE_STALE; } +namespace FailPoints +{ + extern const char use_delayed_remote_source[]; +} + namespace ClusterProxy { @@ -134,6 +140,12 @@ void SelectStreamFactory::createForShard( const auto & settings = context->getSettingsRef(); + fiu_do_on(FailPoints::use_delayed_remote_source, + { + emplace_remote_stream(/*lazy=*/true, /*local_delay=*/999999); + return; + }); + if (settings.prefer_localhost_replica && shard_info.isLocal()) { StoragePtr main_table_storage; diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 85e0c5a0ae7..3052c688e5f 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -33,7 +32,13 @@ class ConcurrentHashJoin : public IJoin { public: - explicit ConcurrentHashJoin(ContextPtr context_, std::shared_ptr table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_ = false); + explicit ConcurrentHashJoin( + ContextPtr context_, + std::shared_ptr table_join_, + size_t slots_, + const Block & right_sample_block, + bool any_take_last_row_ = false); + ~ConcurrentHashJoin() override = default; std::string getName() const override { return "ConcurrentHashJoin"; } @@ -67,7 +72,6 @@ private: IColumn::Selector selectDispatchBlock(const Strings & key_columns_names, const Block & from_block); Blocks dispatchBlock(const Strings & key_columns_names, const Block & from_block); - }; } diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index d734e6bc149..50a6ef437bf 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -308,7 +308,7 @@ private: static inline size_t getFirstLevelIdx(const UUID & uuid) { - return uuid.toUnderType().items[0] >> (64 - bits_for_first_level); + return UUIDHelpers::getHighBytes(uuid) >> (64 - bits_for_first_level); } void dropTableDataTask(); diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index a46f0018611..ac2a0058c70 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -326,8 +326,8 @@ static void insertUUID(IColumn & column, DataTypePtr type, const char * value, s throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack UUID into column with type {}.", type->getName()); ReadBufferFromMemory buf(value, size); UUID uuid; - readBinaryBigEndian(uuid.toUnderType().items[0], buf); - readBinaryBigEndian(uuid.toUnderType().items[1], buf); + readBinaryBigEndian(UUIDHelpers::getHighBytes(uuid), buf); + readBinaryBigEndian(UUIDHelpers::getLowBytes(uuid), buf); assert_cast(column).insertValue(uuid); } diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index 9c601492217..12bbd35b77b 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -270,8 +270,8 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr { WriteBufferFromOwnString buf; UUID value = uuid_column.getElement(row_num); - writeBinaryBigEndian(value.toUnderType().items[0], buf); - writeBinaryBigEndian(value.toUnderType().items[1], buf); + writeBinaryBigEndian(UUIDHelpers::getHighBytes(value), buf); + writeBinaryBigEndian(UUIDHelpers::getLowBytes(value), buf); std::string_view uuid_ext = buf.stringView(); packer.pack_ext(sizeof(UUID), int8_t(MsgPackExtensionTypes::UUIDType)); packer.pack_ext_body(uuid_ext.data(), static_cast(uuid_ext.size())); diff --git a/src/Processors/Sources/DelayedSource.cpp b/src/Processors/Sources/DelayedSource.cpp index ee7fd757949..a18ac802648 100644 --- a/src/Processors/Sources/DelayedSource.cpp +++ b/src/Processors/Sources/DelayedSource.cpp @@ -148,7 +148,9 @@ Processors DelayedSource::expandPipeline() inputs.emplace_back(outputs.front().getHeader(), this); /// Connect checks that header is same for ports. connect(*output, inputs.back()); - inputs.back().setNeeded(); + + if (output == main_output) + inputs.back().setNeeded(); } /// Executor will check that all processors are connected. diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 3ba9ebb11de..2625a7cdabb 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -439,11 +439,7 @@ namespace } if (!executor->pull(chunk)) - { - if (check_exit_code) - command->wait(); return {}; - } current_read_rows += chunk.getNumRows(); } @@ -466,6 +462,21 @@ namespace if (thread.joinable()) thread.join(); + if (check_exit_code) + { + if (process_pool) + { + bool valid_command + = configuration.read_fixed_number_of_rows && current_read_rows >= configuration.number_of_rows_to_read; + + // We can only wait for pooled commands when they are invalid. + if (!valid_command) + command->wait(); + } + else + command->wait(); + } + rethrowExceptionDuringSendDataIfNeeded(); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a0dc154f7ef..2c09ab9725c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1217,10 +1217,10 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( auto part_size_str = res.size_of_part ? formatReadableSizeWithBinarySuffix(*res.size_of_part) : "failed to calculate size"; LOG_ERROR(log, - "Detaching broken part {}{} (size: {}). " + "Detaching broken part {} (size: {}). " "If it happened after update, it is likely because of backward incompatibility. " "You need to resolve this manually", - getFullPathOnDisk(part_disk_ptr), part_name, part_size_str); + fs::path(getFullPathOnDisk(part_disk_ptr)) / part_name, part_size_str); }; try diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index f49f72c40a7..ddeaf69136a 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -84,15 +84,7 @@ namespace } void operator() (const UUID & x) const { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - auto tmp_x = x.toUnderType(); - char * start = reinterpret_cast(&tmp_x); - char * end = start + sizeof(tmp_x); - std::reverse(start, end); - operator()(tmp_x); -#else operator()(x.toUnderType()); -#endif } void operator() (const IPv4 & x) const { diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 293beca9c24..8d9b8f5d8d0 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -360,7 +360,7 @@ ColumnPtr fillColumnWithRandomData( auto column = ColumnUUID::create(); column->getData().resize(limit); /// NOTE This is slightly incorrect as random UUIDs should have fixed version 4. - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(UUID), rng); + fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(UUID), rng, true); return column; } case TypeIndex::Int8: diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 9c8a3860807..8cbe9f8d6e1 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -527,7 +527,7 @@ StorageS3Source::StorageS3Source( const String & bucket_, const String & version_id_, std::shared_ptr file_iterator_, - const size_t download_thread_num_, + const size_t max_parsing_threads_, std::optional query_info_) : ISource(info.source_header, false) , WithContext(context_) @@ -546,7 +546,7 @@ StorageS3Source::StorageS3Source( , query_info(std::move(query_info_)) , requested_virtual_columns(info.requested_virtual_columns) , file_iterator(file_iterator_) - , download_thread_num(download_thread_num_) + , max_parsing_threads(max_parsing_threads_) , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3Reader")) { @@ -573,9 +573,17 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() auto read_buf = createS3ReadBuffer(key_with_info.key, object_size); auto input_format = FormatFactory::instance().getInput( - format, *read_buf, sample_block, getContext(), max_block_size, - format_settings, std::nullopt, std::nullopt, - /* is_remote_fs */ true, compression_method); + format, + *read_buf, + sample_block, + getContext(), + max_block_size, + format_settings, + max_parsing_threads, + /* max_download_threads= */ std::nullopt, + /* is_remote_fs */ true, + compression_method); + if (query_info.has_value()) input_format->setQueryInfo(query_info.value(), getContext()); @@ -1035,7 +1043,9 @@ Pipe StorageS3::read( auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); - const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; + const size_t max_threads = local_context->getSettingsRef().max_threads; + const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams); + for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(std::make_shared( @@ -1051,7 +1061,7 @@ Pipe StorageS3::read( query_configuration.url.bucket, query_configuration.url.version_id, iterator_wrapper, - max_download_threads, + max_parsing_threads, query_info)); } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index f0486a8a0b0..aadceb1217b 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -130,7 +130,7 @@ public: const String & bucket, const String & version_id, std::shared_ptr file_iterator_, - size_t download_thread_num, + size_t max_parsing_threads, std::optional query_info); ~StorageS3Source() override; @@ -218,7 +218,7 @@ private: NamesAndTypesList requested_virtual_columns; std::shared_ptr file_iterator; - size_t download_thread_num = 1; + size_t max_parsing_threads = 1; Poco::Logger * log = &Poco::Logger::get("StorageS3Source"); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 617b421fa24..5638934cb6b 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -221,7 +221,7 @@ StorageURLSource::StorageURLSource( UInt64 max_block_size, const ConnectionTimeouts & timeouts, CompressionMethod compression_method, - size_t download_threads, + size_t max_parsing_threads, const SelectQueryInfo & query_info, const HTTPHeaderEntries & headers_, const URIParams & params, @@ -275,7 +275,6 @@ StorageURLSource::StorageURLSource( file_progress_callback(FileProgress(0, file_size)); } - // TODO: Pass max_parsing_threads and max_download_threads adjusted for num_streams. input_format = FormatFactory::instance().getInput( format, *read_buf, @@ -283,9 +282,9 @@ StorageURLSource::StorageURLSource( context, max_block_size, format_settings, - download_threads, - /*max_download_threads*/ std::nullopt, - /* is_remote_fs */ true, + max_parsing_threads, + /* max_download_threads= */ std::nullopt, + /* is_remote_fs= */ true, compression_method); input_format->setQueryInfo(query_info, context); @@ -706,8 +705,6 @@ Pipe IStorageURLBase::read( { auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size); - size_t max_download_threads = local_context->getSettingsRef().max_download_threads; - std::shared_ptr iterator_wrapper{nullptr}; bool is_url_with_globs = urlWithGlobs(uri); size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; @@ -754,7 +751,9 @@ Pipe IStorageURLBase::read( Pipes pipes; pipes.reserve(num_streams); - size_t download_threads = num_streams >= max_download_threads ? 1 : (max_download_threads / num_streams); + const size_t max_threads = local_context->getSettingsRef().max_threads; + const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams); + for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(std::make_shared( @@ -775,7 +774,7 @@ Pipe IStorageURLBase::read( max_block_size, getHTTPTimeouts(local_context), compression_method, - download_threads, + max_parsing_threads, query_info, headers, params, @@ -793,7 +792,7 @@ Pipe StorageURLWithFailover::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - size_t /*num_streams*/) + size_t num_streams) { auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size); @@ -807,6 +806,9 @@ Pipe StorageURLWithFailover::read( auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + const size_t max_threads = local_context->getSettingsRef().max_threads; + const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams); + auto pipe = Pipe(std::make_shared( read_from_format_info, iterator_wrapper, @@ -819,7 +821,7 @@ Pipe StorageURLWithFailover::read( max_block_size, getHTTPTimeouts(local_context), compression_method, - local_context->getSettingsRef().max_download_threads, + max_parsing_threads, query_info, headers, params)); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 140f3d42f7b..9b74e3236ca 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -170,7 +170,7 @@ public: UInt64 max_block_size, const ConnectionTimeouts & timeouts, CompressionMethod compression_method, - size_t download_threads, + size_t max_parsing_threads, const SelectQueryInfo & query_info, const HTTPHeaderEntries & headers_ = {}, const URIParams & params = {}, diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index f4ef52d7605..39a61f22b89 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -32,6 +32,12 @@ NamesAndTypesList StorageSystemClusters::getNamesAndTypes() }; } +NamesAndAliases StorageSystemClusters::getNamesAndAliases() +{ + return { + {"name", std::make_shared(), "cluster"}, + }; +} void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index 9aa1a6a5183..071ad423b89 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -22,6 +22,8 @@ public: static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases(); + protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; using NameAndCluster = std::pair>; diff --git a/src/TableFunctions/CMakeLists.txt b/src/TableFunctions/CMakeLists.txt index c9e5c66fe4a..b02a0e79f9c 100644 --- a/src/TableFunctions/CMakeLists.txt +++ b/src/TableFunctions/CMakeLists.txt @@ -6,16 +6,18 @@ if (TARGET ch_contrib::hivemetastore) add_headers_and_sources(clickhouse_table_functions Hive) endif () -list(REMOVE_ITEM clickhouse_table_functions_sources +extract_into_parent_list(clickhouse_table_functions_sources dbms_sources ITableFunction.cpp TableFunctionView.cpp - TableFunctionFactory.cpp) -list(REMOVE_ITEM clickhouse_table_functions_headers + TableFunctionFactory.cpp +) +extract_into_parent_list(clickhouse_table_functions_headers dbms_headers ITableFunction.h TableFunctionView.h - TableFunctionFactory.h) + TableFunctionFactory.h +) -add_library(clickhouse_table_functions ${clickhouse_table_functions_sources}) +add_library(clickhouse_table_functions ${clickhouse_table_functions_headers} ${clickhouse_table_functions_sources}) target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index ad7fd8632f2..8a7ca59011d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1467,7 +1467,7 @@ class TestSuite: else: raise Exception(f"Unknown file_extension: {filename}") - def parse_tags_from_line(line, comment_sign) -> set[str]: + def parse_tags_from_line(line, comment_sign) -> Set[str]: if not line.startswith(comment_sign): return set() tags_str = line[len(comment_sign) :].lstrip() # noqa: ignore E203 diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index f2bffe69495..a5c82e1a2c6 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -371,6 +371,22 @@ def test_mysql_replacement_query(started_cluster): "database()\ndefault\n", ] + # SELECT SCHEMA(). + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run( + """ + mysql --protocol tcp -h {host} -P {port} default -u default + --password=123 -e "select schema();" + """.format( + host=started_cluster.get_instance_ip("node"), port=server_port + ), + demux=True, + ) + assert code == 0 + assert stdout.decode().lower() in [ + "currentdatabase()\ndefault\n", + "schema()\ndefault\n", + ] + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run( """ mysql --protocol tcp -h {host} -P {port} default -u default diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index d1ae39ca378..d31457488c1 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -160,6 +160,10 @@ def get_ssl_context(cert_name): ) context.verify_mode = ssl.CERT_REQUIRED context.check_hostname = True + # Python 3.10 has removed many ciphers from the cipher suite. + # Hence based on https://github.com/urllib3/urllib3/issues/3100#issuecomment-1671106236 + # we are expanding the list of cipher suites. + context.set_ciphers("DEFAULT") return context diff --git a/tests/performance/encrypt_decrypt_empty_string_slow.xml b/tests/performance/encrypt_decrypt_empty_string_slow.xml deleted file mode 100644 index 4218f377c8c..00000000000 --- a/tests/performance/encrypt_decrypt_empty_string_slow.xml +++ /dev/null @@ -1,55 +0,0 @@ - - - - - - func - - - encrypt('aes-128-cbc', materialize(plaintext), key16, iv16) - encrypt('aes-128-ecb', materialize(plaintext), key16) - encrypt('aes-128-gcm', materialize(plaintext), key16, iv12, 'aadaadaadaad') - - encrypt('aes-192-cbc', materialize(plaintext), key24, iv16) - encrypt('aes-192-ecb', materialize(plaintext), key24) - encrypt('aes-192-gcm', materialize(plaintext), key24, iv12, 'aadaadaadaad') - - encrypt('aes-256-cbc', materialize(plaintext), key32, iv16) - encrypt('aes-256-ecb', materialize(plaintext), key32) - encrypt('aes-256-gcm', materialize(plaintext), key32, iv12, 'aadaadaadaad') - - - decrypt('aes-128-cbc', encrypt('aes-128-cbc', materialize(plaintext), key16, iv16), key16, iv16) - decrypt('aes-128-ecb', encrypt('aes-128-ecb', materialize(plaintext), key16), key16) - decrypt('aes-128-gcm', encrypt('aes-128-gcm', materialize(plaintext), key16, iv12, 'aadaadaadaad'), key16, iv12, 'aadaadaadaad') - - decrypt('aes-192-cbc', encrypt('aes-192-cbc', materialize(plaintext), key24, iv16), key24, iv16) - decrypt('aes-192-ecb', encrypt('aes-192-ecb', materialize(plaintext), key24), key24) - decrypt('aes-192-gcm', encrypt('aes-192-gcm', materialize(plaintext), key24, iv12, 'aadaadaadaad'), key24, iv12, 'aadaadaadaad') - - decrypt('aes-256-cbc', encrypt('aes-256-cbc', materialize(plaintext), key32, iv16), key32, iv16) - decrypt('aes-256-ecb', encrypt('aes-256-ecb', materialize(plaintext), key32), key32) - decrypt('aes-256-gcm', encrypt('aes-256-gcm', materialize(plaintext), key32, iv12, 'aadaadaadaad'), key32, iv12, 'aadaadaadaad') - - - - - table - - numbers(2000000) - - - - plaintext - - '' - - - - - - WITH {plaintext} as plaintext, repeat('k', 32) as key32, substring(key32, 1, 24) as key24, substring(key32, 1, 16) as key16, repeat('iv', 8) as iv16, substring(iv16, 1, 12) as iv12 SELECT count() FROM {table} WHERE NOT ignore({func}) LIMIT 1 - - WITH {plaintext} as plaintext, repeat('k', 32) as key32, substring(key32, 1, 24) as key24, substring(key32, 1, 16) as key16, repeat('iv', 8) as iv16, substring(iv16, 1, 12) as iv12 SELECT count() FROM {table} WHERE NOT ignore({func}) - diff --git a/tests/queries/0_stateless/00304_http_external_data.sh b/tests/queries/0_stateless/00304_http_external_data.sh index def17bc5cd1..4a097249cca 100755 --- a/tests/queries/0_stateless/00304_http_external_data.sh +++ b/tests/queries/0_stateless/00304_http_external_data.sh @@ -6,10 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo -ne '1,Hello\n2,World\n' | ${CLICKHOUSE_CURL} -sSF 'file=@-' "${CLICKHOUSE_URL}&query=SELECT+*+FROM+file&file_format=CSV&file_types=UInt8,String"; echo -ne '1@Hello\n2@World\n' | ${CLICKHOUSE_CURL} -sSF 'file=@-' "${CLICKHOUSE_URL}&query=SELECT+*+FROM+file&file_format=CSV&file_types=UInt8,String&format_csv_delimiter=@"; - -# use big-endian version of binary data for s390x -if [[ $(uname -a | grep s390x) ]]; then - echo -ne '\x00\x00\x00\x01\x00\x00\x00\x02' | ${CLICKHOUSE_CURL} -sSF "tmp=@-" "${CLICKHOUSE_URL}&query=SELECT+*+FROM+tmp&tmp_structure=TaskID+UInt32&tmp_format=RowBinary"; -else - echo -ne '\x01\x00\x00\x00\x02\x00\x00\x00' | ${CLICKHOUSE_CURL} -sSF "tmp=@-" "${CLICKHOUSE_URL}&query=SELECT+*+FROM+tmp&tmp_structure=TaskID+UInt32&tmp_format=RowBinary"; -fi +echo -ne '\x01\x00\x00\x00\x02\x00\x00\x00' | ${CLICKHOUSE_CURL} -sSF "tmp=@-" "${CLICKHOUSE_URL}&query=SELECT+*+FROM+tmp&tmp_structure=TaskID+UInt32&tmp_format=RowBinary"; diff --git a/tests/queries/0_stateless/01472_obfuscator_uuid.sh b/tests/queries/0_stateless/01472_obfuscator_uuid.sh index 6654dcaad71..eae9c1e3081 100755 --- a/tests/queries/0_stateless/01472_obfuscator_uuid.sh +++ b/tests/queries/0_stateless/01472_obfuscator_uuid.sh @@ -4,9 +4,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS t_uuid" $CLICKHOUSE_CLIENT --query="CREATE TABLE t_uuid(Id UUID) ENGINE=MergeTree ORDER BY (Id)" $CLICKHOUSE_CLIENT --query="INSERT INTO t_uuid VALUES ('3f5ffba3-19ff-4f3d-8861-60ae6e1fc1aa'),('4bd62524-e33c-43e5-882d-f1d96cf5561e'),('7a8b45d2-c18b-4e8c-89eb-abf5bee88931'),('45bb7333-965b-4526-870e-4f941edb025b'),('a4e72d0e-f9fa-465e-8d9d-151b9ced94df'),('cb5818ab-83b5-48a8-94b0-5177e30176d9'),('701e8006-fc9f-4496-80ba-efa6817b917b'),('e0936acf-6e8f-42aa-8f56-d1363476eece'),('239bb790-5293-40df-92ae-472294b6e178'),('508d0e80-729f-4e3b-9336-4c5c8792f6be'),('94abef70-f2d6-4f7b-ad60-3889409f1dac'),('b6f1ec08-8473-4fa2-b134-73db040b0d82'),('7e54dcae-0bb4-4c4f-a636-54a705fb8b40'),('d1d258c2-a35f-4c00-abfa-8addbcbc5471'),('7c74fbd8-bf79-46ee-adfe-96271040a4f7'),('41e3a274-eea9-41d8-a128-de5a6658fcfd'),('a72dc048-f72f-470e-b0f9-60cfad6e1157'),('40634f4f-37bf-44e4-ac7c-6f024ad19990')" -$CLICKHOUSE_CLIENT --query="SELECT Id FROM t_uuid FORMAT TSV" > "${CLICKHOUSE_TMP}"/data.tsv +$CLICKHOUSE_CLIENT --query="SELECT Id FROM t_uuid ORDER BY (Id) FORMAT TSV" > "${CLICKHOUSE_TMP}"/data.tsv echo FROM RAW DATA && cat "${CLICKHOUSE_TMP}"/data.tsv echo TRANSFORMED TO && $CLICKHOUSE_OBFUSCATOR --structure "Id UUID" --input-format TSV --output-format TSV --seed dsrub < "${CLICKHOUSE_TMP}"/data.tsv 2>/dev/null diff --git a/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.reference b/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.reference index 682652152dc..de5a62159ef 100644 --- a/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.reference +++ b/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.reference @@ -13,6 +13,7 @@ SELECT covarPop(1, 1), covarSamp(1, 1), currentDatabase(), + currentDatabase(), dateDiff('DAY', toDate('2020-10-24'), toDate('2019-10-24')), exp(1), arrayFlatten([[1]]), diff --git a/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.sql b/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.sql index 9b35087182c..dda2e045e76 100644 --- a/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.sql +++ b/tests/queries/0_stateless/01705_normalize_case_insensitive_function_names.sql @@ -1 +1 @@ -EXPLAIN SYNTAX SELECT CAST(1 AS INT), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH('1'), CHARACTER_LENGTH('1'), COALESCE(1), CONCAT('1', '1'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), DATEDIFF('DAY', toDate('2020-10-24'), toDate('2019-10-24')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE('A'), LEAST(1), LENGTH('1'), LN(1), LOCATE('1', '1'), LOG(1), LOG10(1), LOG2(1), LOWER('A'), MAX(1), MID('123', 1, 1), MIN(1), MOD(1, 1), NOT(1), NOW(), NOW64(), NULLIF(1, 1), PI(), POSITION('123', '2'), POW(1, 1), POWER(1, 1), RAND(), REPLACE('1', '1', '2'), REVERSE('123'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR('123', 2), SUBSTRING('123', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE('A'), UPPER('A'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate('2020-10-24')), YEARWEEK(toDate('2020-10-24')) format TSVRaw; +EXPLAIN SYNTAX SELECT CAST(1 AS INT), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH('1'), CHARACTER_LENGTH('1'), COALESCE(1), CONCAT('1', '1'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), SCHEMA(), DATEDIFF('DAY', toDate('2020-10-24'), toDate('2019-10-24')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE('A'), LEAST(1), LENGTH('1'), LN(1), LOCATE('1', '1'), LOG(1), LOG10(1), LOG2(1), LOWER('A'), MAX(1), MID('123', 1, 1), MIN(1), MOD(1, 1), NOT(1), NOW(), NOW64(), NULLIF(1, 1), PI(), POSITION('123', '2'), POW(1, 1), POWER(1, 1), RAND(), REPLACE('1', '1', '2'), REVERSE('123'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR('123', 2), SUBSTRING('123', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE('A'), UPPER('A'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate('2020-10-24')), YEARWEEK(toDate('2020-10-24')) format TSVRaw; diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 0a58e41f9d5..67fbc894ed3 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -50,7 +50,8 @@ CREATE TABLE system.clusters `estimated_recovery_time` UInt32, `database_shard_name` String, `database_replica_name` String, - `is_active` Nullable(UInt8) + `is_active` Nullable(UInt8), + `name` String ) ENGINE = SystemClusters COMMENT 'SYSTEM TABLE is built on the fly.' diff --git a/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql b/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql index 6bf287a3d77..2aa23e2b70d 100644 --- a/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql +++ b/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql @@ -1,3 +1,5 @@ +DROP TABLE IF EXISTS events; + create table events ( `organisation_id` UUID, `session_id` UUID, diff --git a/tests/queries/0_stateless/02861_uuid_format_serialization.reference b/tests/queries/0_stateless/02861_uuid_format_serialization.reference new file mode 100644 index 00000000000..4c6b4cd21e8 Binary files /dev/null and b/tests/queries/0_stateless/02861_uuid_format_serialization.reference differ diff --git a/tests/queries/0_stateless/02861_uuid_format_serialization.sql b/tests/queries/0_stateless/02861_uuid_format_serialization.sql new file mode 100644 index 00000000000..e73ef2d5197 --- /dev/null +++ b/tests/queries/0_stateless/02861_uuid_format_serialization.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS t_uuid; +CREATE TABLE t_uuid (x UUID) ENGINE=MergeTree ORDER BY x; + +INSERT INTO t_uuid VALUES ('61f0c404-5cb3-11e7-907b-a6006ad3dba0'), ('992f6910-42b2-43cd-98bc-c812fbf9b683'), ('417ddc5d-e556-4d27-95dd-a34d84e46a50'); + +SELECT * FROM t_uuid ORDER BY x LIMIT 1 FORMAT RowBinary; +SELECT * FROM t_uuid ORDER BY x FORMAT RowBinary; + +DROP TABLE IF EXISTS t_uuid; diff --git a/tests/queries/0_stateless/02862_uuid_reinterpret_as_numeric.reference b/tests/queries/0_stateless/02862_uuid_reinterpret_as_numeric.reference new file mode 100644 index 00000000000..a874ad9ebc5 --- /dev/null +++ b/tests/queries/0_stateless/02862_uuid_reinterpret_as_numeric.reference @@ -0,0 +1,5 @@ +61f0c404-5cb3-11e7-907b-a6006ad3dba0 +403229640000000000 6.034192082918747e163 +-25 4583 1555239399 7057356139103719911 -148231516101255056243829344033567469081 192050850819683407219545263398200742375 +231 4583 1555239399 7057356139103719911 192050850819683407219545263398200742375 192050850819683407219545263398200742375 +00000000-5cb3-11e7-0000-000000000000 diff --git a/tests/queries/0_stateless/02862_uuid_reinterpret_as_numeric.sql b/tests/queries/0_stateless/02862_uuid_reinterpret_as_numeric.sql new file mode 100644 index 00000000000..d6369835f04 --- /dev/null +++ b/tests/queries/0_stateless/02862_uuid_reinterpret_as_numeric.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS t_uuid; +CREATE TABLE t_uuid (x UUID) ENGINE=MergeTree ORDER BY x; + +INSERT INTO t_uuid VALUES ('61f0c404-5cb3-11e7-907b-a6006ad3dba0'); + +SELECT reinterpretAsUUID(x) FROM t_uuid; +SELECT reinterpretAsFloat32(x), reinterpretAsFloat64(x) FROM t_uuid; +SELECT reinterpretAsInt8(x), reinterpretAsInt16(x), reinterpretAsInt32(x), reinterpretAsInt64(x), reinterpretAsInt128(x), reinterpretAsInt256(x) FROM t_uuid; +SELECT reinterpretAsUInt8(x), reinterpretAsUInt16(x), reinterpretAsUInt32(x), reinterpretAsUInt64(x), reinterpretAsUInt128(x), reinterpretAsUInt256(x) FROM t_uuid; + +SELECT reinterpretAsUUID(reinterpretAsUInt128(reinterpretAsUInt32(reinterpretAsUInt256(x)))) FROM t_uuid; + +DROP TABLE IF EXISTS t_uuid; diff --git a/tests/queries/0_stateless/02863_delayed_source_with_totals_and_extremes.reference b/tests/queries/0_stateless/02863_delayed_source_with_totals_and_extremes.reference new file mode 100644 index 00000000000..4657a3024bf --- /dev/null +++ b/tests/queries/0_stateless/02863_delayed_source_with_totals_and_extremes.reference @@ -0,0 +1,7 @@ +3 + +3 + +3 +3 +1 diff --git a/tests/queries/0_stateless/02863_delayed_source_with_totals_and_extremes.sql b/tests/queries/0_stateless/02863_delayed_source_with_totals_and_extremes.sql new file mode 100644 index 00000000000..db2c3d92f47 --- /dev/null +++ b/tests/queries/0_stateless/02863_delayed_source_with_totals_and_extremes.sql @@ -0,0 +1,16 @@ +-- Tags: no-parallel +-- Tag no-parallel: failpoint is used which can force DelayedSource on other tests + +DROP TABLE IF EXISTS 02863_delayed_source; + +CREATE TABLE 02863_delayed_source(a Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/02863_delayed_source/{replica}', 'r1') ORDER BY a; +INSERT INTO 02863_delayed_source VALUES (1), (2); + +SYSTEM ENABLE FAILPOINT use_delayed_remote_source; + +SELECT sum(a) FROM remote('127.0.0.4', currentDatabase(), '02863_delayed_source') WITH TOTALS SETTINGS extremes = 1; +SELECT max(explain like '%Delayed%') FROM (EXPLAIN PIPELINE graph=1 SELECT sum(a) FROM remote('127.0.0.4', currentDatabase(), '02863_delayed_source') WITH TOTALS SETTINGS extremes = 1); + +SYSTEM DISABLE FAILPOINT use_delayed_remote_source; + +DROP TABLE 02863_delayed_source; \ No newline at end of file