mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge remote-tracking branch 'blessed/master' into i71382
This commit is contained in:
commit
097fb78a6e
@ -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
|
||||
|
@ -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 | ❌ |
|
||||
|
@ -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
2
contrib/krb5
vendored
@ -1 +1 @@
|
||||
Subproject commit 71b06c2276009ae649c7703019f3b4605f66fd3d
|
||||
Subproject commit c5b4b994c18db86933255907a97eee5993fd18fe
|
@ -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=""
|
||||
|
||||
|
@ -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=""
|
||||
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
412
docs/changelogs/v24.10.1.2812-stable.md
Normal file
412
docs/changelogs/v24.10.1.2812-stable.md
Normal file
File diff suppressed because one or more lines are too long
50
docs/changelogs/v24.8.6.70-lts.md
Normal file
50
docs/changelogs/v24.8.6.70-lts.md
Normal file
File diff suppressed because one or more lines are too long
@ -1353,9 +1353,11 @@ try
|
||||
}
|
||||
|
||||
FailPointInjection::enableFromGlobalConfig(config());
|
||||
#endif
|
||||
|
||||
memory_worker.start();
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
int default_oom_score = 0;
|
||||
|
||||
#if !defined(NDEBUG)
|
||||
|
@ -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()}},
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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.
|
||||
|
@ -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,
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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) \
|
||||
|
@ -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) \
|
||||
|
@ -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",
|
||||
|
@ -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},
|
||||
|
@ -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.
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
|
||||
#include <Common/MemoryTracker.h>
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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_))
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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_,
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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; }
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
|
||||
|
@ -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)
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
#include <Analyzer/JoinNode.h>
|
||||
|
||||
#include <Planner/PlannerContext.h>
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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()
|
||||
|
@ -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.
|
||||
|
@ -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.
|
||||
|
102
src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp
Normal file
102
src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
@ -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())
|
||||
{
|
||||
|
@ -35,6 +35,8 @@ public:
|
||||
|
||||
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
const StoragePtr & getStorage() const { return storage; }
|
||||
|
||||
private:
|
||||
static constexpr auto name = "ReadFromMemoryStorage";
|
||||
|
||||
|
49
src/Processors/Transforms/ColumnPermuteTransform.cpp
Normal file
49
src/Processors/Transforms/ColumnPermuteTransform.cpp
Normal 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);
|
||||
}
|
||||
|
||||
|
||||
}
|
30
src/Processors/Transforms/ColumnPermuteTransform.h
Normal file
30
src/Processors/Transforms/ColumnPermuteTransform.h
Normal 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;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -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;
|
||||
}
|
||||
|
@ -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.
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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(); }
|
||||
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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,
|
||||
});
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
@ -155,7 +155,7 @@ StorageMergeTree::StorageMergeTree(
|
||||
|
||||
loadMutations();
|
||||
loadDeduplicationLog();
|
||||
prewarmMarkCache(getActivePartsLoadingThreadPool().get());
|
||||
prewarmMarkCacheIfNeeded(getActivePartsLoadingThreadPool().get());
|
||||
}
|
||||
|
||||
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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>
|
||||
|
@ -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]
|
||||
|
@ -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__":
|
||||
|
@ -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(
|
||||
|
@ -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()
|
@ -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:
|
||||
|
@ -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),
|
||||
}
|
||||
|
@ -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/
|
||||
|
2
tests/fuzz/clickhouse_fuzzer.options
Normal file
2
tests/fuzz/clickhouse_fuzzer.options
Normal file
@ -0,0 +1,2 @@
|
||||
[CI]
|
||||
FUZZER_ARGS = true
|
@ -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__":
|
||||
|
@ -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")
|
||||
|
||||
|
@ -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):
|
||||
|
@ -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)
|
||||
|
@ -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}")
|
||||
|
@ -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"
|
||||
)
|
||||
|
@ -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 ---';
|
||||
|
@ -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;
|
||||
|
@ -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 }}';
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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 -%}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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 }
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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 }
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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
Loading…
Reference in New Issue
Block a user