Merge remote-tracking branch 'origin/master' into pr-right-joins

This commit is contained in:
Igor Nikonov 2024-11-04 22:45:58 +00:00
commit 5fbced454d
81 changed files with 1288 additions and 341 deletions

View File

@ -86,7 +86,7 @@ using StringRefs = std::vector<StringRef>;
* For more information, see hash_map_string_2.cpp * 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( return 0xFFFF == _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p1)), _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) #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( uint64_t mask = getNibbleMask(vceqq_u8(
vld1q_u8(reinterpret_cast<const unsigned char *>(p1)), vld1q_u8(reinterpret_cast<const unsigned char *>(p2)))); 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) switch (size / 16) // NOLINT(bugprone-switch-missing-default-case)
{ {
case 3: if (!compare8(p1 + 32, p2 + 32)) return false; [[fallthrough]]; case 3:
case 2: if (!compare8(p1 + 16, p2 + 16)) return false; [[fallthrough]]; if (!compare16(p1 + 32, p2 + 32))
case 1: if (!compare8(p1, p2)) return false; [[fallthrough]]; return false;
[[fallthrough]];
case 2:
if (!compare16(p1 + 16, p2 + 16))
return false;
[[fallthrough]];
case 1:
if (!compare16(p1, p2))
return false;
[[fallthrough]];
default: ; default: ;
} }
return compare8(p1 + size - 16, p2 + size - 16); return compare16(p1 + size - 16, p2 + size - 16);
} }
#endif #endif

View File

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

View File

@ -1912,6 +1912,9 @@ See also:
For single JOIN in case of identifier ambiguity prefer left table For single JOIN in case of identifier ambiguity prefer left table
)", IMPORTANT) \ )", 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"( 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. 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) \ )", 0) \

View File

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

View File

@ -85,6 +85,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"restore_replace_external_dictionary_source_to_null", false, false, "New setting."}, {"restore_replace_external_dictionary_source_to_null", false, false, "New setting."},
{"show_create_query_identifier_quoting_rule", "when_necessary", "when_necessary", "New setting."}, {"show_create_query_identifier_quoting_rule", "when_necessary", "when_necessary", "New setting."},
{"show_create_query_identifier_quoting_style", "Backticks", "Backticks", "New setting."}, {"show_create_query_identifier_quoting_style", "Backticks", "Backticks", "New setting."},
{"query_plan_join_inner_table_selection", "auto", "auto", "New setting."},
{"merge_tree_min_read_task_size", 8, 8, "New setting"}, {"merge_tree_min_read_task_size", 8, 8, "New setting"},
{"merge_tree_min_rows_for_concurrent_read_for_remote_filesystem", (20 * 8192), 0, "Setting is deprecated"}, {"merge_tree_min_rows_for_concurrent_read_for_remote_filesystem", (20 * 8192), 0, "Setting is deprecated"},
{"merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem", (24 * 10 * 1024 * 1024), 0, "Setting is deprecated"}, {"merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem", (24 * 10 * 1024 * 1024), 0, "Setting is deprecated"},

View File

@ -55,6 +55,10 @@ IMPLEMENT_SETTING_MULTI_ENUM(JoinAlgorithm, ErrorCodes::UNKNOWN_JOIN,
{"full_sorting_merge", JoinAlgorithm::FULL_SORTING_MERGE}, {"full_sorting_merge", JoinAlgorithm::FULL_SORTING_MERGE},
{"grace_hash", JoinAlgorithm::GRACE_HASH}}) {"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, IMPLEMENT_SETTING_ENUM(TotalsMode, ErrorCodes::UNKNOWN_TOTALS_MODE,
{{"before_having", TotalsMode::BEFORE_HAVING}, {{"before_having", TotalsMode::BEFORE_HAVING},

View File

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

View File

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

View File

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

View File

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

View File

@ -18,24 +18,14 @@ namespace ErrorCodes
extern const int CANNOT_SEEK_THROUGH_FILE; 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( ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(
ReadBufferCreator && read_buffer_creator_, ReadBufferCreator && read_buffer_creator_,
const StoredObjects & blobs_to_read_, const StoredObjects & blobs_to_read_,
const ReadSettings & settings_, const ReadSettings & settings_,
std::shared_ptr<FilesystemCacheLog> cache_log_, std::shared_ptr<FilesystemCacheLog> cache_log_,
bool use_external_buffer_) bool use_external_buffer_,
: ReadBufferFromFileBase(use_external_buffer_ ? 0 : chooseBufferSizeForRemoteReading( size_t buffer_size)
settings_, getTotalSize(blobs_to_read_)), nullptr, 0) : ReadBufferFromFileBase(use_external_buffer_ ? 0 : buffer_size, nullptr, 0)
, settings(settings_) , settings(settings_)
, blobs_to_read(blobs_to_read_) , blobs_to_read(blobs_to_read_)
, read_buffer_creator(std::move(read_buffer_creator_)) , read_buffer_creator(std::move(read_buffer_creator_))

View File

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

View File

@ -641,19 +641,33 @@ std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
return impl; 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; const bool use_async_buffer = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool;
auto impl = std::make_unique<ReadBufferFromRemoteFSGather>( auto impl = std::make_unique<ReadBufferFromRemoteFSGather>(
std::move(read_buffer_creator), std::move(read_buffer_creator),
storage_objects, storage_objects,
read_settings, read_settings,
global_context->getFilesystemCacheLog(), 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) if (use_async_buffer)
{ {
auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER);
return std::make_unique<AsynchronousBoundedReadBuffer>( return std::make_unique<AsynchronousBoundedReadBuffer>(
std::move(impl), reader, read_settings, std::move(impl),
reader,
read_settings,
buffer_size,
global_context->getAsyncReadCounters(), global_context->getAsyncReadCounters(),
global_context->getFilesystemReadPrefetchesLog()); global_context->getFilesystemReadPrefetchesLog());

View File

@ -51,7 +51,7 @@ TEST_F(AsynchronousBoundedReadBufferTest, setReadUntilPosition)
for (bool with_prefetch : {false, true}) 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); read_buffer.setReadUntilPosition(20);
auto try_read = [&](size_t count) auto try_read = [&](size_t count)

View File

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

View File

@ -60,6 +60,17 @@ public:
IBlocksStreamPtr IBlocksStreamPtr
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; 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: private:
struct InternalHashJoin struct InternalHashJoin
{ {

View File

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

View File

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

View File

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

View File

@ -56,7 +56,6 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; 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]); 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. /** 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 * 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(); added_columns.buildJoinGetOutput();
else else
added_columns.buildOutput(); 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) for (size_t i = 0; i < added_columns.size(); ++i)
block.insert(added_columns.moveColumn(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.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate);
} }
} }
block.erase(block_columns_to_erase);
return remaining_block; return remaining_block;
} }

View File

@ -1888,7 +1888,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
expressions.join, expressions.join,
settings[Setting::max_block_size], settings[Setting::max_block_size],
max_streams, 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())); join_step->setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType()));
std::vector<QueryPlanPtr> plans; std::vector<QueryPlanPtr> plans;

View File

@ -1310,7 +1310,7 @@ RefreshTaskList InterpreterSystemQuery::getRefreshTasks()
void InterpreterSystemQuery::prewarmMarkCache() void InterpreterSystemQuery::prewarmMarkCache()
{ {
if (table_id.empty()) 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); getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE, table_id);

View File

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

View File

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

View File

@ -1353,12 +1353,15 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
if (tables_with_columns.size() > 1) if (tables_with_columns.size() > 1)
{ {
auto columns_from_left_table = tables_with_columns[0].columns;
const auto & right_table = tables_with_columns[1]; const auto & right_table = tables_with_columns[1];
auto columns_from_joined_table = right_table.columns; auto columns_from_joined_table = right_table.columns;
/// query can use materialized or aliased columns from right joined table, /// query can use materialized or aliased columns from right joined table,
/// we want to request it for right 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()); 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); translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns);

View File

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

View File

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

View File

@ -104,6 +104,7 @@ namespace Setting
extern const SettingsBool optimize_move_to_prewhere; extern const SettingsBool optimize_move_to_prewhere;
extern const SettingsBool optimize_move_to_prewhere_if_final; extern const SettingsBool optimize_move_to_prewhere_if_final;
extern const SettingsBool use_concurrency_control; extern const SettingsBool use_concurrency_control;
extern const SettingsJoinInnerTableSelectionMode query_plan_join_inner_table_selection;
} }
namespace ErrorCodes namespace ErrorCodes
@ -1267,6 +1268,55 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP
plan_to_add_cast.addStep(std::move(cast_join_columns_step)); 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 buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression,
JoinTreeQueryPlan left_join_tree_query_plan, JoinTreeQueryPlan left_join_tree_query_plan,
JoinTreeQueryPlan right_join_tree_query_plan, JoinTreeQueryPlan right_join_tree_query_plan,
@ -1539,21 +1589,37 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
} }
const Block & left_header = left_plan.getCurrentHeader(); const Block & left_header = left_plan.getCurrentHeader();
auto left_table_names = left_header.getNames(); const Block & right_header = right_plan.getCurrentHeader();
NameSet left_table_names_set(left_table_names.begin(), left_table_names.end());
auto columns_from_joined_table = right_plan.getCurrentHeader().getNamesAndTypesList(); auto columns_from_left_table = left_header.getNamesAndTypesList();
table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); 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) && if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) &&
outer_scope_columns.contains(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 join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context);
auto result_plan = QueryPlan(); auto result_plan = QueryPlan();
@ -1644,13 +1710,26 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
} }
auto join_pipeline_type = join_algorithm->pipelineType(); 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>( auto join_step = std::make_unique<JoinStep>(
left_plan.getCurrentHeader(), left_plan.getCurrentHeader(),
right_plan.getCurrentHeader(), right_plan.getCurrentHeader(),
std::move(join_algorithm), std::move(join_algorithm),
settings[Setting::max_block_size], settings[Setting::max_block_size],
settings[Setting::max_threads], 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)); join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type));
@ -1661,47 +1740,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
result_plan.unitePlans(std::move(join_step), {std::move(plans)}); result_plan.unitePlans(std::move(join_step), {std::move(plans)});
} }
ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentHeader().getColumnsWithTypeAndName()); const auto & header_after_join = result_plan.getCurrentHeader();
ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; if (header_after_join.columns() > outer_scope_columns.size())
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 & output = drop_unused_columns_after_join_actions_dag_outputs[i]; 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)
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))
{ {
if (!first_skipped_column_node_index) auto drop_unused_columns_after_join_transform_step = std::make_unique<ExpressionStep>(result_plan.getCurrentHeader(), std::move(*drop_unused_columns_after_join_actions_dag));
first_skipped_column_node_index = i; drop_unused_columns_after_join_transform_step->setStepDescription("Drop unused columns after JOIN");
continue; 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) 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); left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy);

View File

@ -6,6 +6,7 @@
#include <IO/Operators.h> #include <IO/Operators.h>
#include <Common/JSONBuilder.h> #include <Common/JSONBuilder.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Processors/Transforms/ColumnPermuteTransform.h>
namespace DB namespace DB
{ {
@ -36,6 +37,37 @@ std::vector<std::pair<String, String>> describeJoinActions(const JoinPtr & join)
return description; 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( JoinStep::JoinStep(
@ -44,8 +76,15 @@ JoinStep::JoinStep(
JoinPtr join_, JoinPtr join_,
size_t max_block_size_, size_t max_block_size_,
size_t max_streams_, size_t max_streams_,
bool keep_left_read_in_order_) NameSet required_output_,
: join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_) 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_}); updateInputHeaders({left_header_, right_header_});
} }
@ -55,23 +94,43 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines
if (pipelines.size() != 2) if (pipelines.size() != 2)
throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); 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) if (join->pipelineType() == JoinPipelineType::YShaped)
{ {
auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( 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); joined_pipeline->resize(max_streams);
return joined_pipeline; return joined_pipeline;
} }
return QueryPipelineBuilder::joinPipelinesRightLeft( auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft(
std::move(pipelines[0]), std::move(pipelines[0]),
std::move(pipelines[1]), std::move(pipelines[1]),
join, join,
*output_header, join_algorithm_header,
max_block_size, max_block_size,
max_streams, max_streams,
keep_left_read_in_order, keep_left_read_in_order,
&processors); &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 bool JoinStep::allowPushDownToRight() const
@ -90,17 +149,49 @@ void JoinStep::describeActions(FormatSettings & settings) const
for (const auto & [name, value] : describeJoinActions(join)) for (const auto & [name, value] : describeJoinActions(join))
settings.out << prefix << name << ": " << value << '\n'; settings.out << prefix << name << ": " << value << '\n';
if (swap_streams)
settings.out << prefix << "Swapped: true\n";
} }
void JoinStep::describeActions(JSONBuilder::JSONMap & map) const void JoinStep::describeActions(JSONBuilder::JSONMap & map) const
{ {
for (const auto & [name, value] : describeJoinActions(join)) for (const auto & [name, value] : describeJoinActions(join))
map.add(name, value); 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() 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() static ITransformingStep::Traits getStorageJoinTraits()

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -517,9 +517,19 @@ std::unique_ptr<ReadBufferFromFileBase> StorageObjectStorageSource::createReadBu
LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); 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); auto & reader = context_->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER);
impl = std::make_unique<AsynchronousBoundedReadBuffer>( impl = std::make_unique<AsynchronousBoundedReadBuffer>(
std::move(impl), reader, modified_read_settings, std::move(impl),
reader,
modified_read_settings,
buffer_size,
context_->getAsyncReadCounters(), context_->getAsyncReadCounters(),
context_->getFilesystemReadPrefetchesLog()); context_->getFilesystemReadPrefetchesLog());

View File

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

View File

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

View File

@ -227,6 +227,194 @@ FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32} 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") } )EOQ") }
}, },
/// Default dashboard for ClickHouse Cloud /// Default dashboard for ClickHouse Cloud
@ -369,7 +557,143 @@ ORDER BY t WITH FILL STEP {rounding:UInt32}
{ "dashboard", "Cloud overview" }, { "dashboard", "Cloud overview" },
{ "title", "Concurrent network connections" }, { "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" } { "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) auto add_dashboards = [&](const auto & dashboards)

View File

@ -789,6 +789,7 @@ def get_localzone():
return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:])) 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: class SettingsRandomizer:
settings = { settings = {
"max_insert_threads": lambda: ( "max_insert_threads": lambda: (
@ -919,6 +920,9 @@ class SettingsRandomizer:
"max_parsing_threads": lambda: random.choice([0, 1, 10]), "max_parsing_threads": lambda: random.choice([0, 1, 10]),
"optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1),
"parallel_replicas_local_plan": 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), "output_format_native_write_json_as_string": lambda: random.randint(0, 1),
"enable_vertical_final": lambda: random.randint(0, 1), "enable_vertical_final": lambda: random.randint(0, 1),
} }

View File

@ -67,6 +67,7 @@ DEFAULT_ENV_NAME = ".env"
DEFAULT_BASE_CONFIG_DIR = os.environ.get( DEFAULT_BASE_CONFIG_DIR = os.environ.get(
"CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/" "CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/"
) )
DOCKER_BASE_TAG = os.environ.get("DOCKER_BASE_TAG", "latest")
SANITIZER_SIGN = "==================" SANITIZER_SIGN = "=================="
@ -503,7 +504,6 @@ class ClickHouseCluster:
"CLICKHOUSE_TESTS_DOCKERD_HOST" "CLICKHOUSE_TESTS_DOCKERD_HOST"
) )
self.docker_api_version = os.environ.get("DOCKER_API_VERSION") 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"] self.base_cmd = ["docker", "compose"]
if custom_dockerd_host: if custom_dockerd_host:
@ -1079,7 +1079,7 @@ class ClickHouseCluster:
env_variables["keeper_binary"] = binary_path env_variables["keeper_binary"] = binary_path
env_variables["keeper_cmd_prefix"] = keeper_cmd_prefix 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["user"] = str(os.getuid())
env_variables["keeper_fs"] = "bind" env_variables["keeper_fs"] = "bind"
for i in range(1, 4): for i in range(1, 4):
@ -1675,7 +1675,7 @@ class ClickHouseCluster:
) )
if tag is None: if tag is None:
tag = self.docker_base_tag tag = DOCKER_BASE_TAG
if not env_variables: if not env_variables:
env_variables = {} env_variables = {}
self.use_keeper = use_keeper self.use_keeper = use_keeper
@ -4538,7 +4538,12 @@ class ClickHouseInstance:
if len(self.custom_dictionaries_paths): if len(self.custom_dictionaries_paths):
write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) 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 # 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") write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml")

View File

@ -5,6 +5,8 @@ def randomize_settings():
yield "max_joined_block_size_rows", random.randint(8000, 100000) yield "max_joined_block_size_rows", random.randint(8000, 100000)
if random.random() < 0.5: if random.random() < 0.5:
yield "max_block_size", random.randint(8000, 100000) 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): def write_random_settings_config(destination):

View File

@ -91,7 +91,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster):
with client(name="client1>", log=client_output, command=command_text) as client1: with client(name="client1>", log=client_output, command=command_text) as client1:
client1.expect(prompt) client1.expect(prompt)
client1.send( 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("Peak memory usage", timeout=60)
client1.expect(prompt) client1.expect(prompt)

View File

@ -15,9 +15,9 @@ INSERT INTO t2_00826 values (1,1), (1,2);
INSERT INTO t2_00826 (a) values (2), (3); INSERT INTO t2_00826 (a) values (2), (3);
SELECT '--- cross ---'; 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 '--- 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 '--- 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 * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b ORDER BY t1_00826.a;
SELECT '--- cross self ---'; 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 * 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 '--- 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 '--- 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 '--- 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 '--- 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 ---'; SELECT '--- cross ---';

View File

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

View File

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

View File

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

View File

@ -64,39 +64,47 @@ USING (id);
INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02'); INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02');
SELECT '-';
SELECT * SELECT *
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l 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 LEFT JOIN (SELECT item_id FROM t ) r
ON l.item_id = r.item_id; ON l.item_id = r.item_id;
SELECT '-';
SELECT * SELECT *
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l 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 RIGHT JOIN (SELECT item_id FROM t ) r
ON l.item_id = r.item_id; ON l.item_id = r.item_id;
SELECT '-';
SELECT * SELECT *
FROM (SELECT item_id FROM t) l 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 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; ON l.item_id = r.item_id;
SELECT '-';
SELECT * SELECT *
FROM (SELECT item_id FROM t) l 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 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; ON l.item_id = r.item_id;
SELECT '-';
SELECT * SELECT *
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l 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 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; ON l.item_id = r.item_id;
SELECT '-';
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 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 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; ON l.item_id = r.item_id;
SELECT '-';
SELECT * SELECT *
FROM (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date) l 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 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; DROP TABLE t;

View File

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

View File

@ -75,7 +75,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverErro
SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2_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 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' 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 -- 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 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 } 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 (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; -- { 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.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 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; 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; 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; 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 -%} {% endfor -%}

View File

@ -33,23 +33,23 @@
2 2 2 2
2 2 2 2
-- { echoOn } -- { 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 1 0
2 2 2 2
SELECT * FROM t1 RIGHT 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 ORDER BY 1 SETTINGS enable_analyzer = 1;
2 2
0 3 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 1 0
2 2 2 2
0 3 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 1 0
2 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 2
0 3 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 1 0
2 0 2 0
0 2 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) 1 ('',0)
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 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
0 ('b',256) 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) 1 ('',0)
0 ('b',256) 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 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) 1 ('',0)
2 2
4 2 Nullable(UInt64) UInt8 4 2 Nullable(UInt64) UInt8

View File

@ -73,20 +73,20 @@ SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 0; --
SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1; SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1;
-- { echoOn } -- { 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;
SELECT * FROM t1 RIGHT 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 ORDER BY 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 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 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 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 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 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 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 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 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 } -- { echoOff }

View File

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

View File

@ -12,8 +12,9 @@ CREATE TABLE without_nullable
insert into with_nullable values(0,'f'),(0,'usa'); insert into with_nullable values(0,'f'),(0,'usa');
insert into without_nullable values(0,'usa'),(0,'us2a'); insert into without_nullable values(0,'usa'),(0,'us2a');
select if(t0.country is null ,t2.country,t0.country) "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; 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 with_nullable;
drop table without_nullable; drop table without_nullable;

View File

@ -48,7 +48,8 @@ SELECT
L2SquaredDistance(v1.v, v2.v), L2SquaredDistance(v1.v, v2.v),
cosineDistance(v1.v, v2.v) cosineDistance(v1.v, v2.v)
FROM vec2 v1, vec2 v2 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)); 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 SELECT
@ -61,7 +62,8 @@ SELECT
L2SquaredDistance(v1.v, v2.v), L2SquaredDistance(v1.v, v2.v),
cosineDistance(v1.v, v2.v) cosineDistance(v1.v, v2.v)
FROM vec2f v1, vec2f v2 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)); 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 SELECT
@ -74,7 +76,8 @@ SELECT
L2SquaredDistance(v1.v, v2.v), L2SquaredDistance(v1.v, v2.v),
cosineDistance(v1.v, v2.v) cosineDistance(v1.v, v2.v)
FROM vec2d v1, vec2d v2 FROM vec2d v1, vec2d v2
WHERE length(v1.v) == length(v2.v); WHERE length(v1.v) == length(v2.v)
ORDER BY ALL;
SELECT SELECT
v1.id, v1.id,
@ -86,7 +89,8 @@ SELECT
L2SquaredDistance(v1.v, v2.v), L2SquaredDistance(v1.v, v2.v),
cosineDistance(v1.v, v2.v) cosineDistance(v1.v, v2.v)
FROM vec2f v1, vec2d v2 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 L1Distance([0, 0], [1]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
SELECT L2Distance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT L2Distance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }

View File

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

View File

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

View File

@ -117,7 +117,7 @@ ORDER BY t1.number, t2.number
-- explain -- explain
Expression (Project names) Expression (Project names)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) Expression ((Before ORDER BY + Projection))
Join (JOIN FillRightFirst) 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))))))))) 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 ReadFromSystemNumbers
@ -161,7 +161,7 @@ ORDER BY t1.number, t2.number
-- explain -- explain
Expression (Project names) Expression (Project names)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) Expression ((Before ORDER BY + Projection))
Join (JOIN FillRightFirst) 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))))))))) 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 ReadFromSystemNumbers

View File

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

View File

@ -8,24 +8,21 @@ Header: count() UInt64
Aggregating Aggregating
Header: __table1.a2 String Header: __table1.a2 String
count() UInt64 count() UInt64
Expression ((Before GROUP BY + DROP unused columns after JOIN)) Expression (Before GROUP BY)
Header: __table1.a2 String Header: __table1.a2 String
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.a2 String Header: __table1.a2 String
__table3.c1 UInt64 Expression (JOIN actions)
Expression ((JOIN actions + DROP unused columns after JOIN))
Header: __table1.a2 String Header: __table1.a2 String
__table3.c1 UInt64 __table3.c1 UInt64
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.a2 String Header: __table1.a2 String
__table2.b1 UInt64
__table3.c1 UInt64 __table3.c1 UInt64
Expression ((JOIN actions + DROP unused columns after JOIN)) Expression (JOIN actions)
Header: __table1.a2 String Header: __table1.a2 String
__table2.b1 UInt64 __table2.b1 UInt64
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.a1 UInt64 Header: __table1.a2 String
__table1.a2 String
__table2.b1 UInt64 __table2.b1 UInt64
Expression ((JOIN actions + Change column names to column identifiers)) Expression ((JOIN actions + Change column names to column identifiers))
Header: __table1.a1 UInt64 Header: __table1.a1 UInt64
@ -48,39 +45,32 @@ Header: count() UInt64
EXPLAIN PLAN header = 1 EXPLAIN PLAN header = 1
SELECT a.a2, d.d2 FROM a JOIN b USING (k) JOIN c USING (k) JOIN d USING (k) SELECT a.a2, d.d2 FROM a JOIN b USING (k) JOIN c USING (k) JOIN d USING (k)
; ;
Expression ((Project names + (Projection + DROP unused columns after JOIN))) Expression ((Project names + Projection))
Header: a2 String Header: a2 String
d2 String d2 String
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.a2 String Header: __table1.a2 String
__table1.k UInt64
__table4.d2 String __table4.d2 String
Expression (DROP unused columns after JOIN) Join (JOIN FillRightFirst)
Header: __table1.a2 String Header: __table1.a2 String
__table1.k UInt64 __table1.k UInt64
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.a2 String Header: __table1.a2 String
__table1.k UInt64 __table1.k UInt64
Expression (DROP unused columns after JOIN) Expression (Change column names to column identifiers)
Header: __table1.a2 String Header: __table1.a2 String
__table1.k UInt64 __table1.k UInt64
Join (JOIN FillRightFirst) ReadFromMemoryStorage
Header: __table1.a2 String Header: a2 String
__table1.k UInt64 k UInt64
Expression (Change column names to column identifiers)
Header: __table1.a2 String
__table1.k UInt64
ReadFromMemoryStorage
Header: a2 String
k UInt64
Expression (Change column names to column identifiers)
Header: __table2.k UInt64
ReadFromMemoryStorage
Header: k UInt64
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
Header: __table3.k UInt64 Header: __table2.k UInt64
ReadFromMemoryStorage ReadFromMemoryStorage
Header: k UInt64 Header: k UInt64
Expression (Change column names to column identifiers)
Header: __table3.k UInt64
ReadFromMemoryStorage
Header: k UInt64
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
Header: __table4.d2 String Header: __table4.d2 String
__table4.k UInt64 __table4.k UInt64
@ -106,27 +96,24 @@ Header: bx String
Header: __table1.a2 String Header: __table1.a2 String
__table2.bx String __table2.bx String
__table4.c2 String __table4.c2 String
__table4.c1 UInt64
Expression Expression
Header: __table1.a2 String Header: __table1.a2 String
__table2.bx String __table2.bx String
__table4.c2 String
__table4.c1 UInt64 __table4.c1 UInt64
__table4.c2 String
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.a2 String Header: __table1.a2 String
__table2.bx String __table2.bx String
__table2.b1 UInt64
__table4.c2 String
__table4.c1 UInt64 __table4.c1 UInt64
Expression ((JOIN actions + DROP unused columns after JOIN)) __table4.c2 String
Expression (JOIN actions)
Header: __table1.a2 String Header: __table1.a2 String
__table2.bx String
__table2.b1 UInt64 __table2.b1 UInt64
__table2.bx String
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.a1 UInt64 Header: __table1.a2 String
__table1.a2 String
__table2.bx String
__table2.b1 UInt64 __table2.b1 UInt64
__table2.bx String
Expression ((JOIN actions + Change column names to column identifiers)) Expression ((JOIN actions + Change column names to column identifiers))
Header: __table1.a1 UInt64 Header: __table1.a1 UInt64
__table1.a2 String __table1.a2 String

View File

@ -16,6 +16,7 @@ CREATE TABLE d (k UInt64, d1 UInt64, d2 String) ENGINE = Memory;
INSERT INTO d VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c'); INSERT INTO d VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c');
SET enable_analyzer = 1; SET enable_analyzer = 1;
SET query_plan_join_inner_table_selection = 'right';
-- { echoOn } -- { echoOn }

View File

@ -5,7 +5,7 @@
1 1
1 1
1 0
\N \N
100000000000000000000 100000000000000000000

View File

@ -1,22 +1,22 @@
Expression ((Project names + (Projection + DROP unused columns after JOIN))) Expression ((Project names + Projection))
Header: id UInt64 Header: id UInt64
value_1 String value_1 String
rhs.id UInt64 rhs.id UInt64
rhs.value_1 String rhs.value_1 String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value_1 String : 1 INPUT : 1 -> __table1.value_1 String : 1
INPUT : 2 -> __table2.value_1 String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value_1 String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value_1 :: 1 -> value_1 String : 0 ALIAS __table1.value_1 :: 1 -> value_1 String : 0
ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2
Positions: 4 0 2 1 Positions: 4 0 1 2
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value_1 String __table1.value_1 String
__table2.value_1 String
__table2.id UInt64 __table2.id UInt64
__table2.value_1 String
Type: INNER Type: INNER
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -50,29 +50,25 @@ Positions: 4 0 2 1
Parts: 1 Parts: 1
Granules: 1 Granules: 1
-- --
Expression ((Project names + (Projection + DROP unused columns after JOIN))) Expression ((Project names + Projection))
Header: id UInt64 Header: id UInt64
value_1 String value_1 String
rhs.id UInt64 rhs.id UInt64
rhs.value_1 String rhs.value_1 String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value_1 String : 1 INPUT : 1 -> __table1.value_1 String : 1
INPUT :: 2 -> __table1.value_2 UInt64 : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.value_1 String : 3 INPUT : 3 -> __table2.value_1 String : 3
INPUT :: 4 -> __table2.value_2 UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
INPUT : 5 -> __table2.id UInt64 : 5
ALIAS __table1.id :: 0 -> id UInt64 : 6
ALIAS __table1.value_1 :: 1 -> value_1 String : 0 ALIAS __table1.value_1 :: 1 -> value_1 String : 0
ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3 ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2
Positions: 6 0 3 1 Positions: 4 0 1 2
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value_1 String __table1.value_1 String
__table1.value_2 UInt64
__table2.value_1 String
__table2.value_2 UInt64
__table2.id UInt64 __table2.id UInt64
__table2.value_1 String
Type: INNER Type: INNER
Strictness: ASOF Strictness: ASOF
Algorithm: HashJoin Algorithm: HashJoin

View File

@ -19,6 +19,8 @@ CREATE TABLE test_table_2
INSERT INTO test_table_1 VALUES (0, 'Value', 0); INSERT INTO test_table_1 VALUES (0, 'Value', 0);
INSERT INTO test_table_2 VALUES (0, 'Value', 0); INSERT INTO test_table_2 VALUES (0, 'Value', 0);
SET query_plan_join_inner_table_selection = 'right';
EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1
FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id;

View File

@ -31,6 +31,7 @@
8 8
9 9
\N \N
--- analyzer ---
0 0
1 1
2 2

View File

@ -21,6 +21,8 @@ SETTINGS join_algorithm = 'partial_merge';
SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL
SETTINGS join_algorithm = 'full_sorting_merge'; SETTINGS join_algorithm = 'full_sorting_merge';
SELECT '--- analyzer ---';
SET enable_analyzer = 1; SET enable_analyzer = 1;
SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL

View File

@ -2,7 +2,9 @@
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5; WHERE lhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
Expression ((Project names + (Projection + ))) Expression ((Project names + (Projection + )))
Header: id UInt64 Header: id UInt64
rhs.id UInt64 rhs.id UInt64
@ -10,18 +12,18 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 2 0 1 Positions: 4 1 0 2
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: INNER Type: INNER
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -69,7 +71,9 @@ SELECT '--';
-- --
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5; WHERE rhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right';
;
Expression ((Project names + (Projection + ))) Expression ((Project names + (Projection + )))
Header: id UInt64 Header: id UInt64
rhs.id UInt64 rhs.id UInt64
@ -77,18 +81,18 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 2 0 1 Positions: 4 1 0 2
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: INNER Type: INNER
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -136,7 +140,9 @@ SELECT '--';
-- --
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6; WHERE lhs.id = 5 AND rhs.id = 6
SETTINGS query_plan_join_inner_table_selection = 'right'
;
Expression ((Project names + (Projection + ))) Expression ((Project names + (Projection + )))
Header: id UInt64 Header: id UInt64
rhs.id UInt64 rhs.id UInt64
@ -144,18 +150,18 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 2 0 1 Positions: 4 1 0 2
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: INNER Type: INNER
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -206,7 +212,9 @@ SELECT '--';
-- --
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5; WHERE lhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
Expression ((Project names + (Projection + ))) Expression ((Project names + (Projection + )))
Header: id UInt64 Header: id UInt64
rhs.id UInt64 rhs.id UInt64
@ -214,18 +222,18 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 2 0 1 Positions: 4 1 0 2
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: LEFT Type: LEFT
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -273,7 +281,9 @@ SELECT '--';
-- --
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5; WHERE rhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
Expression ((Project names + Projection)) Expression ((Project names + Projection))
Header: id UInt64 Header: id UInt64
rhs.id UInt64 rhs.id UInt64
@ -281,31 +291,31 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 2 0 1 Positions: 4 1 0 2
Filter ((WHERE + DROP unused columns after JOIN)) Filter (WHERE)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Filter column: equals(__table2.id, 5_UInt8) (removed) Filter column: equals(__table2.id, 5_UInt8) (removed)
Actions: INPUT :: 0 -> __table1.id UInt64 : 0 Actions: INPUT :: 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1 INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT :: 3 -> __table2.value String : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5
Positions: 5 0 1 2 3 Positions: 5 0 1 2 3
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: LEFT Type: LEFT
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -347,7 +357,9 @@ SELECT '--';
-- --
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5; WHERE lhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
Expression ((Project names + Projection)) Expression ((Project names + Projection))
Header: id UInt64 Header: id UInt64
rhs.id UInt64 rhs.id UInt64
@ -355,31 +367,31 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 2 0 1 Positions: 4 1 0 2
Filter ((WHERE + DROP unused columns after JOIN)) Filter (WHERE)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Filter column: equals(__table1.id, 5_UInt8) (removed) Filter column: equals(__table1.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1 INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2 INPUT :: 2 -> __table2.id UInt64 : 2
INPUT :: 3 -> __table2.id UInt64 : 3 INPUT :: 3 -> __table2.value String : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5
Positions: 5 0 1 2 3 Positions: 5 0 1 2 3
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: RIGHT Type: RIGHT
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -421,7 +433,9 @@ SELECT '--';
-- --
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5; WHERE rhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
Expression ((Project names + (Projection + ))) Expression ((Project names + (Projection + )))
Header: id UInt64 Header: id UInt64
rhs.id UInt64 rhs.id UInt64
@ -429,18 +443,18 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 2 0 1 Positions: 4 1 0 2
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: RIGHT Type: RIGHT
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -488,7 +502,9 @@ SELECT '--';
-- --
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5; WHERE lhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
Expression ((Project names + Projection)) Expression ((Project names + Projection))
Header: id UInt64 Header: id UInt64
rhs.id UInt64 rhs.id UInt64
@ -496,31 +512,31 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 2 0 1 Positions: 4 1 0 2
Filter ((WHERE + DROP unused columns after JOIN)) Filter (WHERE)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Filter column: equals(__table1.id, 5_UInt8) (removed) Filter column: equals(__table1.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1 INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2 INPUT :: 2 -> __table2.id UInt64 : 2
INPUT :: 3 -> __table2.id UInt64 : 3 INPUT :: 3 -> __table2.value String : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5
Positions: 5 0 1 2 3 Positions: 5 0 1 2 3
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: FULL Type: FULL
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -562,7 +578,9 @@ SELECT '--';
-- --
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5; WHERE rhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
Expression ((Project names + Projection)) Expression ((Project names + Projection))
Header: id UInt64 Header: id UInt64
rhs.id UInt64 rhs.id UInt64
@ -570,31 +588,31 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 2 0 1 Positions: 4 1 0 2
Filter ((WHERE + DROP unused columns after JOIN)) Filter (WHERE)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Filter column: equals(__table2.id, 5_UInt8) (removed) Filter column: equals(__table2.id, 5_UInt8) (removed)
Actions: INPUT :: 0 -> __table1.id UInt64 : 0 Actions: INPUT :: 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1 INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT :: 3 -> __table2.value String : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5
Positions: 5 0 1 2 3 Positions: 5 0 1 2 3
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: FULL Type: FULL
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -636,7 +654,9 @@ SELECT '--';
-- --
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6; WHERE lhs.id = 5 AND rhs.id = 6
SETTINGS query_plan_join_inner_table_selection = 'right'
;
Expression ((Project names + Projection)) Expression ((Project names + Projection))
Header: id UInt64 Header: id UInt64
rhs.id UInt64 rhs.id UInt64
@ -644,34 +664,34 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 2 0 1 Positions: 4 1 0 2
Filter ((WHERE + DROP unused columns after JOIN)) Filter (WHERE)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed) Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed)
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1 INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT :: 3 -> __table2.value String : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 5 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 5
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 6 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 6
FUNCTION equals(__table2.id : 3, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4 FUNCTION equals(__table2.id : 2, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4
FUNCTION and(equals(__table1.id, 5_UInt8) :: 6, equals(__table2.id, 6_UInt8) :: 4) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 5 FUNCTION and(equals(__table1.id, 5_UInt8) :: 6, equals(__table2.id, 6_UInt8) :: 4) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 5
Positions: 5 0 1 2 3 Positions: 5 0 1 2 3
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: FULL Type: FULL
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin

View File

@ -22,7 +22,9 @@ INSERT INTO test_table_2 SELECT number, number FROM numbers(10);
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5; WHERE lhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';
@ -33,7 +35,9 @@ SELECT '--';
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5; WHERE rhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right';
;
SELECT '--'; SELECT '--';
@ -44,7 +48,9 @@ SELECT '--';
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6; WHERE lhs.id = 5 AND rhs.id = 6
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6; WHERE lhs.id = 5 AND rhs.id = 6;
@ -53,7 +59,9 @@ SELECT '--';
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5; WHERE lhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';
@ -64,7 +72,9 @@ SELECT '--';
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5; WHERE rhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';
@ -75,7 +85,9 @@ SELECT '--';
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5; WHERE lhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';
@ -86,7 +98,9 @@ SELECT '--';
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5; WHERE rhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';
@ -97,7 +111,9 @@ SELECT '--';
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5; WHERE lhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';
@ -108,7 +124,9 @@ SELECT '--';
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5; WHERE rhs.id = 5
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';
@ -119,7 +137,9 @@ SELECT '--';
EXPLAIN header = 1, actions = 1 EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6; WHERE lhs.id = 5 AND rhs.id = 6
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';

View File

@ -52,7 +52,9 @@ WITH RECURSIVE search_graph AS (
FROM graph g, search_graph sg FROM graph g, search_graph sg
WHERE g.f = sg.t AND NOT is_cycle WHERE g.f = sg.t AND NOT is_cycle
) )
SELECT * FROM search_graph; SELECT * FROM search_graph
SETTINGS query_plan_join_inner_table_selection = 'right'
;
1 2 arc 1 -> 2 false [(1,2)] 1 2 arc 1 -> 2 false [(1,2)]
1 3 arc 1 -> 3 false [(1,3)] 1 3 arc 1 -> 3 false [(1,3)]
2 3 arc 2 -> 3 false [(2,3)] 2 3 arc 2 -> 3 false [(2,3)]

View File

@ -55,7 +55,9 @@ WITH RECURSIVE search_graph AS (
FROM graph g, search_graph sg FROM graph g, search_graph sg
WHERE g.f = sg.t AND NOT is_cycle WHERE g.f = sg.t AND NOT is_cycle
) )
SELECT * FROM search_graph; SELECT * FROM search_graph
SETTINGS query_plan_join_inner_table_selection = 'right'
;
-- ordering by the path column has same effect as SEARCH DEPTH FIRST -- ordering by the path column has same effect as SEARCH DEPTH FIRST
WITH RECURSIVE search_graph AS ( WITH RECURSIVE search_graph AS (

File diff suppressed because one or more lines are too long

View File

@ -5,18 +5,18 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 0 2 1 Positions: 4 0 1 2
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: INNER Type: INNER
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -75,18 +75,18 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 0 2 1 Positions: 4 0 1 2
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: INNER Type: INNER
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin
@ -145,18 +145,18 @@ Header: id UInt64
rhs.value String rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0 Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1 INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2 INPUT : 2 -> __table2.id UInt64 : 2
INPUT : 3 -> __table2.id UInt64 : 3 INPUT : 3 -> __table2.value String : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0 ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1 ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 ALIAS __table2.value :: 3 -> rhs.value String : 2
Positions: 4 0 2 1 Positions: 4 0 1 2
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.id UInt64 Header: __table1.id UInt64
__table1.value String __table1.value String
__table2.value String
__table2.id UInt64 __table2.id UInt64
__table2.value String
Type: INNER Type: INNER
Strictness: ALL Strictness: ALL
Algorithm: HashJoin Algorithm: HashJoin

View File

@ -22,7 +22,10 @@ SETTINGS index_granularity = 16
INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2'); INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2');
INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3'); INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3');
EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0;
EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';
@ -30,7 +33,9 @@ SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.
SELECT '--'; SELECT '--';
EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0; EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';
@ -38,7 +43,9 @@ SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs
SELECT '--'; SELECT '--';
EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0; EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0
SETTINGS query_plan_join_inner_table_selection = 'right'
;
SELECT '--'; SELECT '--';

View File

@ -65,8 +65,7 @@ SELECT name FROM users RIGHT JOIN users2 USING name WHERE users2.name ='Alice';
Expression ((Project names + (Projection + ))) Expression ((Project names + (Projection + )))
Header: name String Header: name String
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Header: __table1.name String Header: __table2.name String
__table2.name String
Filter (( + Change column names to column identifiers)) Filter (( + Change column names to column identifiers))
Header: __table1.name String Header: __table1.name String
ReadFromMergeTree (default.users) ReadFromMergeTree (default.users)

View File

@ -11,6 +11,7 @@ CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS
SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2;
SET max_memory_usage = '1G'; SET max_memory_usage = '1G';
SET query_plan_join_inner_table_selection = 'right';
CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS
SELECT id_values.id1 AS id, SELECT id_values.id1 AS id,

View File

@ -0,0 +1,4 @@
20000
20000
1
0

View File

@ -0,0 +1,27 @@
-- Tags: no-parallel, no-shared-merge-tree
DROP TABLE IF EXISTS t_prewarm_cache;
CREATE TABLE t_prewarm_cache (a UInt64, b UInt64, c UInt64)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03254_prewarm_mark_cache_smt/t_prewarm_cache', '1')
ORDER BY a SETTINGS prewarm_mark_cache = 0;
SYSTEM DROP MARK CACHE;
INSERT INTO t_prewarm_cache SELECT number, rand(), rand() FROM numbers(20000);
SELECT count() FROM t_prewarm_cache WHERE NOT ignore(*);
SYSTEM DROP MARK CACHE;
SYSTEM PREWARM MARK CACHE t_prewarm_cache;
SELECT count() FROM t_prewarm_cache WHERE NOT ignore(*);
SYSTEM FLUSH LOGS;
SELECT ProfileEvents['LoadedMarksCount'] > 0 FROM system.query_log
WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query LIKE 'SELECT count() FROM t_prewarm_cache%'
ORDER BY event_time_microseconds;
DROP TABLE IF EXISTS t_prewarm_cache;