Merge remote-tracking branch 'blessed/master' into i71382

This commit is contained in:
Raúl Marín 2024-11-05 12:36:37 +01:00
commit 097fb78a6e
124 changed files with 2103 additions and 1564 deletions

View File

@ -88,6 +88,7 @@ string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC)
list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES)
option (ENABLE_FUZZING "Fuzzy testing using libfuzzer" OFF)
option (ENABLE_FUZZER_TEST "Build testing fuzzers in order to test libFuzzer functionality" OFF)
if (ENABLE_FUZZING)
# Also set WITH_COVERAGE=1 for better fuzzing process

View File

@ -14,9 +14,10 @@ The following versions of ClickHouse server are currently supported with securit
| Version | Supported |
|:-|:-|
| 24.10 | ✔️ |
| 24.9 | ✔️ |
| 24.8 | ✔️ |
| 24.7 | ✔️ |
| 24.7 | |
| 24.6 | ❌ |
| 24.5 | ❌ |
| 24.4 | ❌ |

View File

@ -86,7 +86,7 @@ using StringRefs = std::vector<StringRef>;
* For more information, see hash_map_string_2.cpp
*/
inline bool compare8(const char * p1, const char * p2)
inline bool compare16(const char * p1, const char * p2)
{
return 0xFFFF == _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p1)),
@ -115,7 +115,7 @@ inline bool compare64(const char * p1, const char * p2)
#elif defined(__aarch64__) && defined(__ARM_NEON)
inline bool compare8(const char * p1, const char * p2)
inline bool compare16(const char * p1, const char * p2)
{
uint64_t mask = getNibbleMask(vceqq_u8(
vld1q_u8(reinterpret_cast<const unsigned char *>(p1)), vld1q_u8(reinterpret_cast<const unsigned char *>(p2))));
@ -185,13 +185,22 @@ inline bool memequalWide(const char * p1, const char * p2, size_t size)
switch (size / 16) // NOLINT(bugprone-switch-missing-default-case)
{
case 3: if (!compare8(p1 + 32, p2 + 32)) return false; [[fallthrough]];
case 2: if (!compare8(p1 + 16, p2 + 16)) return false; [[fallthrough]];
case 1: if (!compare8(p1, p2)) return false; [[fallthrough]];
case 3:
if (!compare16(p1 + 32, p2 + 32))
return false;
[[fallthrough]];
case 2:
if (!compare16(p1 + 16, p2 + 16))
return false;
[[fallthrough]];
case 1:
if (!compare16(p1, p2))
return false;
[[fallthrough]];
default: ;
}
return compare8(p1 + size - 16, p2 + size - 16);
return compare16(p1 + size - 16, p2 + size - 16);
}
#endif

2
contrib/krb5 vendored

@ -1 +1 @@
Subproject commit 71b06c2276009ae649c7703019f3b4605f66fd3d
Subproject commit c5b4b994c18db86933255907a97eee5993fd18fe

View File

@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="24.9.2.42"
ARG VERSION="24.10.1.2812"
ARG PACKAGES="clickhouse-keeper"
ARG DIRECT_DOWNLOAD_URLS=""

View File

@ -35,7 +35,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="24.9.2.42"
ARG VERSION="24.10.1.2812"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
ARG DIRECT_DOWNLOAD_URLS=""

View File

@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
ARG VERSION="24.9.2.42"
ARG VERSION="24.10.1.2812"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
#docker-official-library:off

View File

@ -33,8 +33,6 @@ RUN apt-get update \
COPY requirements.txt /
RUN pip3 install --no-cache-dir -r /requirements.txt
ENV FUZZER_ARGS="-max_total_time=60"
SHELL ["/bin/bash", "-c"]
# docker run --network=host --volume <workspace>:/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/libfuzzer

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

View File

@ -1353,9 +1353,11 @@ try
}
FailPointInjection::enableFromGlobalConfig(config());
#endif
memory_worker.start();
#if defined(OS_LINUX)
int default_oom_score = 0;
#if !defined(NDEBUG)

View File

@ -608,7 +608,7 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po
}
catch (...)
{
tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed");
tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed", LogsLevel::information);
WriteBufferFromOwnString message;
message << credentials.getUserName() << ": Authentication failed: password is incorrect, or there is no user with such name.";
@ -622,8 +622,9 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po
<< "and deleting this file will reset the password.\n"
<< "See also /etc/clickhouse-server/users.xml on the server where ClickHouse is installed.\n\n";
/// We use the same message for all authentication failures because we don't want to give away any unnecessary information for security reasons,
/// only the log will show the exact reason.
/// We use the same message for all authentication failures because we don't want to give away any unnecessary information for security reasons.
/// Only the log ((*), above) will show the exact reason. Note that (*) logs at information level instead of the default error level as
/// authentication failures are not an unusual event.
throw Exception(PreformattedMessage{message.str(),
"{}: Authentication failed: password is incorrect, or there is no user with such name",
std::vector<std::string>{credentials.getUserName()}},

View File

@ -1,2 +1,2 @@
clickhouse_add_executable(aggregate_function_state_deserialization_fuzzer aggregate_function_state_deserialization_fuzzer.cpp ${SRCS})
target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions)
target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions dbms)

View File

@ -251,7 +251,7 @@ void Exception::setThreadFramePointers(ThreadFramePointersBase frame_pointers)
thread_frame_pointers.frame_pointers = std::move(frame_pointers);
}
static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string & start_of_message)
static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string & start_of_message, LogsLevel level)
{
if (!isLoggingEnabled())
return;
@ -262,14 +262,25 @@ static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string
if (!start_of_message.empty())
message.text = fmt::format("{}: {}", start_of_message, message.text);
LOG_ERROR(logger, message);
switch (level)
{
case LogsLevel::none: break;
case LogsLevel::test: LOG_TEST(logger, message); break;
case LogsLevel::trace: LOG_TRACE(logger, message); break;
case LogsLevel::debug: LOG_DEBUG(logger, message); break;
case LogsLevel::information: LOG_INFO(logger, message); break;
case LogsLevel::warning: LOG_WARNING(logger, message); break;
case LogsLevel::error: LOG_ERROR(logger, message); break;
case LogsLevel::fatal: LOG_FATAL(logger, message); break;
}
}
catch (...) // NOLINT(bugprone-empty-catch)
{
}
}
void tryLogCurrentException(const char * log_name, const std::string & start_of_message)
void tryLogCurrentException(const char * log_name, const std::string & start_of_message, LogsLevel level)
{
if (!isLoggingEnabled())
return;
@ -283,10 +294,10 @@ void tryLogCurrentException(const char * log_name, const std::string & start_of_
/// getLogger can allocate memory too
auto logger = getLogger(log_name);
tryLogCurrentExceptionImpl(logger.get(), start_of_message);
tryLogCurrentExceptionImpl(logger.get(), start_of_message, level);
}
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message)
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message, LogsLevel level)
{
/// Under high memory pressure, new allocations throw a
/// MEMORY_LIMIT_EXCEEDED exception.
@ -295,17 +306,17 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_
/// MemoryTracker until the exception will be logged.
LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global);
tryLogCurrentExceptionImpl(logger, start_of_message);
tryLogCurrentExceptionImpl(logger, start_of_message, level);
}
void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message)
void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message, LogsLevel level)
{
tryLogCurrentException(logger.get(), start_of_message);
tryLogCurrentException(logger.get(), start_of_message, level);
}
void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message)
void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message, LogsLevel level)
{
tryLogCurrentException(logger.load(), start_of_message);
tryLogCurrentException(logger.load(), start_of_message, level);
}
static void getNoSpaceLeftInfoMessage(std::filesystem::path path, String & msg)

View File

@ -7,6 +7,7 @@
#include <Common/Logger.h>
#include <Common/LoggingFormatStringHelpers.h>
#include <Common/StackTrace.h>
#include <Core/LogsLevel.h>
#include <cerrno>
#include <exception>
@ -276,10 +277,10 @@ using Exceptions = std::vector<std::exception_ptr>;
* Can be used in destructors in the catch-all block.
*/
/// TODO: Logger leak constexpr overload
void tryLogCurrentException(const char * log_name, const std::string & start_of_message = "");
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = "");
void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message = "");
void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message = "");
void tryLogCurrentException(const char * log_name, const std::string & start_of_message = "", LogsLevel level = LogsLevel::error);
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = "", LogsLevel level = LogsLevel::error);
void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message = "", LogsLevel level = LogsLevel::error);
void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message = "", LogsLevel level = LogsLevel::error);
/** Prints current exception in canonical format.

View File

@ -119,4 +119,15 @@ enum class JoinTableSide : uint8_t
const char * toString(JoinTableSide join_table_side);
/// Setting to choose which table to use as the inner table in hash join
enum class JoinInnerTableSelectionMode : uint8_t
{
/// Use left table
Left,
/// Use right table
Right,
/// Use the table with the smallest number of rows
Auto,
};
}

View File

@ -1912,6 +1912,9 @@ See also:
For single JOIN in case of identifier ambiguity prefer left table
)", IMPORTANT) \
\
DECLARE(JoinInnerTableSelectionMode, query_plan_join_inner_table_selection, JoinInnerTableSelectionMode::Auto, R"(
Select the side of the join to be the inner table in the query plan. Supported only for `ALL` join strictness with `JOIN ON` clause. Possible values: 'auto', 'left', 'right'.
)", 0) \
DECLARE(UInt64, preferred_block_size_bytes, 1000000, R"(
This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality.
)", 0) \

View File

@ -66,6 +66,7 @@ class WriteBuffer;
M(CLASS_NAME, IntervalOutputFormat) \
M(CLASS_NAME, JoinAlgorithm) \
M(CLASS_NAME, JoinStrictness) \
M(CLASS_NAME, JoinInnerTableSelectionMode) \
M(CLASS_NAME, LightweightMutationProjectionMode) \
M(CLASS_NAME, LoadBalancing) \
M(CLASS_NAME, LocalFSReadMethod) \

View File

@ -71,6 +71,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"backup_restore_keeper_max_retries_while_initializing", 0, 20, "New setting."},
{"backup_restore_keeper_max_retries_while_handling_error", 0, 20, "New setting."},
{"backup_restore_finish_timeout_after_error_sec", 0, 180, "New setting."},
{"query_plan_join_inner_table_selection", "auto", "auto", "New setting."},
}
},
{"24.10",

View File

@ -55,6 +55,10 @@ IMPLEMENT_SETTING_MULTI_ENUM(JoinAlgorithm, ErrorCodes::UNKNOWN_JOIN,
{"full_sorting_merge", JoinAlgorithm::FULL_SORTING_MERGE},
{"grace_hash", JoinAlgorithm::GRACE_HASH}})
IMPLEMENT_SETTING_ENUM(JoinInnerTableSelectionMode, ErrorCodes::BAD_ARGUMENTS,
{{"left", JoinInnerTableSelectionMode::Left},
{"right", JoinInnerTableSelectionMode::Right},
{"auto", JoinInnerTableSelectionMode::Auto}})
IMPLEMENT_SETTING_ENUM(TotalsMode, ErrorCodes::UNKNOWN_TOTALS_MODE,
{{"before_having", TotalsMode::BEFORE_HAVING},

View File

@ -128,8 +128,8 @@ constexpr auto getEnumValues();
DECLARE_SETTING_ENUM(LoadBalancing)
DECLARE_SETTING_ENUM(JoinStrictness)
DECLARE_SETTING_MULTI_ENUM(JoinAlgorithm)
DECLARE_SETTING_ENUM(JoinInnerTableSelectionMode)
/// Which rows should be included in TOTALS.

View File

@ -1,2 +1,2 @@
clickhouse_add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp)
target_link_libraries (names_and_types_fuzzer PRIVATE)
target_link_libraries (names_and_types_fuzzer PRIVATE dbms)

View File

@ -1,2 +1,3 @@
clickhouse_add_executable(data_type_deserialization_fuzzer data_type_deserialization_fuzzer.cpp ${SRCS})
target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions)
target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions dbms)

View File

@ -3,6 +3,7 @@
#include <IO/ReadBufferFromMemory.h>
#include <IO/ReadHelpers.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeFactory.h>
#include <Common/MemoryTracker.h>

View File

@ -46,11 +46,13 @@ AsynchronousBoundedReadBuffer::AsynchronousBoundedReadBuffer(
ImplPtr impl_,
IAsynchronousReader & reader_,
const ReadSettings & settings_,
size_t buffer_size_,
AsyncReadCountersPtr async_read_counters_,
FilesystemReadPrefetchesLogPtr prefetches_log_)
: ReadBufferFromFileBase(0, nullptr, 0)
, impl(std::move(impl_))
, read_settings(settings_)
, buffer_size(buffer_size_)
, reader(reader_)
, query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "")
, current_reader_id(getRandomASCIIString(8))
@ -112,7 +114,7 @@ void AsynchronousBoundedReadBuffer::prefetch(Priority priority)
last_prefetch_info.submit_time = std::chrono::system_clock::now();
last_prefetch_info.priority = priority;
prefetch_buffer.resize(chooseBufferSizeForRemoteReading(read_settings, impl->getFileSize()));
prefetch_buffer.resize(buffer_size);
prefetch_future = readAsync(prefetch_buffer.data(), prefetch_buffer.size(), priority);
ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches);
}
@ -211,7 +213,7 @@ bool AsynchronousBoundedReadBuffer::nextImpl()
}
else
{
memory.resize(chooseBufferSizeForRemoteReading(read_settings, impl->getFileSize()));
memory.resize(buffer_size);
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::SynchronousRemoteReadWaitMicroseconds);

View File

@ -27,6 +27,7 @@ public:
ImplPtr impl_,
IAsynchronousReader & reader_,
const ReadSettings & settings_,
size_t buffer_size_,
AsyncReadCountersPtr async_read_counters_ = nullptr,
FilesystemReadPrefetchesLogPtr prefetches_log_ = nullptr);
@ -53,6 +54,7 @@ public:
private:
const ImplPtr impl;
const ReadSettings read_settings;
const size_t buffer_size;
IAsynchronousReader & reader;
size_t file_offset_of_buffer_end = 0;

View File

@ -41,6 +41,8 @@ public:
~CachedOnDiskReadBufferFromFile() override;
bool isCached() const override { return true; }
bool nextImpl() override;
off_t seek(off_t off, int whence) override;

View File

@ -18,24 +18,14 @@ namespace ErrorCodes
extern const int CANNOT_SEEK_THROUGH_FILE;
}
size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size)
{
/// Only when cache is used we could download bigger portions of FileSegments than what we actually gonna read within particular task.
if (!settings.enable_filesystem_cache && !settings.read_through_distributed_cache)
return settings.remote_fs_buffer_size;
/// Buffers used for prefetch and pre-download better to have enough size, but not bigger than the whole file.
return std::min<size_t>(std::max<size_t>(settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE), file_size);
}
ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(
ReadBufferCreator && read_buffer_creator_,
const StoredObjects & blobs_to_read_,
const ReadSettings & settings_,
std::shared_ptr<FilesystemCacheLog> cache_log_,
bool use_external_buffer_)
: ReadBufferFromFileBase(use_external_buffer_ ? 0 : chooseBufferSizeForRemoteReading(
settings_, getTotalSize(blobs_to_read_)), nullptr, 0)
bool use_external_buffer_,
size_t buffer_size)
: ReadBufferFromFileBase(use_external_buffer_ ? 0 : buffer_size, nullptr, 0)
, settings(settings_)
, blobs_to_read(blobs_to_read_)
, read_buffer_creator(std::move(read_buffer_creator_))

View File

@ -28,7 +28,8 @@ public:
const StoredObjects & blobs_to_read_,
const ReadSettings & settings_,
std::shared_ptr<FilesystemCacheLog> cache_log_,
bool use_external_buffer_);
bool use_external_buffer_,
size_t buffer_size);
~ReadBufferFromRemoteFSGather() override;
@ -84,6 +85,4 @@ private:
LoggerPtr log;
};
size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size);
}

View File

@ -641,19 +641,33 @@ std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
return impl;
};
/// Avoid cache fragmentation by choosing bigger buffer size.
bool prefer_bigger_buffer_size = object_storage->supportsCache() && read_settings.enable_filesystem_cache;
size_t buffer_size = prefer_bigger_buffer_size
? std::max<size_t>(settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE)
: settings.remote_fs_buffer_size;
size_t total_objects_size = file_size ? *file_size : getTotalSize(storage_objects);
if (total_objects_size)
buffer_size = std::min(buffer_size, total_objects_size);
const bool use_async_buffer = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool;
auto impl = std::make_unique<ReadBufferFromRemoteFSGather>(
std::move(read_buffer_creator),
storage_objects,
read_settings,
global_context->getFilesystemCacheLog(),
/* use_external_buffer */use_async_buffer);
/* use_external_buffer */use_async_buffer,
/* buffer_size */use_async_buffer ? 0 : buffer_size);
if (use_async_buffer)
{
auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER);
return std::make_unique<AsynchronousBoundedReadBuffer>(
std::move(impl), reader, read_settings,
std::move(impl),
reader,
read_settings,
buffer_size,
global_context->getAsyncReadCounters(),
global_context->getFilesystemReadPrefetchesLog());

View File

@ -51,7 +51,7 @@ TEST_F(AsynchronousBoundedReadBufferTest, setReadUntilPosition)
for (bool with_prefetch : {false, true})
{
AsynchronousBoundedReadBuffer read_buffer(createReadBufferFromFileBase(file_path, {}), remote_fs_reader, {});
AsynchronousBoundedReadBuffer read_buffer(createReadBufferFromFileBase(file_path, {}), remote_fs_reader, {}, DBMS_DEFAULT_BUFFER_SIZE);
read_buffer.setReadUntilPosition(20);
auto try_read = [&](size_t count)

View File

@ -1,2 +1,2 @@
clickhouse_add_executable(format_fuzzer format_fuzzer.cpp ${SRCS})
target_link_libraries(format_fuzzer PRIVATE clickhouse_aggregate_functions)
target_link_libraries(format_fuzzer PRIVATE clickhouse_aggregate_functions dbms)

View File

@ -60,6 +60,8 @@ public:
/// file offset and what getPosition() returns.
virtual bool isRegularLocalFile(size_t * /*out_view_offsee*/) { return false; }
virtual bool isCached() const { return false; }
protected:
std::optional<size_t> file_size;
ProfileCallback profile_callback;

View File

@ -60,6 +60,17 @@ public:
IBlocksStreamPtr
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override;
bool isCloneSupported() const override
{
return !getTotals() && getTotalRowCount() == 0;
}
std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_, const Block &, const Block & right_sample_block_) const override
{
return std::make_shared<ConcurrentHashJoin>(context, table_join_, slots, right_sample_block_, stats_collecting_params);
}
private:
struct InternalHashJoin
{

View File

@ -36,7 +36,7 @@ public:
bool isCloneSupported() const override
{
return true;
return !getTotals();
}
std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_,

View File

@ -383,6 +383,16 @@ size_t HashJoin::getTotalByteCount() const
return res;
}
bool HashJoin::isUsedByAnotherAlgorithm() const
{
return table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH);
}
bool HashJoin::canRemoveColumnsFromLeftBlock() const
{
return table_join->enableEnalyzer() && !table_join->hasUsing() && !isUsedByAnotherAlgorithm();
}
void HashJoin::initRightBlockStructure(Block & saved_block_sample)
{
if (isCrossOrComma(kind))
@ -394,8 +404,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample)
bool multiple_disjuncts = !table_join->oneDisjunct();
/// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any).
bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) ||
table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) ||
bool save_key_columns = isUsedByAnotherAlgorithm() ||
isRightOrFull(kind) ||
multiple_disjuncts ||
table_join->getMixedJoinExpression();
@ -1228,7 +1237,10 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block,
{
if (!JoinCommon::hasNonJoinedBlocks(*table_join))
return {};
size_t left_columns_count = left_sample_block.columns();
if (canRemoveColumnsFromLeftBlock())
left_columns_count = table_join->getOutputColumns(JoinTableSide::Left).size();
bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join);
if (!flag_per_row)

View File

@ -127,7 +127,7 @@ public:
bool isCloneSupported() const override
{
return true;
return !getTotals() && getTotalRowCount() == 0;
}
std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_,
@ -464,6 +464,9 @@ private:
bool empty() const;
bool isUsedByAnotherAlgorithm() const;
bool canRemoveColumnsFromLeftBlock() const;
void validateAdditionalFilterExpression(std::shared_ptr<ExpressionActions> additional_filter_expression);
bool needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const;

View File

@ -56,7 +56,6 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right;
join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]);
}
size_t existing_columns = block.columns();
/** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized.
* Because if they are constants, then in the "not joined" rows, they may have different values
@ -99,6 +98,22 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
added_columns.buildJoinGetOutput();
else
added_columns.buildOutput();
const auto & table_join = join.table_join;
std::set<size_t> block_columns_to_erase;
if (join.canRemoveColumnsFromLeftBlock())
{
std::unordered_set<String> left_output_columns;
for (const auto & out_column : table_join->getOutputColumns(JoinTableSide::Left))
left_output_columns.insert(out_column.name);
for (size_t i = 0; i < block.columns(); ++i)
{
if (!left_output_columns.contains(block.getByPosition(i).name))
block_columns_to_erase.insert(i);
}
}
size_t existing_columns = block.columns();
for (size_t i = 0; i < added_columns.size(); ++i)
block.insert(added_columns.moveColumn(i));
@ -160,6 +175,7 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate);
}
}
block.erase(block_columns_to_erase);
return remaining_block;
}

View File

@ -1888,7 +1888,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
expressions.join,
settings[Setting::max_block_size],
max_streams,
analysis_result.optimize_read_in_order);
/* required_output_ = */ NameSet{},
analysis_result.optimize_read_in_order,
/* use_new_analyzer_ = */ false);
join_step->setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType()));
std::vector<QueryPlanPtr> plans;

View File

@ -1310,7 +1310,7 @@ RefreshTaskList InterpreterSystemQuery::getRefreshTasks()
void InterpreterSystemQuery::prewarmMarkCache()
{
if (table_id.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for prewarming marks cache");
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for PREWARM MARK CACHE command");
getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE, table_id);

View File

@ -41,6 +41,7 @@ namespace DB
namespace Setting
{
extern const SettingsBool allow_experimental_join_right_table_sorting;
extern const SettingsBool allow_experimental_analyzer;
extern const SettingsUInt64 cross_join_min_bytes_to_compress;
extern const SettingsUInt64 cross_join_min_rows_to_compress;
extern const SettingsUInt64 default_max_bytes_in_join;
@ -143,6 +144,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary
, max_memory_usage(settings[Setting::max_memory_usage])
, tmp_volume(tmp_volume_)
, tmp_data(tmp_data_)
, enable_analyzer(settings[Setting::allow_experimental_analyzer])
{
}
@ -161,6 +163,8 @@ void TableJoin::resetCollected()
clauses.clear();
columns_from_joined_table.clear();
columns_added_by_join.clear();
columns_from_left_table.clear();
result_columns_from_left_table.clear();
original_names.clear();
renames.clear();
left_type_map.clear();
@ -203,6 +207,20 @@ size_t TableJoin::rightKeyInclusion(const String & name) const
return count;
}
void TableJoin::setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns)
{
columns_from_left_table = std::move(left_output_columns);
columns_from_joined_table = std::move(right_output_columns);
}
const NamesAndTypesList & TableJoin::getOutputColumns(JoinTableSide side)
{
if (side == JoinTableSide::Left)
return result_columns_from_left_table;
return columns_added_by_join;
}
void TableJoin::deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix)
{
NameSet joined_columns;
@ -351,9 +369,18 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const
return forceNullableRight() && JoinCommon::canBecomeNullable(column_type);
}
void TableJoin::setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side)
{
if (side == JoinTableSide::Left)
result_columns_from_left_table.push_back(joined_column);
else
columns_added_by_join.push_back(joined_column);
}
void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column)
{
columns_added_by_join.emplace_back(joined_column);
setUsedColumn(joined_column, JoinTableSide::Right);
}
NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const
@ -995,5 +1022,32 @@ size_t TableJoin::getMaxMemoryUsage() const
return max_memory_usage;
}
void TableJoin::swapSides()
{
assertEnableEnalyzer();
std::swap(key_asts_left, key_asts_right);
std::swap(left_type_map, right_type_map);
for (auto & clause : clauses)
{
std::swap(clause.key_names_left, clause.key_names_right);
std::swap(clause.on_filter_condition_left, clause.on_filter_condition_right);
std::swap(clause.analyzer_left_filter_condition_column_name, clause.analyzer_right_filter_condition_column_name);
}
std::swap(columns_from_left_table, columns_from_joined_table);
std::swap(result_columns_from_left_table, columns_added_by_join);
if (table_join.kind == JoinKind::Left)
table_join.kind = JoinKind::Right;
else if (table_join.kind == JoinKind::Right)
table_join.kind = JoinKind::Left;
}
void TableJoin::assertEnableEnalyzer() const
{
if (!enable_analyzer)
throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TableJoin: analyzer is disabled");
}
}

View File

@ -167,6 +167,9 @@ private:
ASOFJoinInequality asof_inequality = ASOFJoinInequality::GreaterOrEquals;
NamesAndTypesList columns_from_left_table;
NamesAndTypesList result_columns_from_left_table;
/// All columns which can be read from joined table. Duplicating names are qualified.
NamesAndTypesList columns_from_joined_table;
/// Columns will be added to block by JOIN.
@ -202,6 +205,8 @@ private:
bool is_join_with_constant = false;
bool enable_analyzer = false;
Names requiredJoinedNames() const;
/// Create converting actions and change key column names if required
@ -266,6 +271,8 @@ public:
VolumePtr getGlobalTemporaryVolume() { return tmp_volume; }
TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; }
bool enableEnalyzer() const { return enable_analyzer; }
void assertEnableEnalyzer() const;
ActionsDAG createJoinedBlockActions(ContextPtr context) const;
@ -282,6 +289,7 @@ public:
}
bool allowParallelHashJoin() const;
void swapSides();
bool joinUseNulls() const { return join_use_nulls; }
@ -372,6 +380,9 @@ public:
bool leftBecomeNullable(const DataTypePtr & column_type) const;
bool rightBecomeNullable(const DataTypePtr & column_type) const;
void addJoinedColumn(const NameAndTypePair & joined_column);
void setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side);
void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value)
{
columns_added_by_join = columns_added_by_join_value;
@ -397,11 +408,17 @@ public:
ASTPtr leftKeysList() const;
ASTPtr rightKeysList() const; /// For ON syntax only
void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix)
void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix, const NamesAndTypesList & columns_from_left_table_)
{
columns_from_joined_table = std::move(columns_from_joined_table_value);
deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix);
result_columns_from_left_table = columns_from_left_table_;
columns_from_left_table = columns_from_left_table_;
}
void setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns);
const NamesAndTypesList & getOutputColumns(JoinTableSide side);
const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; }
const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; }

View File

@ -1353,12 +1353,15 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
if (tables_with_columns.size() > 1)
{
auto columns_from_left_table = tables_with_columns[0].columns;
const auto & right_table = tables_with_columns[1];
auto columns_from_joined_table = right_table.columns;
/// query can use materialized or aliased columns from right joined table,
/// we want to request it for right table
columns_from_joined_table.insert(columns_from_joined_table.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end());
result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix());
columns_from_left_table.insert(columns_from_left_table.end(), tables_with_columns[0].hidden_columns.begin(), tables_with_columns[0].hidden_columns.end());
result.analyzed_join->setColumnsFromJoinedTable(
std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix(), columns_from_left_table);
}
translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns);

View File

@ -3,5 +3,6 @@ target_link_libraries(execute_query_fuzzer PRIVATE
dbms
clickhouse_table_functions
clickhouse_aggregate_functions
clickhouse_functions
clickhouse_dictionaries
clickhouse_dictionaries_embedded)

View File

@ -31,7 +31,7 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r
/// If we generate random UUIDs for already existing tables then those UUIDs will not be correct making those inner target table inaccessible.
/// Thus it's not safe for example to replace
/// "ATTACH MATERIALIZED VIEW mv AS SELECT a FROM b" with
/// "ATTACH MATERIALIZED VIEW mv TO INNER UUID "XXXX" AS SELECT a FROM b"
/// "ATTACH MATERIALIZED VIEW mv TO INNER UUID '123e4567-e89b-12d3-a456-426614174000' AS SELECT a FROM b"
/// This replacement is safe only for CREATE queries when inner target tables don't exist yet.
if (!query.attach)
{

View File

@ -2,10 +2,10 @@ clickhouse_add_executable(lexer_fuzzer lexer_fuzzer.cpp ${SRCS})
target_link_libraries(lexer_fuzzer PRIVATE clickhouse_parsers)
clickhouse_add_executable(select_parser_fuzzer select_parser_fuzzer.cpp ${SRCS})
target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers dbms)
target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers clickhouse_functions dbms)
clickhouse_add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS})
target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers dbms)
target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers clickhouse_functions dbms)
add_subdirectory(codegen_fuzzer)

View File

@ -47,4 +47,4 @@ target_compile_options (codegen_select_fuzzer PRIVATE -Wno-newline-eof)
target_link_libraries(protoc ch_contrib::fuzzer)
target_include_directories(codegen_select_fuzzer SYSTEM BEFORE PRIVATE "${CMAKE_CURRENT_BINARY_DIR}")
target_link_libraries(codegen_select_fuzzer PRIVATE ch_contrib::protobuf_mutator ch_contrib::protoc dbms)
target_link_libraries(codegen_select_fuzzer PRIVATE ch_contrib::protobuf_mutator ch_contrib::protoc clickhouse_functions dbms)

View File

@ -2,6 +2,7 @@
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/JoinNode.h>
#include <Planner/PlannerContext.h>

View File

@ -104,6 +104,7 @@ namespace Setting
extern const SettingsBool optimize_move_to_prewhere;
extern const SettingsBool optimize_move_to_prewhere_if_final;
extern const SettingsBool use_concurrency_control;
extern const SettingsJoinInnerTableSelectionMode query_plan_join_inner_table_selection;
}
namespace ErrorCodes
@ -1241,6 +1242,55 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP
plan_to_add_cast.addStep(std::move(cast_join_columns_step));
}
std::optional<ActionsDAG> createStepToDropColumns(
const Block & header,
const ColumnIdentifierSet & outer_scope_columns,
const PlannerContextPtr & planner_context)
{
ActionsDAG drop_unused_columns_after_join_actions_dag(header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs;
std::unordered_set<std::string_view> drop_unused_columns_after_join_actions_dag_updated_outputs_names;
std::optional<size_t> first_skipped_column_node_index;
auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs();
size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size();
const auto & global_planner_context = planner_context->getGlobalPlannerContext();
for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i)
{
const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i];
if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name)
|| !global_planner_context->hasColumnIdentifier(output->result_name))
continue;
if (!outer_scope_columns.contains(output->result_name))
{
if (!first_skipped_column_node_index)
first_skipped_column_node_index = i;
continue;
}
drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output);
drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name);
}
if (!first_skipped_column_node_index)
return {};
/** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope.
*
* Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2;
*/
if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index)
drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]);
drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs);
return drop_unused_columns_after_join_actions_dag;
}
JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression,
JoinTreeQueryPlan left_join_tree_query_plan,
JoinTreeQueryPlan right_join_tree_query_plan,
@ -1513,21 +1563,37 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
}
const Block & left_header = left_plan.getCurrentHeader();
auto left_table_names = left_header.getNames();
NameSet left_table_names_set(left_table_names.begin(), left_table_names.end());
const Block & right_header = right_plan.getCurrentHeader();
auto columns_from_joined_table = right_plan.getCurrentHeader().getNamesAndTypesList();
table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, "");
auto columns_from_left_table = left_header.getNamesAndTypesList();
auto columns_from_right_table = right_header.getNamesAndTypesList();
for (auto & column_from_joined_table : columns_from_joined_table)
table_join->setInputColumns(columns_from_left_table, columns_from_right_table);
for (auto & column_from_joined_table : columns_from_left_table)
{
/// Add columns from joined table only if they are presented in outer scope, otherwise they can be dropped
/// Add columns to output only if they are presented in outer scope, otherwise they can be dropped
if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) &&
outer_scope_columns.contains(column_from_joined_table.name))
table_join->addJoinedColumn(column_from_joined_table);
table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Left);
}
for (auto & column_from_joined_table : columns_from_right_table)
{
/// Add columns to output only if they are presented in outer scope, otherwise they can be dropped
if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) &&
outer_scope_columns.contains(column_from_joined_table.name))
table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right);
}
if (table_join->getOutputColumns(JoinTableSide::Left).empty() && table_join->getOutputColumns(JoinTableSide::Right).empty())
{
if (!columns_from_left_table.empty())
table_join->setUsedColumn(columns_from_left_table.front(), JoinTableSide::Left);
else if (!columns_from_right_table.empty())
table_join->setUsedColumn(columns_from_right_table.front(), JoinTableSide::Right);
}
const Block & right_header = right_plan.getCurrentHeader();
auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context);
auto result_plan = QueryPlan();
@ -1618,13 +1684,26 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
}
auto join_pipeline_type = join_algorithm->pipelineType();
ColumnIdentifierSet outer_scope_columns_nonempty;
if (outer_scope_columns.empty())
{
if (left_header.columns() > 1)
outer_scope_columns_nonempty.insert(left_header.getByPosition(0).name);
else if (right_header.columns() > 1)
outer_scope_columns_nonempty.insert(right_header.getByPosition(0).name);
}
auto join_step = std::make_unique<JoinStep>(
left_plan.getCurrentHeader(),
right_plan.getCurrentHeader(),
std::move(join_algorithm),
settings[Setting::max_block_size],
settings[Setting::max_threads],
false /*optimize_read_in_order*/);
outer_scope_columns.empty() ? outer_scope_columns_nonempty : outer_scope_columns,
false /*optimize_read_in_order*/,
true /*optimize_skip_unused_shards*/);
join_step->inner_table_selection_mode = settings[Setting::query_plan_join_inner_table_selection];
join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type));
@ -1635,47 +1714,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
result_plan.unitePlans(std::move(join_step), {std::move(plans)});
}
ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentHeader().getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs;
std::unordered_set<std::string_view> drop_unused_columns_after_join_actions_dag_updated_outputs_names;
std::optional<size_t> first_skipped_column_node_index;
auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs();
size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size();
for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i)
const auto & header_after_join = result_plan.getCurrentHeader();
if (header_after_join.columns() > outer_scope_columns.size())
{
const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i];
const auto & global_planner_context = planner_context->getGlobalPlannerContext();
if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name)
|| !global_planner_context->hasColumnIdentifier(output->result_name))
continue;
if (!outer_scope_columns.contains(output->result_name))
auto drop_unused_columns_after_join_actions_dag = createStepToDropColumns(header_after_join, outer_scope_columns, planner_context);
if (drop_unused_columns_after_join_actions_dag)
{
if (!first_skipped_column_node_index)
first_skipped_column_node_index = i;
continue;
auto drop_unused_columns_after_join_transform_step = std::make_unique<ExpressionStep>(result_plan.getCurrentHeader(), std::move(*drop_unused_columns_after_join_actions_dag));
drop_unused_columns_after_join_transform_step->setStepDescription("Drop unused columns after JOIN");
result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step));
}
drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output);
drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name);
}
/** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope.
*
* Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2;
*/
if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index)
drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]);
drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs);
auto drop_unused_columns_after_join_transform_step = std::make_unique<ExpressionStep>(result_plan.getCurrentHeader(), std::move(drop_unused_columns_after_join_actions_dag));
drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN");
result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step));
for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies)
left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy);

View File

@ -6,6 +6,7 @@
#include <IO/Operators.h>
#include <Common/JSONBuilder.h>
#include <Common/typeid_cast.h>
#include <Processors/Transforms/ColumnPermuteTransform.h>
namespace DB
{
@ -36,6 +37,37 @@ std::vector<std::pair<String, String>> describeJoinActions(const JoinPtr & join)
return description;
}
std::vector<size_t> getPermutationForBlock(
const Block & block,
const Block & lhs_block,
const Block & rhs_block,
const NameSet & name_filter)
{
std::vector<size_t> permutation;
permutation.reserve(block.columns());
Block::NameMap name_map = block.getNamesToIndexesMap();
bool is_trivial = true;
for (const auto & other_block : {lhs_block, rhs_block})
{
for (const auto & col : other_block)
{
if (!name_filter.contains(col.name))
continue;
if (auto it = name_map.find(col.name); it != name_map.end())
{
is_trivial = is_trivial && it->second == permutation.size();
permutation.push_back(it->second);
}
}
}
if (is_trivial && permutation.size() == block.columns())
return {};
return permutation;
}
}
JoinStep::JoinStep(
@ -44,8 +76,15 @@ JoinStep::JoinStep(
JoinPtr join_,
size_t max_block_size_,
size_t max_streams_,
bool keep_left_read_in_order_)
: join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_)
NameSet required_output_,
bool keep_left_read_in_order_,
bool use_new_analyzer_)
: join(std::move(join_))
, max_block_size(max_block_size_)
, max_streams(max_streams_)
, required_output(std::move(required_output_))
, keep_left_read_in_order(keep_left_read_in_order_)
, use_new_analyzer(use_new_analyzer_)
{
updateInputHeaders({left_header_, right_header_});
}
@ -55,23 +94,43 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines
if (pipelines.size() != 2)
throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps");
Block lhs_header = pipelines[0]->getHeader();
Block rhs_header = pipelines[1]->getHeader();
if (swap_streams)
std::swap(pipelines[0], pipelines[1]);
if (join->pipelineType() == JoinPipelineType::YShaped)
{
auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped(
std::move(pipelines[0]), std::move(pipelines[1]), join, *output_header, max_block_size, &processors);
std::move(pipelines[0]), std::move(pipelines[1]), join, join_algorithm_header, max_block_size, &processors);
joined_pipeline->resize(max_streams);
return joined_pipeline;
}
return QueryPipelineBuilder::joinPipelinesRightLeft(
auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft(
std::move(pipelines[0]),
std::move(pipelines[1]),
join,
*output_header,
join_algorithm_header,
max_block_size,
max_streams,
keep_left_read_in_order,
&processors);
if (!use_new_analyzer)
return pipeline;
auto column_permutation = getPermutationForBlock(pipeline->getHeader(), lhs_header, rhs_header, required_output);
if (!column_permutation.empty())
{
pipeline->addSimpleTransform([&column_permutation](const Block & header)
{
return std::make_shared<ColumnPermuteTransform>(header, column_permutation);
});
}
return pipeline;
}
bool JoinStep::allowPushDownToRight() const
@ -90,17 +149,49 @@ void JoinStep::describeActions(FormatSettings & settings) const
for (const auto & [name, value] : describeJoinActions(join))
settings.out << prefix << name << ": " << value << '\n';
if (swap_streams)
settings.out << prefix << "Swapped: true\n";
}
void JoinStep::describeActions(JSONBuilder::JSONMap & map) const
{
for (const auto & [name, value] : describeJoinActions(join))
map.add(name, value);
if (swap_streams)
map.add("Swapped", true);
}
void JoinStep::setJoin(JoinPtr join_, bool swap_streams_)
{
join_algorithm_header.clear();
swap_streams = swap_streams_;
join = std::move(join_);
updateOutputHeader();
}
void JoinStep::updateOutputHeader()
{
output_header = JoiningTransform::transformHeader(input_headers.front(), join);
if (join_algorithm_header)
return;
const auto & header = swap_streams ? input_headers[1] : input_headers[0];
Block result_header = JoiningTransform::transformHeader(header, join);
join_algorithm_header = result_header;
if (!use_new_analyzer)
{
if (swap_streams)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot swap streams without new analyzer");
output_header = result_header;
return;
}
auto column_permutation = getPermutationForBlock(result_header, input_headers[0], input_headers[1], required_output);
if (!column_permutation.empty())
result_header = ColumnPermuteTransform::permute(result_header, column_permutation);
output_header = result_header;
}
static ITransformingStep::Traits getStorageJoinTraits()

View File

@ -2,6 +2,7 @@
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <Processors/QueryPlan/ITransformingStep.h>
#include <Core/Joins.h>
namespace DB
{
@ -19,7 +20,9 @@ public:
JoinPtr join_,
size_t max_block_size_,
size_t max_streams_,
bool keep_left_read_in_order_);
NameSet required_output_,
bool keep_left_read_in_order_,
bool use_new_analyzer_);
String getName() const override { return "Join"; }
@ -31,16 +34,26 @@ public:
void describeActions(FormatSettings & settings) const override;
const JoinPtr & getJoin() const { return join; }
void setJoin(JoinPtr join_) { join = std::move(join_); }
void setJoin(JoinPtr join_, bool swap_streams_ = false);
bool allowPushDownToRight() const;
JoinInnerTableSelectionMode inner_table_selection_mode = JoinInnerTableSelectionMode::Right;
private:
void updateOutputHeader() override;
/// Header that expected to be returned from IJoin
Block join_algorithm_header;
JoinPtr join;
size_t max_block_size;
size_t max_streams;
const NameSet required_output;
std::set<size_t> columns_to_remove;
bool keep_left_read_in_order;
bool use_new_analyzer = false;
bool swap_streams = false;
};
/// Special step for the case when Join is already filled.

View File

@ -113,6 +113,7 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack);
void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes);
void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes);
void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &);
void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &);
void optimizeDistinctInOrder(QueryPlan::Node & node, QueryPlan::Nodes &);
/// A separate tree traverse to apply sorting properties after *InOrder optimizations.

View File

@ -0,0 +1,102 @@
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/FilterStep.h>
#include <Processors/QueryPlan/ITransformingStep.h>
#include <Processors/QueryPlan/JoinStep.h>
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Processors/QueryPlan/SortingStep.h>
#include <Storages/StorageMemory.h>
#include <Processors/QueryPlan/ReadFromMemoryStorageStep.h>
#include <Core/Settings.h>
#include <Interpreters/IJoin.h>
#include <Interpreters/HashJoin/HashJoin.h>
#include <Interpreters/TableJoin.h>
#include <Common/logger_useful.h>
#include <Core/Joins.h>
#include <ranges>
namespace DB::QueryPlanOptimizations
{
static std::optional<UInt64> estimateReadRowsCount(QueryPlan::Node & node)
{
IQueryPlanStep * step = node.step.get();
if (const auto * reading = typeid_cast<const ReadFromMergeTree *>(step))
{
if (auto analyzed_result = reading->getAnalyzedResult())
return analyzed_result->selected_rows;
if (auto analyzed_result = reading->selectRangesToRead())
return analyzed_result->selected_rows;
return {};
}
if (const auto * reading = typeid_cast<const ReadFromMemoryStorageStep *>(step))
return reading->getStorage()->totalRows(Settings{});
if (node.children.size() != 1)
return {};
if (typeid_cast<ExpressionStep *>(step) || typeid_cast<FilterStep *>(step))
return estimateReadRowsCount(*node.children.front());
return {};
}
void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &)
{
auto * join_step = typeid_cast<JoinStep *>(node.step.get());
if (!join_step || node.children.size() != 2)
return;
const auto & join = join_step->getJoin();
if (join->pipelineType() != JoinPipelineType::FillRightFirst || !join->isCloneSupported())
return;
const auto & table_join = join->getTableJoin();
/// Algorithms other than HashJoin may not support OUTER JOINs
if (table_join.kind() != JoinKind::Inner && !typeid_cast<const HashJoin *>(join.get()))
return;
/// fixme: USING clause handled specially in join algorithm, so swap breaks it
/// fixme: Swapping for SEMI and ANTI joins should be alright, need to try to enable it and test
if (table_join.hasUsing() || table_join.strictness() != JoinStrictness::All)
return;
bool need_swap = false;
if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Auto)
{
auto lhs_extimation = estimateReadRowsCount(*node.children[0]);
auto rhs_extimation = estimateReadRowsCount(*node.children[1]);
LOG_TRACE(getLogger("optimizeJoin"), "Left table estimation: {}, right table estimation: {}",
lhs_extimation.transform(toString<UInt64>).value_or("unknown"),
rhs_extimation.transform(toString<UInt64>).value_or("unknown"));
if (lhs_extimation && rhs_extimation && *lhs_extimation < *rhs_extimation)
need_swap = true;
}
else if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Left)
{
need_swap = true;
}
if (!need_swap)
return;
const auto & headers = join_step->getInputHeaders();
if (headers.size() != 2)
return;
const auto & left_stream_input_header = headers.front();
const auto & right_stream_input_header = headers.back();
auto updated_table_join = std::make_shared<TableJoin>(table_join);
updated_table_join->swapSides();
auto updated_join = join->clone(updated_table_join, right_stream_input_header, left_stream_input_header);
join_step->setJoin(std::move(updated_join), /* swap_streams= */ true);
}
}

View File

@ -227,6 +227,9 @@ void addStepsToBuildSets(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::No
/// NOTE: frame cannot be safely used after stack was modified.
auto & frame = stack.back();
if (frame.next_child == 0)
optimizeJoin(*frame.node, nodes);
/// Traverse all children first.
if (frame.next_child < frame.node->children.size())
{

View File

@ -35,6 +35,8 @@ public:
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
const StoragePtr & getStorage() const { return storage; }
private:
static constexpr auto name = "ReadFromMemoryStorage";

View File

@ -0,0 +1,49 @@
#include <Processors/Transforms/ColumnPermuteTransform.h>
namespace DB
{
namespace
{
template <typename T>
void applyPermutation(std::vector<T> & data, const std::vector<size_t> & permutation)
{
std::vector<T> res;
res.reserve(permutation.size());
for (size_t i : permutation)
res.push_back(data[i]);
data = std::move(res);
}
void permuteChunk(Chunk & chunk, const std::vector<size_t> & permutation)
{
size_t num_rows = chunk.getNumRows();
auto columns = chunk.detachColumns();
applyPermutation(columns, permutation);
chunk.setColumns(std::move(columns), num_rows);
}
}
Block ColumnPermuteTransform::permute(const Block & block, const std::vector<size_t> & permutation)
{
auto columns = block.getColumnsWithTypeAndName();
applyPermutation(columns, permutation);
return Block(columns);
}
ColumnPermuteTransform::ColumnPermuteTransform(const Block & header_, const std::vector<size_t> & permutation_)
: ISimpleTransform(header_, permute(header_, permutation_), false)
, permutation(permutation_)
{
}
void ColumnPermuteTransform::transform(Chunk & chunk)
{
permuteChunk(chunk, permutation);
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <atomic>
#include <mutex>
#include <vector>
#include <Processors/ISimpleTransform.h>
#include <Poco/Logger.h>
#include <Interpreters/Set.h>
namespace DB
{
class ColumnPermuteTransform : public ISimpleTransform
{
public:
ColumnPermuteTransform(const Block & header_, const std::vector<size_t> & permutation_);
String getName() const override { return "ColumnPermuteTransform"; }
void transform(Chunk & chunk) override;
static Block permute(const Block & block, const std::vector<size_t> & permutation);
private:
Names column_names;
std::vector<size_t> permutation;
};
}

View File

@ -19,6 +19,7 @@ Block JoiningTransform::transformHeader(Block header, const JoinPtr & join)
join->initialize(header);
ExtraBlockPtr tmp;
join->joinBlock(header, tmp);
materializeBlockInplace(header);
LOG_TEST(getLogger("JoiningTransform"), "After join block: '{}'", header.dumpStructure());
return header;
}

View File

@ -1614,7 +1614,8 @@ void TCPHandler::receiveHello()
if (e.code() != DB::ErrorCodes::AUTHENTICATION_FAILED)
throw;
tryLogCurrentException(log, "SSL authentication failed, falling back to password authentication");
tryLogCurrentException(log, "SSL authentication failed, falling back to password authentication", LogsLevel::information);
/// ^^ Log at debug level instead of default error level as authentication failures are not an unusual event.
}
}
}

View File

@ -2343,11 +2343,16 @@ void MergeTreeData::stopOutdatedAndUnexpectedDataPartsLoadingTask()
}
}
void MergeTreeData::prewarmMarkCache(ThreadPool & pool)
void MergeTreeData::prewarmMarkCacheIfNeeded(ThreadPool & pool)
{
if (!(*getSettings())[MergeTreeSetting::prewarm_mark_cache])
return;
prewarmMarkCache(pool);
}
void MergeTreeData::prewarmMarkCache(ThreadPool & pool)
{
auto * mark_cache = getContext()->getMarkCache().get();
if (!mark_cache)
return;

View File

@ -508,6 +508,7 @@ public:
/// Prewarm mark cache for the most recent data parts.
void prewarmMarkCache(ThreadPool & pool);
void prewarmMarkCacheIfNeeded(ThreadPool & pool);
String getLogName() const { return log.loadName(); }

View File

@ -517,9 +517,19 @@ std::unique_ptr<ReadBufferFromFileBase> StorageObjectStorageSource::createReadBu
LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size);
bool prefer_bigger_buffer_size = impl->isCached();
size_t buffer_size = prefer_bigger_buffer_size
? std::max<size_t>(read_settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE)
: read_settings.remote_fs_buffer_size;
if (object_size)
buffer_size = std::min<size_t>(object_size, buffer_size);
auto & reader = context_->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER);
impl = std::make_unique<AsynchronousBoundedReadBuffer>(
std::move(impl), reader, modified_read_settings,
std::move(impl),
reader,
modified_read_settings,
buffer_size,
context_->getAsyncReadCounters(),
context_->getFilesystemReadPrefetchesLog());

View File

@ -1,233 +0,0 @@
#include <Storages/StorageExternalDistributed.h>
#include <Core/Settings.h>
#include <Storages/StorageFactory.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Core/PostgreSQL/PoolWithFailover.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/parseAddress.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Common/parseRemoteDescription.h>
#include <Storages/StorageMySQL.h>
#include <Storages/MySQL/MySQLSettings.h>
#include <Storages/StoragePostgreSQL.h>
#include <Storages/StorageURL.h>
#include <Storages/MySQL/MySQLHelpers.h>
#include <Storages/NamedCollectionsHelpers.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <Common/logger_useful.h>
#include <Processors/QueryPlan/UnionStep.h>
namespace DB
{
namespace Setting
{
extern const SettingsUInt64 glob_expansion_max_elements;
extern const SettingsUInt64 postgresql_connection_attempt_timeout;
extern const SettingsBool postgresql_connection_pool_auto_close_connection;
extern const SettingsUInt64 postgresql_connection_pool_retries;
extern const SettingsUInt64 postgresql_connection_pool_size;
extern const SettingsUInt64 postgresql_connection_pool_wait_timeout;
}
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
StorageExternalDistributed::StorageExternalDistributed(
const StorageID & table_id_,
std::unordered_set<StoragePtr> && shards_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment)
: IStorage(table_id_)
, shards(shards_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
}
void StorageExternalDistributed::read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams)
{
std::vector<std::unique_ptr<QueryPlan>> plans;
for (const auto & shard : shards)
{
plans.emplace_back(std::make_unique<QueryPlan>());
shard->read(
*plans.back(),
column_names,
storage_snapshot,
query_info,
context,
processed_stage,
max_block_size,
num_streams
);
}
if (plans.empty())
{
auto header = storage_snapshot->getSampleBlockForColumns(column_names);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info);
}
if (plans.size() == 1)
{
query_plan = std::move(*plans.front());
return;
}
Headers input_headers;
input_headers.reserve(plans.size());
for (auto & plan : plans)
input_headers.emplace_back(plan->getCurrentHeader());
auto union_step = std::make_unique<UnionStep>(std::move(input_headers));
query_plan.unitePlans(std::move(union_step), std::move(plans));
}
void registerStorageExternalDistributed(StorageFactory & factory)
{
factory.registerStorage("ExternalDistributed", [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
if (engine_args.size() < 2)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Engine ExternalDistributed must have at least 2 arguments: "
"engine_name, named_collection and/or description");
auto context = args.getLocalContext();
const auto & settings = context->getSettingsRef();
size_t max_addresses = settings[Setting::glob_expansion_max_elements];
auto get_addresses = [&](const std::string addresses_expr)
{
return parseRemoteDescription(addresses_expr, 0, addresses_expr.size(), ',', max_addresses);
};
std::unordered_set<StoragePtr> shards;
ASTs inner_engine_args(engine_args.begin() + 1, engine_args.end());
ASTPtr * address_arg = nullptr;
/// If there is a named collection argument, named `addresses_expr`
for (auto & node : inner_engine_args)
{
if (ASTFunction * func = node->as<ASTFunction>(); func && func->name == "equals" && func->arguments)
{
if (ASTExpressionList * func_args = func->arguments->as<ASTExpressionList>(); func_args && func_args->children.size() == 2)
{
if (ASTIdentifier * arg_name = func_args->children[0]->as<ASTIdentifier>(); arg_name && arg_name->name() == "addresses_expr")
{
address_arg = &func_args->children[1];
break;
}
}
}
}
/// Otherwise it is the first argument.
if (!address_arg)
address_arg = &inner_engine_args.at(0);
String addresses_expr = checkAndGetLiteralArgument<String>(*address_arg, "addresses");
Strings shards_addresses = get_addresses(addresses_expr);
auto engine_name = checkAndGetLiteralArgument<String>(engine_args[0], "engine_name");
if (engine_name == "URL")
{
auto format_settings = StorageURL::getFormatSettingsFromArgs(args);
for (const auto & shard_address : shards_addresses)
{
*address_arg = std::make_shared<ASTLiteral>(shard_address);
auto configuration = StorageURL::getConfiguration(inner_engine_args, context);
auto uri_options = parseRemoteDescription(shard_address, 0, shard_address.size(), '|', max_addresses);
if (uri_options.size() > 1)
{
shards.insert(
std::make_shared<StorageURLWithFailover>(
uri_options, args.table_id, configuration.format, format_settings,
args.columns, args.constraints, context, configuration.compression_method));
}
else
{
shards.insert(std::make_shared<StorageURL>(
shard_address, args.table_id, configuration.format, format_settings,
args.columns, args.constraints, String{}, context, configuration.compression_method));
}
}
}
#if USE_MYSQL
else if (engine_name == "MySQL")
{
MySQLSettings mysql_settings;
for (const auto & shard_address : shards_addresses)
{
*address_arg = std::make_shared<ASTLiteral>(shard_address);
auto configuration = StorageMySQL::getConfiguration(inner_engine_args, context, mysql_settings);
configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 3306);
auto pool = createMySQLPoolWithFailover(configuration, mysql_settings);
shards.insert(std::make_shared<StorageMySQL>(
args.table_id, std::move(pool), configuration.database, configuration.table,
/* replace_query = */ false, /* on_duplicate_clause = */ "",
args.columns, args.constraints, String{}, context, mysql_settings));
}
}
#endif
#if USE_LIBPQXX
else if (engine_name == "PostgreSQL")
{
for (const auto & shard_address : shards_addresses)
{
*address_arg = std::make_shared<ASTLiteral>(shard_address);
auto configuration = StoragePostgreSQL::getConfiguration(inner_engine_args, context);
configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 5432);
auto pool = std::make_shared<postgres::PoolWithFailover>(
configuration,
settings[Setting::postgresql_connection_pool_size],
settings[Setting::postgresql_connection_pool_wait_timeout],
settings[Setting::postgresql_connection_pool_retries],
settings[Setting::postgresql_connection_pool_auto_close_connection],
settings[Setting::postgresql_connection_attempt_timeout]);
shards.insert(std::make_shared<StoragePostgreSQL>(
args.table_id, std::move(pool), configuration.table, args.columns, args.constraints, String{}, context));
}
}
#endif
else
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"External storage engine {} is not supported for StorageExternalDistributed. "
"Supported engines are: MySQL, PostgreSQL, URL",
engine_name);
}
return std::make_shared<StorageExternalDistributed>(
args.table_id,
std::move(shards),
args.columns,
args.constraints,
args.comment);
},
{
.source_access_type = AccessType::SOURCES,
});
}
}

View File

@ -1,43 +0,0 @@
#pragma once
#include "config.h"
#include <Storages/IStorage.h>
namespace DB
{
/// Storages MySQL and PostgreSQL use ConnectionPoolWithFailover and support multiple replicas.
/// This class unites multiple storages with replicas into multiple shards with replicas.
/// A query to external database is passed to one replica on each shard, the result is united.
/// Replicas on each shard have the same priority, traversed replicas are moved to the end of the queue.
/// Similar approach is used for URL storage.
class StorageExternalDistributed final : public DB::IStorage
{
public:
StorageExternalDistributed(
const StorageID & table_id_,
std::unordered_set<StoragePtr> && shards_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment);
std::string getName() const override { return "ExternalDistributed"; }
void read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams) override;
private:
using Shards = std::unordered_set<StoragePtr>;
Shards shards;
};
}

View File

@ -155,7 +155,7 @@ StorageMergeTree::StorageMergeTree(
loadMutations();
loadDeduplicationLog();
prewarmMarkCache(getActivePartsLoadingThreadPool().get());
prewarmMarkCacheIfNeeded(getActivePartsLoadingThreadPool().get());
}

View File

@ -509,7 +509,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
}
loadDataParts(skip_sanity_checks, expected_parts_on_this_replica);
prewarmMarkCache(getActivePartsLoadingThreadPool().get());
prewarmMarkCacheIfNeeded(getActivePartsLoadingThreadPool().get());
if (LoadingStrictnessLevel::ATTACH <= mode)
{

View File

@ -227,6 +227,194 @@ FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
/// Default per host dashboard for self-managed ClickHouse
{
{ "dashboard", "Overview (host)" },
{ "title", "Queries/second" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_Query)
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "CPU Usage (cores)" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSCPUVirtualTimeMicroseconds) / 1000000
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Queries Running" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(CurrentMetric_Query)
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Merges Running" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(CurrentMetric_Merge)
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Selected Bytes/second" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_SelectedBytes)
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "IO Wait" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSIOWaitMicroseconds) / 1000000
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "CPU Wait" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSCPUWaitMicroseconds) / 1000000
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "OS CPU Usage (Userspace)" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value)
FROM merge('system', '^asynchronous_metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSUserTimeNormalized'
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "OS CPU Usage (Kernel)" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value)
FROM merge('system', '^asynchronous_metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSSystemTimeNormalized'
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Read From Disk" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSReadBytes)
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Read From Filesystem" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSReadChars)
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Memory (tracked)" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(CurrentMetric_MemoryTracking)
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Load Average (15 minutes)" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value)
FROM merge('system', '^asynchronous_metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'LoadAverage15'
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Selected Rows/second" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_SelectedRows)
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Inserted Rows/second" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_InsertedRows)
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Total MergeTree Parts" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value)
FROM merge('system', '^asynchronous_metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'TotalPartsOfMergeTreeTables'
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "Overview (host)" },
{ "title", "Max Parts For Partition" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, max(value)
FROM merge('system', '^asynchronous_metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'MaxPartCountForPartition'
GROUP BY t, hostname
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
/// Default dashboard for ClickHouse Cloud
@ -369,7 +557,143 @@ ORDER BY t WITH FILL STEP {rounding:UInt32}
{ "dashboard", "Cloud overview" },
{ "title", "Concurrent network connections" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(TCP_Connections), max(MySQL_Connections), max(HTTP_Connections) FROM (SELECT event_time, sum(CurrentMetric_TCPConnection) AS TCP_Connections, sum(CurrentMetric_MySQLConnection) AS MySQL_Connections, sum(CurrentMetric_HTTPConnection) AS HTTP_Connections FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
}
},
/// Default per host dashboard for ClickHouse Cloud
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Queries/second" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_Query) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "CPU Usage (cores)" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(metric) / 1000000\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSCPUVirtualTimeMicroseconds) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32} GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Queries Running" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_Query) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Merges Running" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_Merge) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Selected Bytes/second" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_SelectedBytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "IO Wait (local fs)" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSIOWaitMicroseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "S3 read wait" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_ReadBufferFromS3Microseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "S3 read errors/sec" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_ReadBufferFromS3RequestsErrors) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "CPU Wait" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSCPUWaitMicroseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "OS CPU Usage (Userspace, normalized)" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'OSUserTimeNormalized'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "OS CPU Usage (Kernel, normalized)" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'OSSystemTimeNormalized'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Read From Disk (bytes/sec)" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSReadBytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Read From Filesystem (bytes/sec)" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSReadChars) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Memory (tracked, bytes)" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_MemoryTracking) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Load Average (15 minutes)" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM (\n SELECT event_time, hostname, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric = 'LoadAverage15'\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Selected Rows/sec" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_SelectedRows) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Inserted Rows/sec" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_InsertedRows) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Total MergeTree Parts" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, max(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'TotalPartsOfMergeTreeTables'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Max Parts For Partition" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, max(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'MaxPartCountForPartition'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Read From S3 (bytes/sec)" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_ReadBufferFromS3Bytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Filesystem Cache Size" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_FilesystemCacheSize) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Disk S3 write req/sec" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_DiskS3PutObject + ProfileEvent_DiskS3UploadPart + ProfileEvent_DiskS3CreateMultipartUpload + ProfileEvent_DiskS3CompleteMultipartUpload) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Disk S3 read req/sec" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_DiskS3GetObject + ProfileEvent_DiskS3HeadObject + ProfileEvent_DiskS3ListObjects) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t\nWITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "FS cache hit rate" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_CachedReadBufferReadFromCacheBytes) / (sum(ProfileEvent_CachedReadBufferReadFromCacheBytes) + sum(ProfileEvent_CachedReadBufferReadFromSourceBytes)) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Page cache hit rate" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, greatest(0, (sum(ProfileEvent_OSReadChars) - sum(ProfileEvent_OSReadBytes)) / (sum(ProfileEvent_OSReadChars) + sum(ProfileEvent_ReadBufferFromS3Bytes))) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Network receive bytes/sec" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM (\n SELECT event_time, hostname, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkReceiveBytes%'\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview (host)" },
{ "title", "Network send bytes/sec" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM (\n SELECT event_time, hostname, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkSendBytes%'\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
};
auto add_dashboards = [&](const auto & dashboards)

View File

@ -4,4 +4,4 @@ clickhouse_add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.c
target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms)
clickhouse_add_executable (columns_description_fuzzer columns_description_fuzzer.cpp)
target_link_libraries (columns_description_fuzzer PRIVATE)
target_link_libraries (columns_description_fuzzer PRIVATE dbms)

View File

@ -93,10 +93,6 @@ void registerStoragePostgreSQL(StorageFactory & factory);
void registerStorageMaterializedPostgreSQL(StorageFactory & factory);
#endif
#if USE_MYSQL || USE_LIBPQXX
void registerStorageExternalDistributed(StorageFactory & factory);
#endif
#if USE_FILELOG
void registerStorageFileLog(StorageFactory & factory);
#endif
@ -205,10 +201,6 @@ void registerStorages(bool use_legacy_mongodb_integration [[maybe_unused]])
registerStorageMaterializedPostgreSQL(factory);
#endif
#if USE_MYSQL || USE_LIBPQXX
registerStorageExternalDistributed(factory);
#endif
#if USE_SQLITE
registerStorageSQLite(factory);
#endif

View File

@ -6,7 +6,6 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/StorageExternalDistributed.h>
#include <Storages/NamedCollectionsHelpers.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Analyzer/FunctionNode.h>

View File

@ -539,9 +539,9 @@ class CI:
JobNames.LIBFUZZER_TEST: JobConfig(
required_builds=[BuildNames.FUZZERS],
run_by_labels=[Tags.libFuzzer],
timeout=10800,
timeout=5400,
run_command='libfuzzer_test_check.py "$CHECK_NAME"',
runner_type=Runners.STYLE_CHECKER,
runner_type=Runners.FUNC_TESTER,
),
JobNames.DOCKER_SERVER: CommonJobConfigs.DOCKER_SERVER.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE, BuildNames.PACKAGE_AARCH64]

View File

@ -3,20 +3,37 @@
import argparse
import logging
import os
import re
import sys
import zipfile
from pathlib import Path
from typing import List
from botocore.exceptions import ClientError
from build_download_helper import download_fuzzers
from clickhouse_helper import CiLogsCredentials
from docker_images_helper import DockerImage, get_docker_image, pull_image
from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH
from env_helper import REPO_COPY, REPORT_PATH, S3_BUILDS_BUCKET, TEMP_PATH
from pr_info import PRInfo
from report import FAILURE, SUCCESS, JobReport, TestResult
from s3_helper import S3Helper
from stopwatch import Stopwatch
from tee_popen import TeePopen
TIMEOUT = 60
NO_CHANGES_MSG = "Nothing to run"
s3 = S3Helper()
def zipdir(path, ziph):
# ziph is zipfile handle
for root, _, files in os.walk(path):
for file in files:
ziph.write(
os.path.join(root, file),
os.path.relpath(os.path.join(root, file), os.path.join(path, "..")),
)
def get_additional_envs(check_name, run_by_hash_num, run_by_hash_total):
@ -59,16 +76,19 @@ def get_run_command(
envs = [
# a static link, don't use S3_URL or S3_DOWNLOAD
'-e S3_URL="https://s3.amazonaws.com/clickhouse-datasets"',
'-e S3_URL="https://s3.amazonaws.com"',
]
envs += [f"-e {e}" for e in additional_envs]
env_str = " ".join(envs)
uid = os.getuid()
gid = os.getgid()
return (
f"docker run "
f"{ci_logs_args} "
f"--user {uid}:{gid} "
f"--workdir=/fuzzers "
f"--volume={fuzzers_path}:/fuzzers "
f"--volume={repo_path}/tests:/usr/share/clickhouse-test "
@ -85,6 +105,115 @@ def parse_args():
return parser.parse_args()
def download_corpus(path: str):
logging.info("Download corpus...")
try:
s3.download_file(
bucket=S3_BUILDS_BUCKET,
s3_path="fuzzer/corpus.zip",
local_file_path=path,
)
except ClientError as e:
if e.response["Error"]["Code"] == "NoSuchKey":
logging.debug("No active corpus exists")
else:
raise
with zipfile.ZipFile(f"{path}/corpus.zip", "r") as zipf:
zipf.extractall(path)
os.remove(f"{path}/corpus.zip")
units = 0
for _, _, files in os.walk(path):
units += len(files)
logging.info("...downloaded %d units", units)
def upload_corpus(path: str):
with zipfile.ZipFile(f"{path}/corpus.zip", "w", zipfile.ZIP_DEFLATED) as zipf:
zipdir(f"{path}/corpus/", zipf)
s3.upload_file(
bucket=S3_BUILDS_BUCKET,
file_path=f"{path}/corpus.zip",
s3_path="fuzzer/corpus.zip",
)
def process_error(path: Path) -> list:
ERROR = r"^==\d+==\s?ERROR: (\S+): (.*)"
# error_source = ""
# error_reason = ""
# test_unit = ""
# TEST_UNIT_LINE = r"artifact_prefix='.*\/'; Test unit written to (.*)"
error_info = []
is_error = False
with open(path, "r", encoding="utf-8") as file:
for line in file:
line = line.rstrip("\n")
if is_error:
error_info.append(line)
# match = re.search(TEST_UNIT_LINE, line)
# if match:
# test_unit = match.group(1)
continue
match = re.search(ERROR, line)
if match:
error_info.append(line)
# error_source = match.group(1)
# error_reason = match.group(2)
is_error = True
return error_info
def read_status(status_path: Path):
result = []
with open(status_path, "r", encoding="utf-8") as file:
for line in file:
result.append(line.rstrip("\n"))
return result
def process_results(result_path: Path):
test_results = []
oks = 0
errors = 0
fails = 0
for file in result_path.glob("*.status"):
fuzzer = file.stem
file_path = file.parent / fuzzer
file_path_unit = file_path.with_suffix(".unit")
file_path_out = file_path.with_suffix(".out")
file_path_stdout = file_path.with_suffix(".stdout")
status = read_status(file)
result = TestResult(fuzzer, status[0], float(status[2]))
if status[0] == "OK":
oks += 1
elif status[0] == "ERROR":
errors += 1
if file_path_out.exists():
result.set_log_files(f"['{file_path_out}']")
elif file_path_stdout.exists():
result.set_log_files(f"['{file_path_stdout}']")
else:
fails += 1
if file_path_out.exists():
result.set_raw_logs("\n".join(process_error(file_path_out)))
if file_path_unit.exists():
result.set_log_files(f"['{file_path_unit}']")
elif file_path_out.exists():
result.set_log_files(f"['{file_path_out}']")
elif file_path_stdout.exists():
result.set_log_files(f"['{file_path_stdout}']")
test_results.append(result)
return [oks, errors, fails, test_results]
def main():
logging.basicConfig(level=logging.INFO)
@ -114,15 +243,18 @@ def main():
fuzzers_path = temp_path / "fuzzers"
fuzzers_path.mkdir(parents=True, exist_ok=True)
download_corpus(fuzzers_path)
download_fuzzers(check_name, reports_path, fuzzers_path)
for file in os.listdir(fuzzers_path):
if file.endswith("_fuzzer"):
os.chmod(fuzzers_path / file, 0o777)
elif file.endswith("_seed_corpus.zip"):
corpus_path = fuzzers_path / (file.removesuffix("_seed_corpus.zip") + ".in")
seed_corpus_path = fuzzers_path / (
file.removesuffix("_seed_corpus.zip") + ".in"
)
with zipfile.ZipFile(fuzzers_path / file, "r") as zfd:
zfd.extractall(corpus_path)
zfd.extractall(seed_corpus_path)
result_path = temp_path / "result_path"
result_path.mkdir(parents=True, exist_ok=True)
@ -133,6 +265,8 @@ def main():
check_name, run_by_hash_num, run_by_hash_total
)
additional_envs.append(f"TIMEOUT={TIMEOUT}")
ci_logs_credentials = CiLogsCredentials(Path(temp_path) / "export-logs-config.sh")
ci_logs_args = ci_logs_credentials.get_docker_arguments(
pr_info, stopwatch.start_time_str, check_name
@ -152,10 +286,25 @@ def main():
retcode = process.wait()
if retcode == 0:
logging.info("Run successfully")
upload_corpus(fuzzers_path)
else:
logging.info("Run failed")
sys.exit(0)
results = process_results(result_path)
success = results[1] == 0 and results[2] == 0
JobReport(
description=f"OK: {results[0]}, ERROR: {results[1]}, FAIL: {results[2]}",
test_results=results[3],
status=SUCCESS if success else FAILURE,
start_time=stopwatch.start_time_str,
duration=stopwatch.duration_seconds,
additional_files=[],
).dump()
if not success:
sys.exit(1)
if __name__ == "__main__":

View File

@ -9,9 +9,10 @@ from get_robot_token import get_best_robot_token
from git_helper import commit as commit_arg
from github_helper import GitHub
from pr_info import PRInfo
from release import RELEASE_READY_STATUS
from report import SUCCESS
RELEASE_READY_STATUS = "Ready for release"
def main():
parser = argparse.ArgumentParser(

View File

@ -1,693 +0,0 @@
#!/usr/bin/env python3
"""
script to create releases for ClickHouse
The `gh` CLI preferred over the PyGithub to have an easy way to rollback bad
release in command line by simple execution giving rollback commands
On another hand, PyGithub is used for convenient getting commit's status from API
To run this script on a freshly installed Ubuntu 22.04 system, it is enough to do the following commands:
sudo apt install pip
pip install requests boto3 github PyGithub
sudo snap install gh
gh auth login
"""
import argparse
import json
import logging
import subprocess
from contextlib import contextmanager
from typing import Any, Final, Iterator, List, Optional, Tuple
from ci_config import Labels
from git_helper import Git, commit, release_branch
from report import SUCCESS
from version_helper import (
FILE_WITH_VERSION_PATH,
GENERATED_CONTRIBUTORS,
ClickHouseVersion,
VersionType,
get_abs_path,
get_version_from_repo,
update_cmake_version,
update_contributors,
)
RELEASE_READY_STATUS = "Ready for release"
class Repo:
VALID = ("ssh", "https", "origin")
def __init__(self, repo: str, protocol: str):
self._repo = repo
self._url = ""
self.url = protocol
@property
def url(self) -> str:
return self._url
@url.setter
def url(self, protocol: str) -> None:
if protocol == "ssh":
self._url = f"git@github.com:{self}.git"
elif protocol == "https":
self._url = f"https://github.com/{self}.git"
elif protocol == "origin":
self._url = protocol
else:
raise ValueError(f"protocol must be in {self.VALID}")
def __str__(self):
return self._repo
class Release:
NEW = "new" # type: Final
PATCH = "patch" # type: Final
VALID_TYPE = (NEW, PATCH) # type: Final[Tuple[str, str]]
CMAKE_PATH = get_abs_path(FILE_WITH_VERSION_PATH)
CONTRIBUTORS_PATH = get_abs_path(GENERATED_CONTRIBUTORS)
def __init__(
self,
repo: Repo,
release_commit: str,
release_type: str,
dry_run: bool,
with_stderr: bool,
):
self.repo = repo
self._release_commit = ""
self.release_commit = release_commit
self.dry_run = dry_run
self.with_stderr = with_stderr
assert release_type in self.VALID_TYPE
self.release_type = release_type
self._git = Git()
self._version = get_version_from_repo(git=self._git)
self.release_version = self.version
self._release_branch = ""
self._version_new_tag = None # type: Optional[ClickHouseVersion]
self._rollback_stack = [] # type: List[str]
def run(
self, cmd: str, cwd: Optional[str] = None, dry_run: bool = False, **kwargs: Any
) -> str:
cwd_text = ""
if cwd:
cwd_text = f" (CWD='{cwd}')"
if dry_run:
logging.info("Would run command%s:\n %s", cwd_text, cmd)
return ""
if not self.with_stderr:
kwargs["stderr"] = subprocess.DEVNULL
logging.info("Running command%s:\n %s", cwd_text, cmd)
return self._git.run(cmd, cwd, **kwargs)
def set_release_info(self):
# Fetch release commit and tags in case they don't exist locally
self.run(
f"git fetch {self.repo.url} {self.release_commit} --no-recurse-submodules"
)
self.run(f"git fetch {self.repo.url} --tags --no-recurse-submodules")
# Get the actual version for the commit before check
with self._checkout(self.release_commit, True):
self.release_branch = f"{self.version.major}.{self.version.minor}"
self.release_version = get_version_from_repo(git=self._git)
self.release_version.with_description(self.get_stable_release_type())
self.read_version()
def read_version(self):
self._git.update()
self.version = get_version_from_repo(git=self._git)
def get_stable_release_type(self) -> str:
if self.version.is_lts:
return VersionType.LTS
return VersionType.STABLE
def check_commit_release_ready(self):
per_page = 100
page = 1
while True:
statuses = json.loads(
self.run(
f"gh api 'repos/{self.repo}/commits/{self.release_commit}"
f"/statuses?per_page={per_page}&page={page}'"
)
)
if not statuses:
break
for status in statuses:
if status["context"] == RELEASE_READY_STATUS:
if not status["state"] == SUCCESS:
raise ValueError(
f"the status {RELEASE_READY_STATUS} is {status['state']}"
", not success"
)
return
page += 1
raise KeyError(
f"the status {RELEASE_READY_STATUS} "
f"is not found for commit {self.release_commit}"
)
def check_prerequisites(self):
"""
Check tooling installed in the system, `git` is checked by Git() init
"""
try:
self.run("gh auth status")
except subprocess.SubprocessError:
logging.error(
"The github-cli either not installed or not setup, please follow "
"the instructions on https://github.com/cli/cli#installation and "
"https://cli.github.com/manual/"
)
raise
if self.release_type == self.PATCH:
self.check_commit_release_ready()
def do(
self, check_dirty: bool, check_run_from_master: bool, check_branch: bool
) -> None:
self.check_prerequisites()
if check_dirty:
logging.info("Checking if repo is clean")
try:
self.run("git diff HEAD --exit-code")
except subprocess.CalledProcessError:
logging.fatal("Repo contains uncommitted changes")
raise
if check_run_from_master and self._git.branch != "master":
raise RuntimeError("the script must be launched only from master")
self.set_release_info()
if check_branch:
self.check_branch()
if self.release_type == self.NEW:
with self._checkout(self.release_commit, True):
# Checkout to the commit, it will provide the correct current version
with self.new_release():
with self.create_release_branch():
logging.info(
"Publishing release %s from commit %s is done",
self.release_version.describe,
self.release_commit,
)
elif self.release_type == self.PATCH:
with self._checkout(self.release_commit, True):
with self.patch_release():
logging.info(
"Publishing release %s from commit %s is done",
self.release_version.describe,
self.release_commit,
)
if self.dry_run:
logging.info("Dry running, clean out possible changes")
rollback = self._rollback_stack.copy()
rollback.reverse()
for cmd in rollback:
self.run(cmd)
return
self.log_post_workflows()
self.log_rollback()
def check_no_tags_after(self):
tags_after_commit = self.run(f"git tag --contains={self.release_commit}")
if tags_after_commit:
raise RuntimeError(
f"Commit {self.release_commit} belongs to following tags:\n"
f"{tags_after_commit}\nChoose another commit"
)
def check_branch(self):
branch = self.release_branch
if self.release_type == self.NEW:
# Commit to spin up the release must belong to a main branch
branch = "master"
elif self.release_type != self.PATCH:
raise (
ValueError(f"release_type {self.release_type} not in {self.VALID_TYPE}")
)
# Prefetch the branch to have it updated
if self._git.branch == branch:
self.run("git pull --no-recurse-submodules")
else:
self.run(
f"git fetch {self.repo.url} {branch}:{branch} --no-recurse-submodules"
)
output = self.run(f"git branch --contains={self.release_commit} {branch}")
if branch not in output:
raise RuntimeError(
f"commit {self.release_commit} must belong to {branch} "
f"for {self.release_type} release"
)
def _update_cmake_contributors(
self, version: ClickHouseVersion, reset_tweak: bool = True
) -> None:
if reset_tweak:
desc = version.description
version = version.reset_tweak()
version.with_description(desc)
update_cmake_version(version)
update_contributors(raise_error=True)
if self.dry_run:
logging.info(
"Dry running, resetting the following changes in the repo:\n%s",
self.run(f"git diff '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}'"),
)
self.run(f"git checkout '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}'")
def _commit_cmake_contributors(
self, version: ClickHouseVersion, reset_tweak: bool = True
) -> None:
if reset_tweak:
version = version.reset_tweak()
self.run(
f"git commit '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}' "
f"-m 'Update autogenerated version to {version.string} and contributors'",
dry_run=self.dry_run,
)
@property
def bump_part(self) -> ClickHouseVersion.PART_TYPE:
if self.release_type == Release.NEW:
if self._version.minor >= 12:
return "major"
return "minor"
return "patch"
@property
def has_rollback(self) -> bool:
return bool(self._rollback_stack)
def log_rollback(self):
if self.has_rollback:
rollback = self._rollback_stack.copy()
rollback.reverse()
logging.info(
"To rollback the action run the following commands:\n %s",
"\n ".join(rollback),
)
def log_post_workflows(self):
logging.info(
"To verify all actions are running good visit the following links:\n %s",
"\n ".join(
f"https://github.com/{self.repo}/actions/workflows/{action}.yml"
for action in ("release", "tags_stable")
),
)
@contextmanager
def create_release_branch(self):
self.check_no_tags_after()
# Create release branch
self.read_version()
assert self._version_new_tag is not None
with self._create_tag(
self._version_new_tag.describe,
self.release_commit,
f"Initial commit for release {self._version_new_tag.major}.{self._version_new_tag.minor}",
):
with self._create_branch(self.release_branch, self.release_commit):
with self._checkout(self.release_branch, True):
with self._bump_release_branch():
yield
@contextmanager
def patch_release(self):
self.check_no_tags_after()
self.read_version()
version_type = self.get_stable_release_type()
self.version.with_description(version_type)
with self._create_gh_release(False):
self.version = self.version.update(self.bump_part)
self.version.with_description(version_type)
self._update_cmake_contributors(self.version)
# Checking out the commit of the branch and not the branch itself,
# then we are able to skip rollback
with self._checkout(f"{self.release_branch}^0", False):
current_commit = self.run("git rev-parse HEAD")
self._commit_cmake_contributors(self.version)
with self._push(
"HEAD", with_rollback_on_fail=False, remote_ref=self.release_branch
):
# DO NOT PUT ANYTHING ELSE HERE
# The push must be the last action and mean the successful release
self._rollback_stack.append(
f"{self.dry_run_prefix}git push {self.repo.url} "
f"+{current_commit}:{self.release_branch}"
)
yield
@contextmanager
def new_release(self):
# Create branch for a version bump
self.read_version()
self.version = self.version.update(self.bump_part)
helper_branch = f"{self.version.major}.{self.version.minor}-prepare"
with self._create_branch(helper_branch, self.release_commit):
with self._checkout(helper_branch, True):
with self._bump_version_in_master(helper_branch):
yield
@property
def version(self) -> ClickHouseVersion:
return self._version
@version.setter
def version(self, version: ClickHouseVersion) -> None:
if not isinstance(version, ClickHouseVersion):
raise ValueError(f"version must be ClickHouseVersion, not {type(version)}")
self._version = version
@property
def release_branch(self) -> str:
return self._release_branch
@release_branch.setter
def release_branch(self, branch: str) -> None:
self._release_branch = release_branch(branch)
@property
def release_commit(self) -> str:
return self._release_commit
@release_commit.setter
def release_commit(self, release_commit: str) -> None:
self._release_commit = commit(release_commit)
@property
def dry_run_prefix(self) -> str:
if self.dry_run:
return "# "
return ""
@contextmanager
def _bump_release_branch(self):
# Update only git, original version stays the same
self._git.update()
new_version = self.version.copy()
version_type = self.get_stable_release_type()
pr_labels = f"--label {Labels.RELEASE}"
if version_type == VersionType.LTS:
pr_labels += f" --label {Labels.RELEASE_LTS}"
new_version.with_description(version_type)
self._update_cmake_contributors(new_version)
self._commit_cmake_contributors(new_version)
with self._push(self.release_branch):
with self._create_gh_label(
f"v{self.release_branch}-must-backport", "10dbed"
):
with self._create_gh_label(
f"v{self.release_branch}-affected", "c2bfff"
):
# The following command is rolled back by deleting branch
# in self._push
self.run(
f"gh pr create --repo {self.repo} --title "
f"'Release pull request for branch {self.release_branch}' "
f"--head {self.release_branch} {pr_labels} "
"--body 'This PullRequest is a part of ClickHouse release "
"cycle. It is used by CI system only. Do not perform any "
"changes with it.'",
dry_run=self.dry_run,
)
# Here the release branch part is done.
# We don't create a release itself automatically to have a
# safe window to backport possible bug fixes.
yield
@contextmanager
def _bump_version_in_master(self, helper_branch: str) -> Iterator[None]:
self.read_version()
self.version = self.version.update(self.bump_part)
self.version.with_description(VersionType.TESTING)
self._update_cmake_contributors(self.version)
self._commit_cmake_contributors(self.version)
# Create a version-new tag
self._version_new_tag = self.version.copy()
self._version_new_tag.tweak = 1
self._version_new_tag.with_description(VersionType.NEW)
with self._push(helper_branch):
body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md")
# The following command is rolled back by deleting branch in self._push
self.run(
f"gh pr create --repo {self.repo} --title 'Update version after "
f"release' --head {helper_branch} --body-file '{body_file}' "
"--label 'do not test' --assignee @me",
dry_run=self.dry_run,
)
# Here the new release part is done
yield
@contextmanager
def _checkout(self, ref: str, with_checkout_back: bool = False) -> Iterator[None]:
self._git.update()
orig_ref = self._git.branch or self._git.sha
rollback_cmd = ""
if ref not in (self._git.branch, self._git.sha):
self.run(f"git checkout {ref}")
# checkout is not put into rollback_stack intentionally
rollback_cmd = f"git checkout {orig_ref}"
# always update version and git after checked out ref
self.read_version()
try:
yield
except (Exception, KeyboardInterrupt):
logging.warning("Rolling back checked out %s for %s", ref, orig_ref)
self.run(f"git reset --hard; git checkout -f {orig_ref}")
raise
# Normal flow when we need to checkout back
if with_checkout_back and rollback_cmd:
self.run(rollback_cmd)
@contextmanager
def _create_branch(self, name: str, start_point: str = "") -> Iterator[None]:
self.run(f"git branch {name} {start_point}")
rollback_cmd = f"git branch -D {name}"
self._rollback_stack.append(rollback_cmd)
try:
yield
except (Exception, KeyboardInterrupt):
logging.warning("Rolling back created branch %s", name)
self.run(rollback_cmd)
raise
@contextmanager
def _create_gh_label(self, label: str, color_hex: str) -> Iterator[None]:
# API call, https://docs.github.com/en/rest/reference/issues#create-a-label
self.run(
f"gh api repos/{self.repo}/labels -f name={label} -f color={color_hex}",
dry_run=self.dry_run,
)
rollback_cmd = (
f"{self.dry_run_prefix}gh api repos/{self.repo}/labels/{label} -X DELETE"
)
self._rollback_stack.append(rollback_cmd)
try:
yield
except (Exception, KeyboardInterrupt):
logging.warning("Rolling back label %s", label)
self.run(rollback_cmd)
raise
@contextmanager
def _create_gh_release(self, as_prerelease: bool) -> Iterator[None]:
tag = self.release_version.describe
with self._create_tag(tag, self.release_commit):
# Preserve tag if version is changed
prerelease = ""
if as_prerelease:
prerelease = "--prerelease"
self.run(
f"gh release create {prerelease} --repo {self.repo} "
f"--title 'Release {tag}' '{tag}'",
dry_run=self.dry_run,
)
rollback_cmd = (
f"{self.dry_run_prefix}gh release delete --yes "
f"--repo {self.repo} '{tag}'"
)
self._rollback_stack.append(rollback_cmd)
try:
yield
except (Exception, KeyboardInterrupt):
logging.warning("Rolling back release publishing")
self.run(rollback_cmd)
raise
@contextmanager
def _create_tag(
self, tag: str, commit: str, tag_message: str = ""
) -> Iterator[None]:
tag_message = tag_message or f"Release {tag}"
# Create tag even in dry-run
self.run(f"git tag -a -m '{tag_message}' '{tag}' {commit}")
rollback_cmd = f"git tag -d '{tag}'"
self._rollback_stack.append(rollback_cmd)
try:
with self._push(tag):
yield
except (Exception, KeyboardInterrupt):
logging.warning("Rolling back tag %s", tag)
self.run(rollback_cmd)
raise
@contextmanager
def _push(
self, ref: str, with_rollback_on_fail: bool = True, remote_ref: str = ""
) -> Iterator[None]:
if remote_ref == "":
remote_ref = ref
self.run(f"git push {self.repo.url} {ref}:{remote_ref}", dry_run=self.dry_run)
if with_rollback_on_fail:
rollback_cmd = (
f"{self.dry_run_prefix}git push -d {self.repo.url} {remote_ref}"
)
self._rollback_stack.append(rollback_cmd)
try:
yield
except (Exception, KeyboardInterrupt):
if with_rollback_on_fail:
logging.warning("Rolling back pushed ref %s", ref)
self.run(rollback_cmd)
raise
def parse_args() -> argparse.Namespace:
parser = argparse.ArgumentParser(
formatter_class=argparse.ArgumentDefaultsHelpFormatter,
description="Script to release a new ClickHouse version, requires `git` and "
"`gh` (github-cli) commands "
"!!! LAUNCH IT ONLY FROM THE MASTER BRANCH !!!",
)
parser.add_argument(
"--commit",
required=True,
type=commit,
help="commit create a release",
)
parser.add_argument(
"--repo",
default="ClickHouse/ClickHouse",
help="repository to create the release",
)
parser.add_argument(
"--remote-protocol",
"-p",
default="ssh",
choices=Repo.VALID,
help="repo protocol for git commands remote, 'origin' is a special case and "
"uses 'origin' as a remote",
)
parser.add_argument(
"--type",
required=True,
choices=Release.VALID_TYPE,
dest="release_type",
help="a release type to bump the major.minor.patch version part, "
"new branch is created only for the value 'new'",
)
parser.add_argument("--with-release-branch", default=True, help=argparse.SUPPRESS)
parser.add_argument("--check-dirty", default=True, help=argparse.SUPPRESS)
parser.add_argument(
"--no-check-dirty",
dest="check_dirty",
action="store_false",
default=argparse.SUPPRESS,
help="(dangerous) if set, skip check repository for uncommitted changes",
)
parser.add_argument("--check-run-from-master", default=True, help=argparse.SUPPRESS)
parser.add_argument(
"--no-run-from-master",
dest="check_run_from_master",
action="store_false",
default=argparse.SUPPRESS,
help="(for development) if set, the script could run from non-master branch",
)
parser.add_argument("--check-branch", default=True, help=argparse.SUPPRESS)
parser.add_argument(
"--no-check-branch",
dest="check_branch",
action="store_false",
default=argparse.SUPPRESS,
help="(debug or development only, dangerous) if set, skip the branch check for "
"a run. By default, 'new' type work only for master, and 'patch' "
"works only for a release branches, that name "
"should be the same as '$MAJOR.$MINOR' version, e.g. 22.2",
)
parser.add_argument(
"--dry-run",
action="store_true",
help="do not make any actual changes in the repo, just show what will be done",
)
parser.add_argument(
"--with-stderr",
action="store_true",
help="if set, the stderr of all subprocess commands will be printed as well",
)
return parser.parse_args()
def main():
logging.basicConfig(level=logging.INFO)
args = parse_args()
repo = Repo(args.repo, args.remote_protocol)
release = Release(
repo, args.commit, args.release_type, args.dry_run, args.with_stderr
)
try:
release.do(args.check_dirty, args.check_run_from_master, args.check_branch)
except:
if release.has_rollback:
logging.error(
"!!The release process finished with error, read the output carefully!!"
)
logging.error(
"Probably, rollback finished with error. "
"If you don't see any of the following commands in the output, "
"execute them manually:"
)
release.log_rollback()
raise
if __name__ == "__main__":
assert False, "Script Deprecated, ask ci team for help"
main()

View File

@ -311,24 +311,32 @@ class S3Helper:
def list_prefix(
self, s3_prefix_path: str, bucket: str = S3_BUILDS_BUCKET
) -> List[str]:
objects = self.client.list_objects_v2(Bucket=bucket, Prefix=s3_prefix_path)
paginator = self.client.get_paginator("list_objects_v2")
pages = paginator.paginate(Bucket=bucket, Prefix=s3_prefix_path)
result = []
if "Contents" in objects:
for obj in objects["Contents"]:
result.append(obj["Key"])
for page in pages:
if "Contents" in page:
for obj in page["Contents"]:
result.append(obj["Key"])
return result
def list_prefix_non_recursive(
self, s3_prefix_path: str, bucket: str = S3_BUILDS_BUCKET
self,
s3_prefix_path: str,
bucket: str = S3_BUILDS_BUCKET,
only_dirs: bool = False,
) -> List[str]:
objects = self.client.list_objects_v2(Bucket=bucket, Prefix=s3_prefix_path)
paginator = self.client.get_paginator("list_objects_v2")
pages = paginator.paginate(Bucket=bucket, Prefix=s3_prefix_path, Delimiter="/")
result = []
if "Contents" in objects:
for obj in objects["Contents"]:
if "/" not in obj["Key"][len(s3_prefix_path) + 1 :]:
for page in pages:
if not only_dirs and "Contents" in page:
for obj in page["Contents"]:
result.append(obj["Key"])
if "CommonPrefixes" in page:
for obj in page["CommonPrefixes"]:
result.append(obj["Prefix"])
return result
def url_if_exists(self, key: str, bucket: str = S3_BUILDS_BUCKET) -> str:

View File

@ -789,6 +789,7 @@ def get_localzone():
return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:]))
# Refer to `tests/integration/helpers/random_settings.py` for integration test random settings
class SettingsRandomizer:
settings = {
"max_insert_threads": lambda: (
@ -919,6 +920,9 @@ class SettingsRandomizer:
"max_parsing_threads": lambda: random.choice([0, 1, 10]),
"optimize_functions_to_subcolumns": lambda: random.randint(0, 1),
"parallel_replicas_local_plan": lambda: random.randint(0, 1),
"query_plan_join_inner_table_selection": lambda: random.choice(
["left", "auto", "right"]
),
"output_format_native_write_json_as_string": lambda: random.randint(0, 1),
"enable_vertical_final": lambda: random.randint(0, 1),
}

View File

@ -1,5 +1,8 @@
#!/bin/bash -eu
# rename clickhouse
mv $OUT/clickhouse $OUT/clickhouse_fuzzer
# copy fuzzer options and dictionaries
cp $SRC/tests/fuzz/*.dict $OUT/
cp $SRC/tests/fuzz/*.options $OUT/

View File

@ -0,0 +1,2 @@
[CI]
FUZZER_ARGS = true

View File

@ -1,26 +1,49 @@
#!/usr/bin/env python3
import configparser
import datetime
import logging
import os
import subprocess
from pathlib import Path
DEBUGGER = os.getenv("DEBUGGER", "")
FUZZER_ARGS = os.getenv("FUZZER_ARGS", "")
TIMEOUT = int(os.getenv("TIMEOUT", "0"))
OUTPUT = "/test_output"
def run_fuzzer(fuzzer: str):
class Stopwatch:
def __init__(self):
self.reset()
@property
def duration_seconds(self) -> float:
return (datetime.datetime.utcnow() - self.start_time).total_seconds()
@property
def start_time_str(self) -> str:
return self.start_time_str_value
def reset(self) -> None:
self.start_time = datetime.datetime.utcnow()
self.start_time_str_value = self.start_time.strftime("%Y-%m-%d %H:%M:%S")
def run_fuzzer(fuzzer: str, timeout: int):
logging.info("Running fuzzer %s...", fuzzer)
corpus_dir = f"{fuzzer}.in"
with Path(corpus_dir) as path:
seed_corpus_dir = f"{fuzzer}.in"
with Path(seed_corpus_dir) as path:
if not path.exists() or not path.is_dir():
corpus_dir = ""
seed_corpus_dir = ""
active_corpus_dir = f"corpus/{fuzzer}"
if not os.path.exists(active_corpus_dir):
os.makedirs(active_corpus_dir)
options_file = f"{fuzzer}.options"
custom_libfuzzer_options = ""
fuzzer_arguments = ""
use_fuzzer_args = False
with Path(options_file) as path:
if path.exists() and path.is_file():
@ -44,7 +67,9 @@ def run_fuzzer(fuzzer: str):
if parser.has_section("libfuzzer"):
custom_libfuzzer_options = " ".join(
f"-{key}={value}" for key, value in parser["libfuzzer"].items()
f"-{key}={value}"
for key, value in parser["libfuzzer"].items()
if key not in ("jobs", "exact_artifact_path")
)
if parser.has_section("fuzzer_arguments"):
@ -53,19 +78,70 @@ def run_fuzzer(fuzzer: str):
for key, value in parser["fuzzer_arguments"].items()
)
cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {corpus_dir}"
if custom_libfuzzer_options:
cmd_line += f" {custom_libfuzzer_options}"
if fuzzer_arguments:
cmd_line += f" {fuzzer_arguments}"
use_fuzzer_args = parser.getboolean("CI", "FUZZER_ARGS", fallback=False)
if not "-dict=" in cmd_line and Path(f"{fuzzer}.dict").exists():
cmd_line += f" -dict={fuzzer}.dict"
exact_artifact_path = f"{OUTPUT}/{fuzzer}.unit"
status_path = f"{OUTPUT}/{fuzzer}.status"
out_path = f"{OUTPUT}/{fuzzer}.out"
stdout_path = f"{OUTPUT}/{fuzzer}.stdout"
cmd_line += " < /dev/null"
if not "-dict=" in custom_libfuzzer_options and Path(f"{fuzzer}.dict").exists():
custom_libfuzzer_options += f" -dict={fuzzer}.dict"
custom_libfuzzer_options += f" -exact_artifact_path={exact_artifact_path}"
logging.info("...will execute: %s", cmd_line)
subprocess.check_call(cmd_line, shell=True)
libfuzzer_corpora = f"{active_corpus_dir} {seed_corpus_dir}"
cmd_line = f"{DEBUGGER} ./{fuzzer} {fuzzer_arguments}"
env = None
with_fuzzer_args = ""
if use_fuzzer_args:
env = {"FUZZER_ARGS": f"{custom_libfuzzer_options} {libfuzzer_corpora}".strip()}
with_fuzzer_args = f" with FUZZER_ARGS '{env['FUZZER_ARGS']}'"
else:
cmd_line += f" {custom_libfuzzer_options} {libfuzzer_corpora}"
logging.info("...will execute: '%s'%s", cmd_line, with_fuzzer_args)
stopwatch = Stopwatch()
try:
with open(out_path, "wb") as out, open(stdout_path, "wb") as stdout:
subprocess.run(
cmd_line.split(),
stdin=subprocess.DEVNULL,
stdout=stdout,
stderr=out,
text=True,
check=True,
shell=False,
errors="replace",
timeout=timeout,
env=env,
)
except subprocess.CalledProcessError:
logging.info("Fail running %s", fuzzer)
with open(status_path, "w", encoding="utf-8") as status:
status.write(
f"FAIL\n{stopwatch.start_time_str}\n{stopwatch.duration_seconds}\n"
)
except subprocess.TimeoutExpired:
logging.info("Successful running %s", fuzzer)
with open(status_path, "w", encoding="utf-8") as status:
status.write(
f"OK\n{stopwatch.start_time_str}\n{stopwatch.duration_seconds}\n"
)
except Exception as e:
logging.info("Unexpected exception running %s: %s", fuzzer, e)
with open(status_path, "w", encoding="utf-8") as status:
status.write(
f"ERROR\n{stopwatch.start_time_str}\n{stopwatch.duration_seconds}\n"
)
else:
logging.info("Error running %s", fuzzer)
with open(status_path, "w", encoding="utf-8") as status:
status.write(
f"ERROR\n{stopwatch.start_time_str}\n{stopwatch.duration_seconds}\n"
)
def main():
@ -73,10 +149,14 @@ def main():
subprocess.check_call("ls -al", shell=True)
timeout = 30 if TIMEOUT == 0 else TIMEOUT
with Path() as current:
for fuzzer in current.iterdir():
if (current / fuzzer).is_file() and os.access(current / fuzzer, os.X_OK):
run_fuzzer(fuzzer)
run_fuzzer(fuzzer.name, timeout)
subprocess.check_call(f"ls -al {OUTPUT}", shell=True)
if __name__ == "__main__":

View File

@ -67,6 +67,7 @@ DEFAULT_ENV_NAME = ".env"
DEFAULT_BASE_CONFIG_DIR = os.environ.get(
"CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/"
)
DOCKER_BASE_TAG = os.environ.get("DOCKER_BASE_TAG", "latest")
SANITIZER_SIGN = "=================="
@ -503,7 +504,6 @@ class ClickHouseCluster:
"CLICKHOUSE_TESTS_DOCKERD_HOST"
)
self.docker_api_version = os.environ.get("DOCKER_API_VERSION")
self.docker_base_tag = os.environ.get("DOCKER_BASE_TAG", "latest")
self.base_cmd = ["docker", "compose"]
if custom_dockerd_host:
@ -1079,7 +1079,7 @@ class ClickHouseCluster:
env_variables["keeper_binary"] = binary_path
env_variables["keeper_cmd_prefix"] = keeper_cmd_prefix
env_variables["image"] = "clickhouse/integration-test:" + self.docker_base_tag
env_variables["image"] = "clickhouse/integration-test:" + DOCKER_BASE_TAG
env_variables["user"] = str(os.getuid())
env_variables["keeper_fs"] = "bind"
for i in range(1, 4):
@ -1675,7 +1675,7 @@ class ClickHouseCluster:
)
if tag is None:
tag = self.docker_base_tag
tag = DOCKER_BASE_TAG
if not env_variables:
env_variables = {}
self.use_keeper = use_keeper
@ -4538,7 +4538,12 @@ class ClickHouseInstance:
if len(self.custom_dictionaries_paths):
write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir)
if self.randomize_settings and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR:
if (
self.randomize_settings
and self.image == "clickhouse/integration-test"
and self.tag == DOCKER_BASE_TAG
and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR
):
# If custom main config is used, do not apply random settings to it
write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml")

View File

@ -5,6 +5,8 @@ def randomize_settings():
yield "max_joined_block_size_rows", random.randint(8000, 100000)
if random.random() < 0.5:
yield "max_block_size", random.randint(8000, 100000)
if random.random() < 0.5:
yield "query_plan_join_inner_table_selection", random.choice(["auto", "left"])
def write_random_settings_config(destination):

View File

@ -91,7 +91,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster):
with client(name="client1>", log=client_output, command=command_text) as client1:
client1.expect(prompt)
client1.send(
"SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number",
"SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS query_plan_join_inner_table_selection = 'right'",
)
client1.expect("Peak memory usage", timeout=60)
client1.expect(prompt)

View File

@ -386,100 +386,6 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('
conn.close()
def test_mysql_distributed(started_cluster):
table_name = "test_replicas"
conn1 = get_mysql_conn(started_cluster, started_cluster.mysql8_ip)
conn2 = get_mysql_conn(started_cluster, started_cluster.mysql2_ip)
conn3 = get_mysql_conn(started_cluster, started_cluster.mysql3_ip)
conn4 = get_mysql_conn(started_cluster, started_cluster.mysql4_ip)
create_mysql_db(conn1, "clickhouse")
create_mysql_db(conn2, "clickhouse")
create_mysql_db(conn3, "clickhouse")
create_mysql_db(conn4, "clickhouse")
create_mysql_table(conn1, table_name)
create_mysql_table(conn2, table_name)
create_mysql_table(conn3, table_name)
create_mysql_table(conn4, table_name)
node2.query("DROP TABLE IF EXISTS test_replicas")
# Storage with with 3 replicas
node2.query(
"""
CREATE TABLE test_replicas
(id UInt32, name String, age UInt32, money UInt32)
ENGINE = MySQL('mysql{2|3|4}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
)
# Fill remote tables with different data to be able to check
nodes = [node1, node2, node2, node2]
for i in range(1, 5):
nodes[i - 1].query("DROP TABLE IF EXISTS test_replica{}".format(i))
nodes[i - 1].query(
"""
CREATE TABLE test_replica{}
(id UInt32, name String, age UInt32, money UInt32)
ENGINE = MySQL('mysql{}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse');""".format(
i, 80 if i == 1 else i
)
)
nodes[i - 1].query(
"INSERT INTO test_replica{} (id, name) SELECT number, 'host{}' from numbers(10) ".format(
i, i
)
)
# test multiple ports parsing
result = node2.query(
"""SELECT DISTINCT(name) FROM mysql('mysql{80|2|3}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
)
assert result == "host1\n" or result == "host2\n" or result == "host3\n"
result = node2.query(
"""SELECT DISTINCT(name) FROM mysql('mysql80:3306|mysql2:3306|mysql3:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
)
assert result == "host1\n" or result == "host2\n" or result == "host3\n"
# check all replicas are traversed
query = "SELECT * FROM ("
for i in range(3):
query += "SELECT name FROM test_replicas UNION DISTINCT "
query += "SELECT name FROM test_replicas) ORDER BY name"
result = node2.query(query)
assert result == "host2\nhost3\nhost4\n"
# Storage with with two shards, each has 2 replicas
node2.query("DROP TABLE IF EXISTS test_shards")
node2.query(
"""
CREATE TABLE test_shards
(id UInt32, name String, age UInt32, money UInt32)
ENGINE = ExternalDistributed('MySQL', 'mysql{80|2}:3306,mysql{3|4}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
)
# Check only one replica in each shard is used
result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
assert result == "host1\nhost3\n"
# check all replicas are traversed
query = "SELECT name FROM ("
for i in range(3):
query += "SELECT name FROM test_shards UNION DISTINCT "
query += "SELECT name FROM test_shards) ORDER BY name"
result = node2.query(query)
assert result == "host1\nhost2\nhost3\nhost4\n"
# disconnect mysql
started_cluster.pause_container("mysql80")
result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
started_cluster.unpause_container("mysql80")
assert result == "host2\nhost4\n" or result == "host3\nhost4\n"
def test_external_settings(started_cluster):
table_name = "test_external_settings"
node1.query(f"DROP TABLE IF EXISTS {table_name}")

View File

@ -449,89 +449,6 @@ def test_concurrent_queries(started_cluster):
node1.query("DROP TABLE test.stat;")
def test_postgres_distributed(started_cluster):
cursor0 = started_cluster.postgres_conn.cursor()
cursor1 = started_cluster.postgres2_conn.cursor()
cursor2 = started_cluster.postgres3_conn.cursor()
cursor3 = started_cluster.postgres4_conn.cursor()
cursors = [cursor0, cursor1, cursor2, cursor3]
for i in range(4):
cursors[i].execute("DROP TABLE IF EXISTS test_replicas")
cursors[i].execute("CREATE TABLE test_replicas (id Integer, name Text)")
cursors[i].execute(
f"""INSERT INTO test_replicas select i, 'host{i+1}' from generate_series(0, 99) as t(i);"""
)
# test multiple ports parsing
result = node2.query(
"""SELECT DISTINCT(name) FROM postgresql('postgres{1|2|3}:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """
)
assert result == "host1\n" or result == "host2\n" or result == "host3\n"
result = node2.query(
"""SELECT DISTINCT(name) FROM postgresql('postgres2:5431|postgres3:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """
)
assert result == "host3\n" or result == "host2\n"
# Create storage with with 3 replicas
node2.query("DROP TABLE IF EXISTS test_replicas")
node2.query(
"""
CREATE TABLE test_replicas
(id UInt32, name String)
ENGINE = PostgreSQL('postgres{2|3|4}:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """
)
# Check all replicas are traversed
query = "SELECT name FROM ("
for i in range(3):
query += "SELECT name FROM test_replicas UNION DISTINCT "
query += "SELECT name FROM test_replicas) ORDER BY name"
result = node2.query(query)
assert result == "host2\nhost3\nhost4\n"
# Create storage with with two two shards, each has 2 replicas
node2.query("DROP TABLE IF EXISTS test_shards")
node2.query(
"""
CREATE TABLE test_shards
(id UInt32, name String, age UInt32, money UInt32)
ENGINE = ExternalDistributed('PostgreSQL', 'postgres{1|2}:5432,postgres{3|4}:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """
)
# Check only one replica in each shard is used
result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
assert result == "host1\nhost3\n"
node2.query(
"""
CREATE TABLE test_shards2
(id UInt32, name String, age UInt32, money UInt32)
ENGINE = ExternalDistributed('PostgreSQL', postgres4, addresses_expr='postgres{1|2}:5432,postgres{3|4}:5432'); """
)
result = node2.query("SELECT DISTINCT(name) FROM test_shards2 ORDER BY name")
assert result == "host1\nhost3\n"
# Check all replicas are traversed
query = "SELECT name FROM ("
for i in range(3):
query += "SELECT name FROM test_shards UNION DISTINCT "
query += "SELECT name FROM test_shards) ORDER BY name"
result = node2.query(query)
assert result == "host1\nhost2\nhost3\nhost4\n"
# Disconnect postgres1
started_cluster.pause_container("postgres1")
result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
started_cluster.unpause_container("postgres1")
assert result == "host2\nhost4\n" or result == "host3\nhost4\n"
node2.query("DROP TABLE test_shards2")
node2.query("DROP TABLE test_shards")
node2.query("DROP TABLE test_replicas")
def test_datetime_with_timezone(started_cluster):
cursor = started_cluster.postgres_conn.cursor()
cursor.execute("DROP TABLE IF EXISTS test_timezone")
@ -850,6 +767,7 @@ def test_filter_pushdown(started_cluster):
"INSERT INTO test_filter_pushdown.test_table VALUES (1, 10), (1, 110), (2, 0), (3, 33), (4, 0)"
)
node1.query("DROP TABLE IF EXISTS test_filter_pushdown_pg_table")
node1.query(
"""
CREATE TABLE test_filter_pushdown_pg_table (id UInt32, value UInt32)
@ -857,12 +775,14 @@ def test_filter_pushdown(started_cluster):
"""
)
node1.query("DROP TABLE IF EXISTS test_filter_pushdown_local_table")
node1.query(
"""
CREATE TABLE test_filter_pushdown_local_table (id UInt32, value UInt32) ENGINE Memory AS SELECT * FROM test_filter_pushdown_pg_table
"""
)
node1.query("DROP TABLE IF EXISTS ch_table")
node1.query(
"CREATE TABLE ch_table (id UInt32, pg_id UInt32) ENGINE MergeTree ORDER BY id"
)

View File

@ -15,9 +15,9 @@ INSERT INTO t2_00826 values (1,1), (1,2);
INSERT INTO t2_00826 (a) values (2), (3);
SELECT '--- cross ---';
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL;
SELECT '--- cross nullable ---';
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b;
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL;
SELECT '--- cross nullable vs not nullable ---';
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b ORDER BY t1_00826.a;
SELECT '--- cross self ---';
@ -41,14 +41,15 @@ SELECT '--- is null or ---';
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a;
SELECT '--- do not rewrite alias ---';
SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0;
SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0 ORDER BY ALL;
SELECT '--- comma ---';
SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a;
SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL;
SELECT '--- comma nullable ---';
SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b;
SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL;
SELECT '--- comma and or ---';
SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2);
SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2)
ORDER BY ALL;
SELECT '--- cross ---';

View File

@ -20,42 +20,42 @@ select t.a, s.b, s.a, s.b, y.a, y.b from t
left join s on (t.a = s.a and s.b = t.b)
left join y on (y.a = s.a and y.b = s.b)
order by t.a
format PrettyCompactNoEscapes;
format PrettyCompactMonoBlock;
select t.a as t_a from t
left join s on s.a = t_a
order by t.a
format PrettyCompactNoEscapes;
format PrettyCompactMonoBlock;
select t.a, s.a as s_a from t
left join s on s.a = t.a
left join y on y.b = s.b
order by t.a
format PrettyCompactNoEscapes;
format PrettyCompactMonoBlock;
select t.a, t.a, t.b as t_b from t
left join s on t.a = s.a
left join y on y.b = s.b
order by t.a
format PrettyCompactNoEscapes;
format PrettyCompactMonoBlock;
select s.a, s.a, s.b as s_b, s.b from t
left join s on s.a = t.a
left join y on s.b = y.b
order by t.a
format PrettyCompactNoEscapes;
format PrettyCompactMonoBlock;
select y.a, y.a, y.b as y_b, y.b from t
left join s on s.a = t.a
left join y on y.b = s.b
order by t.a
format PrettyCompactNoEscapes;
format PrettyCompactMonoBlock;
select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t
left join s on t.a = s.a
left join y on y.b = s.b
order by t.a
format PrettyCompactNoEscapes;
format PrettyCompactMonoBlock;
drop table t;
drop table s;

View File

@ -1,5 +1,7 @@
SET joined_subquery_requires_alias = 0;
SET query_plan_join_inner_table_selection = 'auto';
{% for join_algorithm in ['partial_merge', 'hash'] -%}
SET join_algorithm = '{{ join_algorithm }}';

View File

@ -18,28 +18,35 @@
0 0
0 0
-
1 1
1 1
0 0
-
1 1
1 1
0 0
-
1 1
1 1
0 0
-
1 1
1 1
0 0
-
1 1
0 0
-
1 foo 1 1 300
0 foo 1 0 300
-
1 100 1970-01-01 1 100 1970-01-01
1 100 1970-01-01 1 200 1970-01-02
1 200 1970-01-02 1 100 1970-01-01

View File

@ -64,39 +64,47 @@ USING (id);
INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02');
SELECT '-';
SELECT *
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l
LEFT JOIN (SELECT item_id FROM t ) r
ON l.item_id = r.item_id;
SELECT '-';
SELECT *
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l
RIGHT JOIN (SELECT item_id FROM t ) r
ON l.item_id = r.item_id;
SELECT '-';
SELECT *
FROM (SELECT item_id FROM t) l
LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r
ON l.item_id = r.item_id;
SELECT '-';
SELECT *
FROM (SELECT item_id FROM t) l
RIGHT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r
ON l.item_id = r.item_id;
SELECT '-';
SELECT *
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l
LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r
ON l.item_id = r.item_id;
SELECT '-';
SELECT *
FROM (SELECT item_id, 'foo' AS key, 1 AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l
LEFT JOIN (SELECT item_id, sum(price_sold) AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r
ON l.item_id = r.item_id;
SELECT '-';
SELECT *
FROM (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date) l
LEFT JOIN (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date ) r
ON l.item_id = r.item_id;
ON l.item_id = r.item_id
ORDER BY ALL;
DROP TABLE t;

View File

@ -26,7 +26,7 @@ Expression ((Projection + Before ORDER BY))
Parts: 1/1
Granules: 1/1
Expression ((Project names + Projection))
Filter ((WHERE + DROP unused columns after JOIN))
Filter (WHERE)
Join (JOIN FillRightFirst)
Expression
ReadFromMergeTree (default.t1)

View File

@ -75,7 +75,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverErro
SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id;
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id ORDER BY ALL;
-- non-equi condition containing columns from different tables doesn't supported yet
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 }
SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 }
@ -89,10 +89,10 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22
SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 }
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 }
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 }
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id;
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2;
SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd;
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd;
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id ORDER BY ALL;
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2 ORDER BY ALL;
SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd ORDER BY ALL;
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd ORDER BY ALL;
{% endfor -%}

View File

@ -33,23 +33,23 @@
2 2
2 2
-- { echoOn }
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1;
1 0
2 2
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
2 2
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1;
0 3
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
2 2
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1;
1 0
2 2
0 3
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1;
1 0
2 0
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1;
0 2
0 3
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1;
1 0
2 0
0 2
@ -59,11 +59,11 @@ SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2
1 ('',0)
SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
0 ('b',256)
SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2;
1 ('',0)
0 ('b',256)
SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2;
1 ('',0)
2
4 2 Nullable(UInt64) UInt8

View File

@ -73,20 +73,20 @@ SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 0; --
SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1;
-- { echoOn }
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1;
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1;
SELECT * FROM (SELECT 1 as a) as t1 INNER JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2;
SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2;
-- { echoOff }

View File

@ -1,8 +1,8 @@
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
CREATE TABLE t1 (id Int) ENGINE = MergeTree ORDER BY id;
CREATE TABLE t2 (id Int) ENGINE = MergeTree ORDER BY id;
CREATE TABLE t1 (id Int) ENGINE = TinyLog;
CREATE TABLE t2 (id Int) ENGINE = TinyLog;
INSERT INTO t1 VALUES (1), (2);
INSERT INTO t2 SELECT number + 5 AS x FROM (SELECT * FROM system.numbers LIMIT 1111);

View File

@ -12,8 +12,9 @@ CREATE TABLE without_nullable
insert into with_nullable values(0,'f'),(0,'usa');
insert into without_nullable values(0,'usa'),(0,'us2a');
select if(t0.country is null ,t2.country,t0.country) "country"
from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country;
select if(t0.country is null ,t2.country,t0.country) "country"
from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country
ORDER BY 1 DESC;
drop table with_nullable;
drop table without_nullable;

View File

@ -48,7 +48,8 @@ SELECT
L2SquaredDistance(v1.v, v2.v),
cosineDistance(v1.v, v2.v)
FROM vec2 v1, vec2 v2
WHERE length(v1.v) == length(v2.v);
WHERE length(v1.v) == length(v2.v)
ORDER BY ALL;
INSERT INTO vec2f VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1));
SELECT
@ -61,7 +62,8 @@ SELECT
L2SquaredDistance(v1.v, v2.v),
cosineDistance(v1.v, v2.v)
FROM vec2f v1, vec2f v2
WHERE length(v1.v) == length(v2.v);
WHERE length(v1.v) == length(v2.v)
ORDER BY ALL;
INSERT INTO vec2d VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1));
SELECT
@ -74,7 +76,8 @@ SELECT
L2SquaredDistance(v1.v, v2.v),
cosineDistance(v1.v, v2.v)
FROM vec2d v1, vec2d v2
WHERE length(v1.v) == length(v2.v);
WHERE length(v1.v) == length(v2.v)
ORDER BY ALL;
SELECT
v1.id,
@ -86,7 +89,8 @@ SELECT
L2SquaredDistance(v1.v, v2.v),
cosineDistance(v1.v, v2.v)
FROM vec2f v1, vec2d v2
WHERE length(v1.v) == length(v2.v);
WHERE length(v1.v) == length(v2.v)
ORDER BY ALL;
SELECT L1Distance([0, 0], [1]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
SELECT L2Distance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }

View File

@ -148,7 +148,6 @@ Header: key String
value String
Join
Header: __table1.key String
__table3.key String
__table3.value String
Sorting
Header: __table1.key String

View File

@ -9,4 +9,5 @@ CREATE TABLE t2__fuzz_47 (id LowCardinality(Int16)) ENGINE = MergeTree() ORDER B
INSERT INTO t1__fuzz_13 VALUES (1);
INSERT INTO t2__fuzz_47 VALUES (1);
SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2;
SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2
ORDER BY ALL;

View File

@ -117,7 +117,7 @@ ORDER BY t1.number, t2.number
-- explain
Expression (Project names)
Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN)))
Expression ((Before ORDER BY + Projection))
Join (JOIN FillRightFirst)
Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))))
ReadFromSystemNumbers
@ -161,7 +161,7 @@ ORDER BY t1.number, t2.number
-- explain
Expression (Project names)
Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN)))
Expression ((Before ORDER BY + Projection))
Join (JOIN FillRightFirst)
Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))))
ReadFromSystemNumbers

View File

@ -79,7 +79,7 @@ Expression (Project names)
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
Distinct (Preliminary DISTINCT)
Expression ((Projection + DROP unused columns after JOIN))
Expression (Projection)
Join (JOIN FillRightFirst)
Expression ((Change column names to column identifiers + Project names))
Distinct (DISTINCT)
@ -244,7 +244,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + Projection))
Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers)
ReadFromSystemNumbers
@ -280,7 +280,7 @@ Expression (Project names)
Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + Projection))
Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers)
ReadFromSystemNumbers
@ -315,7 +315,7 @@ Expression (Project names)
Expression ((Before ORDER BY + Projection))
Rollup
Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers)
ReadFromSystemNumbers
@ -348,7 +348,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
Expression ((Before ORDER BY + Projection))
Rollup
Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers)
ReadFromSystemNumbers
@ -386,7 +386,7 @@ Expression (Project names)
Expression ((Before ORDER BY + Projection))
Cube
Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers)
ReadFromSystemNumbers
@ -419,7 +419,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
Expression ((Before ORDER BY + Projection))
Cube
Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers)
ReadFromSystemNumbers
@ -457,7 +457,7 @@ Expression (Project names)
Expression ((Before ORDER BY + Projection))
TotalsHaving
Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers)
ReadFromSystemNumbers
@ -491,7 +491,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
Expression ((Before ORDER BY + Projection))
TotalsHaving
Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers)
ReadFromSystemNumbers

Some files were not shown because too many files have changed in this diff Show More