mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge remote-tracking branch 'origin' into reject_invalid_mutation_queries
This commit is contained in:
commit
a03aa7bd65
@ -86,7 +86,7 @@ using StringRefs = std::vector<StringRef>;
|
||||
* For more information, see hash_map_string_2.cpp
|
||||
*/
|
||||
|
||||
inline bool compare8(const char * p1, const char * p2)
|
||||
inline bool compare16(const char * p1, const char * p2)
|
||||
{
|
||||
return 0xFFFF == _mm_movemask_epi8(_mm_cmpeq_epi8(
|
||||
_mm_loadu_si128(reinterpret_cast<const __m128i *>(p1)),
|
||||
@ -115,7 +115,7 @@ inline bool compare64(const char * p1, const char * p2)
|
||||
|
||||
#elif defined(__aarch64__) && defined(__ARM_NEON)
|
||||
|
||||
inline bool compare8(const char * p1, const char * p2)
|
||||
inline bool compare16(const char * p1, const char * p2)
|
||||
{
|
||||
uint64_t mask = getNibbleMask(vceqq_u8(
|
||||
vld1q_u8(reinterpret_cast<const unsigned char *>(p1)), vld1q_u8(reinterpret_cast<const unsigned char *>(p2))));
|
||||
@ -185,13 +185,22 @@ inline bool memequalWide(const char * p1, const char * p2, size_t size)
|
||||
|
||||
switch (size / 16) // NOLINT(bugprone-switch-missing-default-case)
|
||||
{
|
||||
case 3: if (!compare8(p1 + 32, p2 + 32)) return false; [[fallthrough]];
|
||||
case 2: if (!compare8(p1 + 16, p2 + 16)) return false; [[fallthrough]];
|
||||
case 1: if (!compare8(p1, p2)) return false; [[fallthrough]];
|
||||
case 3:
|
||||
if (!compare16(p1 + 32, p2 + 32))
|
||||
return false;
|
||||
[[fallthrough]];
|
||||
case 2:
|
||||
if (!compare16(p1 + 16, p2 + 16))
|
||||
return false;
|
||||
[[fallthrough]];
|
||||
case 1:
|
||||
if (!compare16(p1, p2))
|
||||
return false;
|
||||
[[fallthrough]];
|
||||
default: ;
|
||||
}
|
||||
|
||||
return compare8(p1 + size - 16, p2 + size - 16);
|
||||
return compare16(p1 + size - 16, p2 + size - 16);
|
||||
}
|
||||
|
||||
#endif
|
||||
|
2
contrib/krb5
vendored
2
contrib/krb5
vendored
@ -1 +1 @@
|
||||
Subproject commit 71b06c2276009ae649c7703019f3b4605f66fd3d
|
||||
Subproject commit c5b4b994c18db86933255907a97eee5993fd18fe
|
@ -23,6 +23,7 @@ functions in ClickHouse. The sample datasets include:
|
||||
- The [NYPD Complaint Data](../getting-started/example-datasets/nypd_complaint_data.md) demonstrates how to use data inference to simplify creating tables
|
||||
- The ["What's on the Menu?" dataset](../getting-started/example-datasets/menus.md) has an example of denormalizing data
|
||||
- The [Laion dataset](../getting-started/example-datasets/laion.md) has an example of [Approximate nearest neighbor search indexes](../engines/table-engines/mergetree-family/annindexes.md) usage
|
||||
- The [TPC-H](../getting-started/example-datasets/tpch.md), [TPC-DS](../getting-started/example-datasets/tpcds.md), and [Star Schema (SSB)](../getting-started/example-datasets/star-schema.md) industry benchmarks for analytics databases
|
||||
- [Getting Data Into ClickHouse - Part 1](https://clickhouse.com/blog/getting-data-into-clickhouse-part-1) provides examples of defining a schema and loading a small Hacker News dataset
|
||||
- [Getting Data Into ClickHouse - Part 3 - Using S3](https://clickhouse.com/blog/getting-data-into-clickhouse-part-3-s3) has examples of loading data from s3
|
||||
- [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse) shows how to generate random data if none of the above fit your needs.
|
||||
|
@ -119,4 +119,15 @@ enum class JoinTableSide : uint8_t
|
||||
|
||||
const char * toString(JoinTableSide join_table_side);
|
||||
|
||||
/// Setting to choose which table to use as the inner table in hash join
|
||||
enum class JoinInnerTableSelectionMode : uint8_t
|
||||
{
|
||||
/// Use left table
|
||||
Left,
|
||||
/// Use right table
|
||||
Right,
|
||||
/// Use the table with the smallest number of rows
|
||||
Auto,
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1912,6 +1912,9 @@ See also:
|
||||
For single JOIN in case of identifier ambiguity prefer left table
|
||||
)", IMPORTANT) \
|
||||
\
|
||||
DECLARE(JoinInnerTableSelectionMode, query_plan_join_inner_table_selection, JoinInnerTableSelectionMode::Auto, R"(
|
||||
Select the side of the join to be the inner table in the query plan. Supported only for `ALL` join strictness with `JOIN ON` clause. Possible values: 'auto', 'left', 'right'.
|
||||
)", 0) \
|
||||
DECLARE(UInt64, preferred_block_size_bytes, 1000000, R"(
|
||||
This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality.
|
||||
)", 0) \
|
||||
|
@ -66,6 +66,7 @@ class WriteBuffer;
|
||||
M(CLASS_NAME, IntervalOutputFormat) \
|
||||
M(CLASS_NAME, JoinAlgorithm) \
|
||||
M(CLASS_NAME, JoinStrictness) \
|
||||
M(CLASS_NAME, JoinInnerTableSelectionMode) \
|
||||
M(CLASS_NAME, LightweightMutationProjectionMode) \
|
||||
M(CLASS_NAME, LoadBalancing) \
|
||||
M(CLASS_NAME, LocalFSReadMethod) \
|
||||
|
@ -72,6 +72,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
{"backup_restore_keeper_max_retries_while_initializing", 0, 20, "New setting."},
|
||||
{"backup_restore_keeper_max_retries_while_handling_error", 0, 20, "New setting."},
|
||||
{"backup_restore_finish_timeout_after_error_sec", 0, 180, "New setting."},
|
||||
{"query_plan_join_inner_table_selection", "auto", "auto", "New setting."},
|
||||
}
|
||||
},
|
||||
{"24.10",
|
||||
|
@ -55,6 +55,10 @@ IMPLEMENT_SETTING_MULTI_ENUM(JoinAlgorithm, ErrorCodes::UNKNOWN_JOIN,
|
||||
{"full_sorting_merge", JoinAlgorithm::FULL_SORTING_MERGE},
|
||||
{"grace_hash", JoinAlgorithm::GRACE_HASH}})
|
||||
|
||||
IMPLEMENT_SETTING_ENUM(JoinInnerTableSelectionMode, ErrorCodes::BAD_ARGUMENTS,
|
||||
{{"left", JoinInnerTableSelectionMode::Left},
|
||||
{"right", JoinInnerTableSelectionMode::Right},
|
||||
{"auto", JoinInnerTableSelectionMode::Auto}})
|
||||
|
||||
IMPLEMENT_SETTING_ENUM(TotalsMode, ErrorCodes::UNKNOWN_TOTALS_MODE,
|
||||
{{"before_having", TotalsMode::BEFORE_HAVING},
|
||||
|
@ -128,8 +128,8 @@ constexpr auto getEnumValues();
|
||||
DECLARE_SETTING_ENUM(LoadBalancing)
|
||||
|
||||
DECLARE_SETTING_ENUM(JoinStrictness)
|
||||
|
||||
DECLARE_SETTING_MULTI_ENUM(JoinAlgorithm)
|
||||
DECLARE_SETTING_ENUM(JoinInnerTableSelectionMode)
|
||||
|
||||
|
||||
/// Which rows should be included in TOTALS.
|
||||
|
@ -46,11 +46,13 @@ AsynchronousBoundedReadBuffer::AsynchronousBoundedReadBuffer(
|
||||
ImplPtr impl_,
|
||||
IAsynchronousReader & reader_,
|
||||
const ReadSettings & settings_,
|
||||
size_t buffer_size_,
|
||||
AsyncReadCountersPtr async_read_counters_,
|
||||
FilesystemReadPrefetchesLogPtr prefetches_log_)
|
||||
: ReadBufferFromFileBase(0, nullptr, 0)
|
||||
, impl(std::move(impl_))
|
||||
, read_settings(settings_)
|
||||
, buffer_size(buffer_size_)
|
||||
, reader(reader_)
|
||||
, query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "")
|
||||
, current_reader_id(getRandomASCIIString(8))
|
||||
@ -112,7 +114,7 @@ void AsynchronousBoundedReadBuffer::prefetch(Priority priority)
|
||||
last_prefetch_info.submit_time = std::chrono::system_clock::now();
|
||||
last_prefetch_info.priority = priority;
|
||||
|
||||
prefetch_buffer.resize(chooseBufferSizeForRemoteReading(read_settings, impl->getFileSize()));
|
||||
prefetch_buffer.resize(buffer_size);
|
||||
prefetch_future = readAsync(prefetch_buffer.data(), prefetch_buffer.size(), priority);
|
||||
ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches);
|
||||
}
|
||||
@ -211,7 +213,7 @@ bool AsynchronousBoundedReadBuffer::nextImpl()
|
||||
}
|
||||
else
|
||||
{
|
||||
memory.resize(chooseBufferSizeForRemoteReading(read_settings, impl->getFileSize()));
|
||||
memory.resize(buffer_size);
|
||||
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::SynchronousRemoteReadWaitMicroseconds);
|
||||
|
@ -27,6 +27,7 @@ public:
|
||||
ImplPtr impl_,
|
||||
IAsynchronousReader & reader_,
|
||||
const ReadSettings & settings_,
|
||||
size_t buffer_size_,
|
||||
AsyncReadCountersPtr async_read_counters_ = nullptr,
|
||||
FilesystemReadPrefetchesLogPtr prefetches_log_ = nullptr);
|
||||
|
||||
@ -53,6 +54,7 @@ public:
|
||||
private:
|
||||
const ImplPtr impl;
|
||||
const ReadSettings read_settings;
|
||||
const size_t buffer_size;
|
||||
IAsynchronousReader & reader;
|
||||
|
||||
size_t file_offset_of_buffer_end = 0;
|
||||
|
@ -41,6 +41,8 @@ public:
|
||||
|
||||
~CachedOnDiskReadBufferFromFile() override;
|
||||
|
||||
bool isCached() const override { return true; }
|
||||
|
||||
bool nextImpl() override;
|
||||
|
||||
off_t seek(off_t off, int whence) override;
|
||||
|
@ -18,24 +18,14 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_SEEK_THROUGH_FILE;
|
||||
}
|
||||
|
||||
size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size)
|
||||
{
|
||||
/// Only when cache is used we could download bigger portions of FileSegments than what we actually gonna read within particular task.
|
||||
if (!settings.enable_filesystem_cache && !settings.read_through_distributed_cache)
|
||||
return settings.remote_fs_buffer_size;
|
||||
|
||||
/// Buffers used for prefetch and pre-download better to have enough size, but not bigger than the whole file.
|
||||
return std::min<size_t>(std::max<size_t>(settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE), file_size);
|
||||
}
|
||||
|
||||
ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(
|
||||
ReadBufferCreator && read_buffer_creator_,
|
||||
const StoredObjects & blobs_to_read_,
|
||||
const ReadSettings & settings_,
|
||||
std::shared_ptr<FilesystemCacheLog> cache_log_,
|
||||
bool use_external_buffer_)
|
||||
: ReadBufferFromFileBase(use_external_buffer_ ? 0 : chooseBufferSizeForRemoteReading(
|
||||
settings_, getTotalSize(blobs_to_read_)), nullptr, 0)
|
||||
bool use_external_buffer_,
|
||||
size_t buffer_size)
|
||||
: ReadBufferFromFileBase(use_external_buffer_ ? 0 : buffer_size, nullptr, 0)
|
||||
, settings(settings_)
|
||||
, blobs_to_read(blobs_to_read_)
|
||||
, read_buffer_creator(std::move(read_buffer_creator_))
|
||||
|
@ -28,7 +28,8 @@ public:
|
||||
const StoredObjects & blobs_to_read_,
|
||||
const ReadSettings & settings_,
|
||||
std::shared_ptr<FilesystemCacheLog> cache_log_,
|
||||
bool use_external_buffer_);
|
||||
bool use_external_buffer_,
|
||||
size_t buffer_size);
|
||||
|
||||
~ReadBufferFromRemoteFSGather() override;
|
||||
|
||||
@ -84,6 +85,4 @@ private:
|
||||
|
||||
LoggerPtr log;
|
||||
};
|
||||
|
||||
size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size);
|
||||
}
|
||||
|
@ -641,19 +641,33 @@ std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
|
||||
return impl;
|
||||
};
|
||||
|
||||
/// Avoid cache fragmentation by choosing bigger buffer size.
|
||||
bool prefer_bigger_buffer_size = object_storage->supportsCache() && read_settings.enable_filesystem_cache;
|
||||
size_t buffer_size = prefer_bigger_buffer_size
|
||||
? std::max<size_t>(settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: settings.remote_fs_buffer_size;
|
||||
|
||||
size_t total_objects_size = file_size ? *file_size : getTotalSize(storage_objects);
|
||||
if (total_objects_size)
|
||||
buffer_size = std::min(buffer_size, total_objects_size);
|
||||
|
||||
const bool use_async_buffer = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool;
|
||||
auto impl = std::make_unique<ReadBufferFromRemoteFSGather>(
|
||||
std::move(read_buffer_creator),
|
||||
storage_objects,
|
||||
read_settings,
|
||||
global_context->getFilesystemCacheLog(),
|
||||
/* use_external_buffer */use_async_buffer);
|
||||
/* use_external_buffer */use_async_buffer,
|
||||
/* buffer_size */use_async_buffer ? 0 : buffer_size);
|
||||
|
||||
if (use_async_buffer)
|
||||
{
|
||||
auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER);
|
||||
return std::make_unique<AsynchronousBoundedReadBuffer>(
|
||||
std::move(impl), reader, read_settings,
|
||||
std::move(impl),
|
||||
reader,
|
||||
read_settings,
|
||||
buffer_size,
|
||||
global_context->getAsyncReadCounters(),
|
||||
global_context->getFilesystemReadPrefetchesLog());
|
||||
|
||||
|
@ -51,7 +51,7 @@ TEST_F(AsynchronousBoundedReadBufferTest, setReadUntilPosition)
|
||||
|
||||
for (bool with_prefetch : {false, true})
|
||||
{
|
||||
AsynchronousBoundedReadBuffer read_buffer(createReadBufferFromFileBase(file_path, {}), remote_fs_reader, {});
|
||||
AsynchronousBoundedReadBuffer read_buffer(createReadBufferFromFileBase(file_path, {}), remote_fs_reader, {}, DBMS_DEFAULT_BUFFER_SIZE);
|
||||
read_buffer.setReadUntilPosition(20);
|
||||
|
||||
auto try_read = [&](size_t count)
|
||||
|
@ -60,6 +60,8 @@ public:
|
||||
/// file offset and what getPosition() returns.
|
||||
virtual bool isRegularLocalFile(size_t * /*out_view_offsee*/) { return false; }
|
||||
|
||||
virtual bool isCached() const { return false; }
|
||||
|
||||
protected:
|
||||
std::optional<size_t> file_size;
|
||||
ProfileCallback profile_callback;
|
||||
|
@ -60,6 +60,17 @@ public:
|
||||
IBlocksStreamPtr
|
||||
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override;
|
||||
|
||||
|
||||
bool isCloneSupported() const override
|
||||
{
|
||||
return !getTotals() && getTotalRowCount() == 0;
|
||||
}
|
||||
|
||||
std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_, const Block &, const Block & right_sample_block_) const override
|
||||
{
|
||||
return std::make_shared<ConcurrentHashJoin>(context, table_join_, slots, right_sample_block_, stats_collecting_params);
|
||||
}
|
||||
|
||||
private:
|
||||
struct InternalHashJoin
|
||||
{
|
||||
|
@ -36,7 +36,7 @@ public:
|
||||
|
||||
bool isCloneSupported() const override
|
||||
{
|
||||
return true;
|
||||
return !getTotals();
|
||||
}
|
||||
|
||||
std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_,
|
||||
|
@ -383,6 +383,16 @@ size_t HashJoin::getTotalByteCount() const
|
||||
return res;
|
||||
}
|
||||
|
||||
bool HashJoin::isUsedByAnotherAlgorithm() const
|
||||
{
|
||||
return table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH);
|
||||
}
|
||||
|
||||
bool HashJoin::canRemoveColumnsFromLeftBlock() const
|
||||
{
|
||||
return table_join->enableEnalyzer() && !table_join->hasUsing() && !isUsedByAnotherAlgorithm();
|
||||
}
|
||||
|
||||
void HashJoin::initRightBlockStructure(Block & saved_block_sample)
|
||||
{
|
||||
if (isCrossOrComma(kind))
|
||||
@ -394,8 +404,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample)
|
||||
|
||||
bool multiple_disjuncts = !table_join->oneDisjunct();
|
||||
/// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any).
|
||||
bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) ||
|
||||
table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) ||
|
||||
bool save_key_columns = isUsedByAnotherAlgorithm() ||
|
||||
isRightOrFull(kind) ||
|
||||
multiple_disjuncts ||
|
||||
table_join->getMixedJoinExpression();
|
||||
@ -1228,7 +1237,10 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block,
|
||||
{
|
||||
if (!JoinCommon::hasNonJoinedBlocks(*table_join))
|
||||
return {};
|
||||
|
||||
size_t left_columns_count = left_sample_block.columns();
|
||||
if (canRemoveColumnsFromLeftBlock())
|
||||
left_columns_count = table_join->getOutputColumns(JoinTableSide::Left).size();
|
||||
|
||||
bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join);
|
||||
if (!flag_per_row)
|
||||
|
@ -127,7 +127,7 @@ public:
|
||||
|
||||
bool isCloneSupported() const override
|
||||
{
|
||||
return true;
|
||||
return !getTotals() && getTotalRowCount() == 0;
|
||||
}
|
||||
|
||||
std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_,
|
||||
@ -464,6 +464,9 @@ private:
|
||||
|
||||
bool empty() const;
|
||||
|
||||
bool isUsedByAnotherAlgorithm() const;
|
||||
bool canRemoveColumnsFromLeftBlock() const;
|
||||
|
||||
void validateAdditionalFilterExpression(std::shared_ptr<ExpressionActions> additional_filter_expression);
|
||||
bool needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const;
|
||||
|
||||
|
@ -56,7 +56,6 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
|
||||
const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right;
|
||||
join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]);
|
||||
}
|
||||
size_t existing_columns = block.columns();
|
||||
|
||||
/** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized.
|
||||
* Because if they are constants, then in the "not joined" rows, they may have different values
|
||||
@ -99,6 +98,22 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
|
||||
added_columns.buildJoinGetOutput();
|
||||
else
|
||||
added_columns.buildOutput();
|
||||
|
||||
const auto & table_join = join.table_join;
|
||||
std::set<size_t> block_columns_to_erase;
|
||||
if (join.canRemoveColumnsFromLeftBlock())
|
||||
{
|
||||
std::unordered_set<String> left_output_columns;
|
||||
for (const auto & out_column : table_join->getOutputColumns(JoinTableSide::Left))
|
||||
left_output_columns.insert(out_column.name);
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
{
|
||||
if (!left_output_columns.contains(block.getByPosition(i).name))
|
||||
block_columns_to_erase.insert(i);
|
||||
}
|
||||
}
|
||||
size_t existing_columns = block.columns();
|
||||
|
||||
for (size_t i = 0; i < added_columns.size(); ++i)
|
||||
block.insert(added_columns.moveColumn(i));
|
||||
|
||||
@ -160,6 +175,7 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
|
||||
block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate);
|
||||
}
|
||||
}
|
||||
block.erase(block_columns_to_erase);
|
||||
return remaining_block;
|
||||
}
|
||||
|
||||
|
@ -1888,7 +1888,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
expressions.join,
|
||||
settings[Setting::max_block_size],
|
||||
max_streams,
|
||||
analysis_result.optimize_read_in_order);
|
||||
/* required_output_ = */ NameSet{},
|
||||
analysis_result.optimize_read_in_order,
|
||||
/* use_new_analyzer_ = */ false);
|
||||
|
||||
join_step->setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType()));
|
||||
std::vector<QueryPlanPtr> plans;
|
||||
|
@ -1310,7 +1310,7 @@ RefreshTaskList InterpreterSystemQuery::getRefreshTasks()
|
||||
void InterpreterSystemQuery::prewarmMarkCache()
|
||||
{
|
||||
if (table_id.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for prewarming marks cache");
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for PREWARM MARK CACHE command");
|
||||
|
||||
getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE, table_id);
|
||||
|
||||
|
@ -41,6 +41,7 @@ namespace DB
|
||||
namespace Setting
|
||||
{
|
||||
extern const SettingsBool allow_experimental_join_right_table_sorting;
|
||||
extern const SettingsBool allow_experimental_analyzer;
|
||||
extern const SettingsUInt64 cross_join_min_bytes_to_compress;
|
||||
extern const SettingsUInt64 cross_join_min_rows_to_compress;
|
||||
extern const SettingsUInt64 default_max_bytes_in_join;
|
||||
@ -143,6 +144,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary
|
||||
, max_memory_usage(settings[Setting::max_memory_usage])
|
||||
, tmp_volume(tmp_volume_)
|
||||
, tmp_data(tmp_data_)
|
||||
, enable_analyzer(settings[Setting::allow_experimental_analyzer])
|
||||
{
|
||||
}
|
||||
|
||||
@ -161,6 +163,8 @@ void TableJoin::resetCollected()
|
||||
clauses.clear();
|
||||
columns_from_joined_table.clear();
|
||||
columns_added_by_join.clear();
|
||||
columns_from_left_table.clear();
|
||||
result_columns_from_left_table.clear();
|
||||
original_names.clear();
|
||||
renames.clear();
|
||||
left_type_map.clear();
|
||||
@ -203,6 +207,20 @@ size_t TableJoin::rightKeyInclusion(const String & name) const
|
||||
return count;
|
||||
}
|
||||
|
||||
void TableJoin::setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns)
|
||||
{
|
||||
columns_from_left_table = std::move(left_output_columns);
|
||||
columns_from_joined_table = std::move(right_output_columns);
|
||||
}
|
||||
|
||||
|
||||
const NamesAndTypesList & TableJoin::getOutputColumns(JoinTableSide side)
|
||||
{
|
||||
if (side == JoinTableSide::Left)
|
||||
return result_columns_from_left_table;
|
||||
return columns_added_by_join;
|
||||
}
|
||||
|
||||
void TableJoin::deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix)
|
||||
{
|
||||
NameSet joined_columns;
|
||||
@ -351,9 +369,18 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const
|
||||
return forceNullableRight() && JoinCommon::canBecomeNullable(column_type);
|
||||
}
|
||||
|
||||
void TableJoin::setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side)
|
||||
{
|
||||
if (side == JoinTableSide::Left)
|
||||
result_columns_from_left_table.push_back(joined_column);
|
||||
else
|
||||
columns_added_by_join.push_back(joined_column);
|
||||
|
||||
}
|
||||
|
||||
void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column)
|
||||
{
|
||||
columns_added_by_join.emplace_back(joined_column);
|
||||
setUsedColumn(joined_column, JoinTableSide::Right);
|
||||
}
|
||||
|
||||
NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const
|
||||
@ -995,5 +1022,32 @@ size_t TableJoin::getMaxMemoryUsage() const
|
||||
return max_memory_usage;
|
||||
}
|
||||
|
||||
void TableJoin::swapSides()
|
||||
{
|
||||
assertEnableEnalyzer();
|
||||
|
||||
std::swap(key_asts_left, key_asts_right);
|
||||
std::swap(left_type_map, right_type_map);
|
||||
for (auto & clause : clauses)
|
||||
{
|
||||
std::swap(clause.key_names_left, clause.key_names_right);
|
||||
std::swap(clause.on_filter_condition_left, clause.on_filter_condition_right);
|
||||
std::swap(clause.analyzer_left_filter_condition_column_name, clause.analyzer_right_filter_condition_column_name);
|
||||
}
|
||||
|
||||
std::swap(columns_from_left_table, columns_from_joined_table);
|
||||
std::swap(result_columns_from_left_table, columns_added_by_join);
|
||||
|
||||
if (table_join.kind == JoinKind::Left)
|
||||
table_join.kind = JoinKind::Right;
|
||||
else if (table_join.kind == JoinKind::Right)
|
||||
table_join.kind = JoinKind::Left;
|
||||
}
|
||||
|
||||
void TableJoin::assertEnableEnalyzer() const
|
||||
{
|
||||
if (!enable_analyzer)
|
||||
throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TableJoin: analyzer is disabled");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -167,6 +167,9 @@ private:
|
||||
|
||||
ASOFJoinInequality asof_inequality = ASOFJoinInequality::GreaterOrEquals;
|
||||
|
||||
NamesAndTypesList columns_from_left_table;
|
||||
NamesAndTypesList result_columns_from_left_table;
|
||||
|
||||
/// All columns which can be read from joined table. Duplicating names are qualified.
|
||||
NamesAndTypesList columns_from_joined_table;
|
||||
/// Columns will be added to block by JOIN.
|
||||
@ -202,6 +205,8 @@ private:
|
||||
|
||||
bool is_join_with_constant = false;
|
||||
|
||||
bool enable_analyzer = false;
|
||||
|
||||
Names requiredJoinedNames() const;
|
||||
|
||||
/// Create converting actions and change key column names if required
|
||||
@ -266,6 +271,8 @@ public:
|
||||
VolumePtr getGlobalTemporaryVolume() { return tmp_volume; }
|
||||
|
||||
TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; }
|
||||
bool enableEnalyzer() const { return enable_analyzer; }
|
||||
void assertEnableEnalyzer() const;
|
||||
|
||||
ActionsDAG createJoinedBlockActions(ContextPtr context) const;
|
||||
|
||||
@ -282,6 +289,7 @@ public:
|
||||
}
|
||||
|
||||
bool allowParallelHashJoin() const;
|
||||
void swapSides();
|
||||
|
||||
bool joinUseNulls() const { return join_use_nulls; }
|
||||
|
||||
@ -372,6 +380,9 @@ public:
|
||||
bool leftBecomeNullable(const DataTypePtr & column_type) const;
|
||||
bool rightBecomeNullable(const DataTypePtr & column_type) const;
|
||||
void addJoinedColumn(const NameAndTypePair & joined_column);
|
||||
|
||||
void setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side);
|
||||
|
||||
void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value)
|
||||
{
|
||||
columns_added_by_join = columns_added_by_join_value;
|
||||
@ -397,11 +408,17 @@ public:
|
||||
ASTPtr leftKeysList() const;
|
||||
ASTPtr rightKeysList() const; /// For ON syntax only
|
||||
|
||||
void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix)
|
||||
void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix, const NamesAndTypesList & columns_from_left_table_)
|
||||
{
|
||||
columns_from_joined_table = std::move(columns_from_joined_table_value);
|
||||
deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix);
|
||||
result_columns_from_left_table = columns_from_left_table_;
|
||||
columns_from_left_table = columns_from_left_table_;
|
||||
}
|
||||
|
||||
void setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns);
|
||||
const NamesAndTypesList & getOutputColumns(JoinTableSide side);
|
||||
|
||||
const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; }
|
||||
const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; }
|
||||
|
||||
|
@ -1353,12 +1353,15 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
|
||||
if (tables_with_columns.size() > 1)
|
||||
{
|
||||
auto columns_from_left_table = tables_with_columns[0].columns;
|
||||
const auto & right_table = tables_with_columns[1];
|
||||
auto columns_from_joined_table = right_table.columns;
|
||||
/// query can use materialized or aliased columns from right joined table,
|
||||
/// we want to request it for right table
|
||||
columns_from_joined_table.insert(columns_from_joined_table.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end());
|
||||
result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix());
|
||||
columns_from_left_table.insert(columns_from_left_table.end(), tables_with_columns[0].hidden_columns.begin(), tables_with_columns[0].hidden_columns.end());
|
||||
result.analyzed_join->setColumnsFromJoinedTable(
|
||||
std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix(), columns_from_left_table);
|
||||
}
|
||||
|
||||
translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns);
|
||||
|
@ -31,7 +31,7 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r
|
||||
/// If we generate random UUIDs for already existing tables then those UUIDs will not be correct making those inner target table inaccessible.
|
||||
/// Thus it's not safe for example to replace
|
||||
/// "ATTACH MATERIALIZED VIEW mv AS SELECT a FROM b" with
|
||||
/// "ATTACH MATERIALIZED VIEW mv TO INNER UUID "XXXX" AS SELECT a FROM b"
|
||||
/// "ATTACH MATERIALIZED VIEW mv TO INNER UUID '123e4567-e89b-12d3-a456-426614174000' AS SELECT a FROM b"
|
||||
/// This replacement is safe only for CREATE queries when inner target tables don't exist yet.
|
||||
if (!query.attach)
|
||||
{
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
#include <Analyzer/JoinNode.h>
|
||||
|
||||
#include <Planner/PlannerContext.h>
|
||||
|
||||
|
@ -104,6 +104,7 @@ namespace Setting
|
||||
extern const SettingsBool optimize_move_to_prewhere;
|
||||
extern const SettingsBool optimize_move_to_prewhere_if_final;
|
||||
extern const SettingsBool use_concurrency_control;
|
||||
extern const SettingsJoinInnerTableSelectionMode query_plan_join_inner_table_selection;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
@ -1241,6 +1242,55 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP
|
||||
plan_to_add_cast.addStep(std::move(cast_join_columns_step));
|
||||
}
|
||||
|
||||
std::optional<ActionsDAG> createStepToDropColumns(
|
||||
const Block & header,
|
||||
const ColumnIdentifierSet & outer_scope_columns,
|
||||
const PlannerContextPtr & planner_context)
|
||||
{
|
||||
ActionsDAG drop_unused_columns_after_join_actions_dag(header.getColumnsWithTypeAndName());
|
||||
ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs;
|
||||
std::unordered_set<std::string_view> drop_unused_columns_after_join_actions_dag_updated_outputs_names;
|
||||
std::optional<size_t> first_skipped_column_node_index;
|
||||
|
||||
auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs();
|
||||
size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size();
|
||||
|
||||
const auto & global_planner_context = planner_context->getGlobalPlannerContext();
|
||||
|
||||
for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i)
|
||||
{
|
||||
const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i];
|
||||
|
||||
if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name)
|
||||
|| !global_planner_context->hasColumnIdentifier(output->result_name))
|
||||
continue;
|
||||
|
||||
if (!outer_scope_columns.contains(output->result_name))
|
||||
{
|
||||
if (!first_skipped_column_node_index)
|
||||
first_skipped_column_node_index = i;
|
||||
continue;
|
||||
}
|
||||
|
||||
drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output);
|
||||
drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name);
|
||||
}
|
||||
|
||||
if (!first_skipped_column_node_index)
|
||||
return {};
|
||||
|
||||
/** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope.
|
||||
*
|
||||
* Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2;
|
||||
*/
|
||||
if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index)
|
||||
drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]);
|
||||
|
||||
drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs);
|
||||
|
||||
return drop_unused_columns_after_join_actions_dag;
|
||||
}
|
||||
|
||||
JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression,
|
||||
JoinTreeQueryPlan left_join_tree_query_plan,
|
||||
JoinTreeQueryPlan right_join_tree_query_plan,
|
||||
@ -1513,21 +1563,37 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
}
|
||||
|
||||
const Block & left_header = left_plan.getCurrentHeader();
|
||||
auto left_table_names = left_header.getNames();
|
||||
NameSet left_table_names_set(left_table_names.begin(), left_table_names.end());
|
||||
const Block & right_header = right_plan.getCurrentHeader();
|
||||
|
||||
auto columns_from_joined_table = right_plan.getCurrentHeader().getNamesAndTypesList();
|
||||
table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, "");
|
||||
auto columns_from_left_table = left_header.getNamesAndTypesList();
|
||||
auto columns_from_right_table = right_header.getNamesAndTypesList();
|
||||
|
||||
for (auto & column_from_joined_table : columns_from_joined_table)
|
||||
table_join->setInputColumns(columns_from_left_table, columns_from_right_table);
|
||||
|
||||
for (auto & column_from_joined_table : columns_from_left_table)
|
||||
{
|
||||
/// Add columns from joined table only if they are presented in outer scope, otherwise they can be dropped
|
||||
/// Add columns to output only if they are presented in outer scope, otherwise they can be dropped
|
||||
if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) &&
|
||||
outer_scope_columns.contains(column_from_joined_table.name))
|
||||
table_join->addJoinedColumn(column_from_joined_table);
|
||||
table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Left);
|
||||
}
|
||||
|
||||
for (auto & column_from_joined_table : columns_from_right_table)
|
||||
{
|
||||
/// Add columns to output only if they are presented in outer scope, otherwise they can be dropped
|
||||
if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) &&
|
||||
outer_scope_columns.contains(column_from_joined_table.name))
|
||||
table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right);
|
||||
}
|
||||
|
||||
if (table_join->getOutputColumns(JoinTableSide::Left).empty() && table_join->getOutputColumns(JoinTableSide::Right).empty())
|
||||
{
|
||||
if (!columns_from_left_table.empty())
|
||||
table_join->setUsedColumn(columns_from_left_table.front(), JoinTableSide::Left);
|
||||
else if (!columns_from_right_table.empty())
|
||||
table_join->setUsedColumn(columns_from_right_table.front(), JoinTableSide::Right);
|
||||
}
|
||||
|
||||
const Block & right_header = right_plan.getCurrentHeader();
|
||||
auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context);
|
||||
|
||||
auto result_plan = QueryPlan();
|
||||
@ -1618,13 +1684,26 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
}
|
||||
|
||||
auto join_pipeline_type = join_algorithm->pipelineType();
|
||||
|
||||
ColumnIdentifierSet outer_scope_columns_nonempty;
|
||||
if (outer_scope_columns.empty())
|
||||
{
|
||||
if (left_header.columns() > 1)
|
||||
outer_scope_columns_nonempty.insert(left_header.getByPosition(0).name);
|
||||
else if (right_header.columns() > 1)
|
||||
outer_scope_columns_nonempty.insert(right_header.getByPosition(0).name);
|
||||
}
|
||||
|
||||
auto join_step = std::make_unique<JoinStep>(
|
||||
left_plan.getCurrentHeader(),
|
||||
right_plan.getCurrentHeader(),
|
||||
std::move(join_algorithm),
|
||||
settings[Setting::max_block_size],
|
||||
settings[Setting::max_threads],
|
||||
false /*optimize_read_in_order*/);
|
||||
outer_scope_columns.empty() ? outer_scope_columns_nonempty : outer_scope_columns,
|
||||
false /*optimize_read_in_order*/,
|
||||
true /*optimize_skip_unused_shards*/);
|
||||
join_step->inner_table_selection_mode = settings[Setting::query_plan_join_inner_table_selection];
|
||||
|
||||
join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type));
|
||||
|
||||
@ -1635,47 +1714,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
result_plan.unitePlans(std::move(join_step), {std::move(plans)});
|
||||
}
|
||||
|
||||
ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentHeader().getColumnsWithTypeAndName());
|
||||
ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs;
|
||||
std::unordered_set<std::string_view> drop_unused_columns_after_join_actions_dag_updated_outputs_names;
|
||||
std::optional<size_t> first_skipped_column_node_index;
|
||||
|
||||
auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs();
|
||||
size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size();
|
||||
|
||||
for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i)
|
||||
const auto & header_after_join = result_plan.getCurrentHeader();
|
||||
if (header_after_join.columns() > outer_scope_columns.size())
|
||||
{
|
||||
const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i];
|
||||
|
||||
const auto & global_planner_context = planner_context->getGlobalPlannerContext();
|
||||
if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name)
|
||||
|| !global_planner_context->hasColumnIdentifier(output->result_name))
|
||||
continue;
|
||||
|
||||
if (!outer_scope_columns.contains(output->result_name))
|
||||
auto drop_unused_columns_after_join_actions_dag = createStepToDropColumns(header_after_join, outer_scope_columns, planner_context);
|
||||
if (drop_unused_columns_after_join_actions_dag)
|
||||
{
|
||||
if (!first_skipped_column_node_index)
|
||||
first_skipped_column_node_index = i;
|
||||
continue;
|
||||
auto drop_unused_columns_after_join_transform_step = std::make_unique<ExpressionStep>(result_plan.getCurrentHeader(), std::move(*drop_unused_columns_after_join_actions_dag));
|
||||
drop_unused_columns_after_join_transform_step->setStepDescription("Drop unused columns after JOIN");
|
||||
result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step));
|
||||
}
|
||||
|
||||
drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output);
|
||||
drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name);
|
||||
}
|
||||
|
||||
/** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope.
|
||||
*
|
||||
* Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2;
|
||||
*/
|
||||
if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index)
|
||||
drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]);
|
||||
|
||||
drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs);
|
||||
|
||||
auto drop_unused_columns_after_join_transform_step = std::make_unique<ExpressionStep>(result_plan.getCurrentHeader(), std::move(drop_unused_columns_after_join_actions_dag));
|
||||
drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN");
|
||||
result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step));
|
||||
|
||||
for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies)
|
||||
left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy);
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Processors/Transforms/ColumnPermuteTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -36,6 +37,37 @@ std::vector<std::pair<String, String>> describeJoinActions(const JoinPtr & join)
|
||||
return description;
|
||||
}
|
||||
|
||||
std::vector<size_t> getPermutationForBlock(
|
||||
const Block & block,
|
||||
const Block & lhs_block,
|
||||
const Block & rhs_block,
|
||||
const NameSet & name_filter)
|
||||
{
|
||||
std::vector<size_t> permutation;
|
||||
permutation.reserve(block.columns());
|
||||
Block::NameMap name_map = block.getNamesToIndexesMap();
|
||||
|
||||
bool is_trivial = true;
|
||||
for (const auto & other_block : {lhs_block, rhs_block})
|
||||
{
|
||||
for (const auto & col : other_block)
|
||||
{
|
||||
if (!name_filter.contains(col.name))
|
||||
continue;
|
||||
if (auto it = name_map.find(col.name); it != name_map.end())
|
||||
{
|
||||
is_trivial = is_trivial && it->second == permutation.size();
|
||||
permutation.push_back(it->second);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (is_trivial && permutation.size() == block.columns())
|
||||
return {};
|
||||
|
||||
return permutation;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
JoinStep::JoinStep(
|
||||
@ -44,8 +76,15 @@ JoinStep::JoinStep(
|
||||
JoinPtr join_,
|
||||
size_t max_block_size_,
|
||||
size_t max_streams_,
|
||||
bool keep_left_read_in_order_)
|
||||
: join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_)
|
||||
NameSet required_output_,
|
||||
bool keep_left_read_in_order_,
|
||||
bool use_new_analyzer_)
|
||||
: join(std::move(join_))
|
||||
, max_block_size(max_block_size_)
|
||||
, max_streams(max_streams_)
|
||||
, required_output(std::move(required_output_))
|
||||
, keep_left_read_in_order(keep_left_read_in_order_)
|
||||
, use_new_analyzer(use_new_analyzer_)
|
||||
{
|
||||
updateInputHeaders({left_header_, right_header_});
|
||||
}
|
||||
@ -55,23 +94,43 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines
|
||||
if (pipelines.size() != 2)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps");
|
||||
|
||||
Block lhs_header = pipelines[0]->getHeader();
|
||||
Block rhs_header = pipelines[1]->getHeader();
|
||||
|
||||
if (swap_streams)
|
||||
std::swap(pipelines[0], pipelines[1]);
|
||||
|
||||
if (join->pipelineType() == JoinPipelineType::YShaped)
|
||||
{
|
||||
auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped(
|
||||
std::move(pipelines[0]), std::move(pipelines[1]), join, *output_header, max_block_size, &processors);
|
||||
std::move(pipelines[0]), std::move(pipelines[1]), join, join_algorithm_header, max_block_size, &processors);
|
||||
joined_pipeline->resize(max_streams);
|
||||
return joined_pipeline;
|
||||
}
|
||||
|
||||
return QueryPipelineBuilder::joinPipelinesRightLeft(
|
||||
auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft(
|
||||
std::move(pipelines[0]),
|
||||
std::move(pipelines[1]),
|
||||
join,
|
||||
*output_header,
|
||||
join_algorithm_header,
|
||||
max_block_size,
|
||||
max_streams,
|
||||
keep_left_read_in_order,
|
||||
&processors);
|
||||
|
||||
if (!use_new_analyzer)
|
||||
return pipeline;
|
||||
|
||||
auto column_permutation = getPermutationForBlock(pipeline->getHeader(), lhs_header, rhs_header, required_output);
|
||||
if (!column_permutation.empty())
|
||||
{
|
||||
pipeline->addSimpleTransform([&column_permutation](const Block & header)
|
||||
{
|
||||
return std::make_shared<ColumnPermuteTransform>(header, column_permutation);
|
||||
});
|
||||
}
|
||||
|
||||
return pipeline;
|
||||
}
|
||||
|
||||
bool JoinStep::allowPushDownToRight() const
|
||||
@ -90,17 +149,49 @@ void JoinStep::describeActions(FormatSettings & settings) const
|
||||
|
||||
for (const auto & [name, value] : describeJoinActions(join))
|
||||
settings.out << prefix << name << ": " << value << '\n';
|
||||
if (swap_streams)
|
||||
settings.out << prefix << "Swapped: true\n";
|
||||
}
|
||||
|
||||
void JoinStep::describeActions(JSONBuilder::JSONMap & map) const
|
||||
{
|
||||
for (const auto & [name, value] : describeJoinActions(join))
|
||||
map.add(name, value);
|
||||
if (swap_streams)
|
||||
map.add("Swapped", true);
|
||||
}
|
||||
|
||||
void JoinStep::setJoin(JoinPtr join_, bool swap_streams_)
|
||||
{
|
||||
join_algorithm_header.clear();
|
||||
swap_streams = swap_streams_;
|
||||
join = std::move(join_);
|
||||
updateOutputHeader();
|
||||
}
|
||||
|
||||
void JoinStep::updateOutputHeader()
|
||||
{
|
||||
output_header = JoiningTransform::transformHeader(input_headers.front(), join);
|
||||
if (join_algorithm_header)
|
||||
return;
|
||||
|
||||
const auto & header = swap_streams ? input_headers[1] : input_headers[0];
|
||||
|
||||
Block result_header = JoiningTransform::transformHeader(header, join);
|
||||
join_algorithm_header = result_header;
|
||||
|
||||
if (!use_new_analyzer)
|
||||
{
|
||||
if (swap_streams)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot swap streams without new analyzer");
|
||||
output_header = result_header;
|
||||
return;
|
||||
}
|
||||
|
||||
auto column_permutation = getPermutationForBlock(result_header, input_headers[0], input_headers[1], required_output);
|
||||
if (!column_permutation.empty())
|
||||
result_header = ColumnPermuteTransform::permute(result_header, column_permutation);
|
||||
|
||||
output_header = result_header;
|
||||
}
|
||||
|
||||
static ITransformingStep::Traits getStorageJoinTraits()
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Processors/QueryPlan/IQueryPlanStep.h>
|
||||
#include <Processors/QueryPlan/ITransformingStep.h>
|
||||
#include <Core/Joins.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -19,7 +20,9 @@ public:
|
||||
JoinPtr join_,
|
||||
size_t max_block_size_,
|
||||
size_t max_streams_,
|
||||
bool keep_left_read_in_order_);
|
||||
NameSet required_output_,
|
||||
bool keep_left_read_in_order_,
|
||||
bool use_new_analyzer_);
|
||||
|
||||
String getName() const override { return "Join"; }
|
||||
|
||||
@ -31,16 +34,26 @@ public:
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
const JoinPtr & getJoin() const { return join; }
|
||||
void setJoin(JoinPtr join_) { join = std::move(join_); }
|
||||
void setJoin(JoinPtr join_, bool swap_streams_ = false);
|
||||
bool allowPushDownToRight() const;
|
||||
|
||||
JoinInnerTableSelectionMode inner_table_selection_mode = JoinInnerTableSelectionMode::Right;
|
||||
|
||||
private:
|
||||
void updateOutputHeader() override;
|
||||
|
||||
/// Header that expected to be returned from IJoin
|
||||
Block join_algorithm_header;
|
||||
|
||||
JoinPtr join;
|
||||
size_t max_block_size;
|
||||
size_t max_streams;
|
||||
|
||||
const NameSet required_output;
|
||||
std::set<size_t> columns_to_remove;
|
||||
bool keep_left_read_in_order;
|
||||
bool use_new_analyzer = false;
|
||||
bool swap_streams = false;
|
||||
};
|
||||
|
||||
/// Special step for the case when Join is already filled.
|
||||
|
@ -113,6 +113,7 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack);
|
||||
void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes);
|
||||
void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes);
|
||||
void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &);
|
||||
void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &);
|
||||
void optimizeDistinctInOrder(QueryPlan::Node & node, QueryPlan::Nodes &);
|
||||
|
||||
/// A separate tree traverse to apply sorting properties after *InOrder optimizations.
|
||||
|
102
src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp
Normal file
102
src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp
Normal file
@ -0,0 +1,102 @@
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
#include <Processors/QueryPlan/FilterStep.h>
|
||||
#include <Processors/QueryPlan/ITransformingStep.h>
|
||||
#include <Processors/QueryPlan/JoinStep.h>
|
||||
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
|
||||
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
#include <Processors/QueryPlan/SortingStep.h>
|
||||
#include <Storages/StorageMemory.h>
|
||||
#include <Processors/QueryPlan/ReadFromMemoryStorageStep.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/IJoin.h>
|
||||
#include <Interpreters/HashJoin/HashJoin.h>
|
||||
|
||||
#include <Interpreters/TableJoin.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Joins.h>
|
||||
#include <ranges>
|
||||
|
||||
namespace DB::QueryPlanOptimizations
|
||||
{
|
||||
|
||||
static std::optional<UInt64> estimateReadRowsCount(QueryPlan::Node & node)
|
||||
{
|
||||
IQueryPlanStep * step = node.step.get();
|
||||
if (const auto * reading = typeid_cast<const ReadFromMergeTree *>(step))
|
||||
{
|
||||
if (auto analyzed_result = reading->getAnalyzedResult())
|
||||
return analyzed_result->selected_rows;
|
||||
if (auto analyzed_result = reading->selectRangesToRead())
|
||||
return analyzed_result->selected_rows;
|
||||
return {};
|
||||
}
|
||||
|
||||
if (const auto * reading = typeid_cast<const ReadFromMemoryStorageStep *>(step))
|
||||
return reading->getStorage()->totalRows(Settings{});
|
||||
|
||||
if (node.children.size() != 1)
|
||||
return {};
|
||||
|
||||
if (typeid_cast<ExpressionStep *>(step) || typeid_cast<FilterStep *>(step))
|
||||
return estimateReadRowsCount(*node.children.front());
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &)
|
||||
{
|
||||
auto * join_step = typeid_cast<JoinStep *>(node.step.get());
|
||||
if (!join_step || node.children.size() != 2)
|
||||
return;
|
||||
|
||||
const auto & join = join_step->getJoin();
|
||||
if (join->pipelineType() != JoinPipelineType::FillRightFirst || !join->isCloneSupported())
|
||||
return;
|
||||
|
||||
const auto & table_join = join->getTableJoin();
|
||||
|
||||
/// Algorithms other than HashJoin may not support OUTER JOINs
|
||||
if (table_join.kind() != JoinKind::Inner && !typeid_cast<const HashJoin *>(join.get()))
|
||||
return;
|
||||
|
||||
/// fixme: USING clause handled specially in join algorithm, so swap breaks it
|
||||
/// fixme: Swapping for SEMI and ANTI joins should be alright, need to try to enable it and test
|
||||
if (table_join.hasUsing() || table_join.strictness() != JoinStrictness::All)
|
||||
return;
|
||||
|
||||
bool need_swap = false;
|
||||
if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Auto)
|
||||
{
|
||||
auto lhs_extimation = estimateReadRowsCount(*node.children[0]);
|
||||
auto rhs_extimation = estimateReadRowsCount(*node.children[1]);
|
||||
LOG_TRACE(getLogger("optimizeJoin"), "Left table estimation: {}, right table estimation: {}",
|
||||
lhs_extimation.transform(toString<UInt64>).value_or("unknown"),
|
||||
rhs_extimation.transform(toString<UInt64>).value_or("unknown"));
|
||||
|
||||
if (lhs_extimation && rhs_extimation && *lhs_extimation < *rhs_extimation)
|
||||
need_swap = true;
|
||||
}
|
||||
else if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Left)
|
||||
{
|
||||
need_swap = true;
|
||||
}
|
||||
|
||||
if (!need_swap)
|
||||
return;
|
||||
|
||||
const auto & headers = join_step->getInputHeaders();
|
||||
if (headers.size() != 2)
|
||||
return;
|
||||
|
||||
const auto & left_stream_input_header = headers.front();
|
||||
const auto & right_stream_input_header = headers.back();
|
||||
|
||||
auto updated_table_join = std::make_shared<TableJoin>(table_join);
|
||||
updated_table_join->swapSides();
|
||||
auto updated_join = join->clone(updated_table_join, right_stream_input_header, left_stream_input_header);
|
||||
join_step->setJoin(std::move(updated_join), /* swap_streams= */ true);
|
||||
}
|
||||
|
||||
}
|
@ -227,6 +227,9 @@ void addStepsToBuildSets(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::No
|
||||
/// NOTE: frame cannot be safely used after stack was modified.
|
||||
auto & frame = stack.back();
|
||||
|
||||
if (frame.next_child == 0)
|
||||
optimizeJoin(*frame.node, nodes);
|
||||
|
||||
/// Traverse all children first.
|
||||
if (frame.next_child < frame.node->children.size())
|
||||
{
|
||||
|
@ -35,6 +35,8 @@ public:
|
||||
|
||||
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
const StoragePtr & getStorage() const { return storage; }
|
||||
|
||||
private:
|
||||
static constexpr auto name = "ReadFromMemoryStorage";
|
||||
|
||||
|
49
src/Processors/Transforms/ColumnPermuteTransform.cpp
Normal file
49
src/Processors/Transforms/ColumnPermuteTransform.cpp
Normal file
@ -0,0 +1,49 @@
|
||||
#include <Processors/Transforms/ColumnPermuteTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
void applyPermutation(std::vector<T> & data, const std::vector<size_t> & permutation)
|
||||
{
|
||||
std::vector<T> res;
|
||||
res.reserve(permutation.size());
|
||||
for (size_t i : permutation)
|
||||
res.push_back(data[i]);
|
||||
data = std::move(res);
|
||||
}
|
||||
|
||||
void permuteChunk(Chunk & chunk, const std::vector<size_t> & permutation)
|
||||
{
|
||||
size_t num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
applyPermutation(columns, permutation);
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Block ColumnPermuteTransform::permute(const Block & block, const std::vector<size_t> & permutation)
|
||||
{
|
||||
auto columns = block.getColumnsWithTypeAndName();
|
||||
applyPermutation(columns, permutation);
|
||||
return Block(columns);
|
||||
}
|
||||
|
||||
ColumnPermuteTransform::ColumnPermuteTransform(const Block & header_, const std::vector<size_t> & permutation_)
|
||||
: ISimpleTransform(header_, permute(header_, permutation_), false)
|
||||
, permutation(permutation_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
void ColumnPermuteTransform::transform(Chunk & chunk)
|
||||
{
|
||||
permuteChunk(chunk, permutation);
|
||||
}
|
||||
|
||||
|
||||
}
|
30
src/Processors/Transforms/ColumnPermuteTransform.h
Normal file
30
src/Processors/Transforms/ColumnPermuteTransform.h
Normal file
@ -0,0 +1,30 @@
|
||||
#pragma once
|
||||
|
||||
#include <atomic>
|
||||
#include <mutex>
|
||||
#include <vector>
|
||||
#include <Processors/ISimpleTransform.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Interpreters/Set.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ColumnPermuteTransform : public ISimpleTransform
|
||||
{
|
||||
public:
|
||||
ColumnPermuteTransform(const Block & header_, const std::vector<size_t> & permutation_);
|
||||
|
||||
String getName() const override { return "ColumnPermuteTransform"; }
|
||||
|
||||
void transform(Chunk & chunk) override;
|
||||
|
||||
static Block permute(const Block & block, const std::vector<size_t> & permutation);
|
||||
|
||||
private:
|
||||
Names column_names;
|
||||
std::vector<size_t> permutation;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -19,6 +19,7 @@ Block JoiningTransform::transformHeader(Block header, const JoinPtr & join)
|
||||
join->initialize(header);
|
||||
ExtraBlockPtr tmp;
|
||||
join->joinBlock(header, tmp);
|
||||
materializeBlockInplace(header);
|
||||
LOG_TEST(getLogger("JoiningTransform"), "After join block: '{}'", header.dumpStructure());
|
||||
return header;
|
||||
}
|
||||
|
@ -2343,11 +2343,16 @@ void MergeTreeData::stopOutdatedAndUnexpectedDataPartsLoadingTask()
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeData::prewarmMarkCache(ThreadPool & pool)
|
||||
void MergeTreeData::prewarmMarkCacheIfNeeded(ThreadPool & pool)
|
||||
{
|
||||
if (!(*getSettings())[MergeTreeSetting::prewarm_mark_cache])
|
||||
return;
|
||||
|
||||
prewarmMarkCache(pool);
|
||||
}
|
||||
|
||||
void MergeTreeData::prewarmMarkCache(ThreadPool & pool)
|
||||
{
|
||||
auto * mark_cache = getContext()->getMarkCache().get();
|
||||
if (!mark_cache)
|
||||
return;
|
||||
|
@ -508,6 +508,7 @@ public:
|
||||
|
||||
/// Prewarm mark cache for the most recent data parts.
|
||||
void prewarmMarkCache(ThreadPool & pool);
|
||||
void prewarmMarkCacheIfNeeded(ThreadPool & pool);
|
||||
|
||||
String getLogName() const { return log.loadName(); }
|
||||
|
||||
|
@ -517,9 +517,19 @@ std::unique_ptr<ReadBufferFromFileBase> StorageObjectStorageSource::createReadBu
|
||||
|
||||
LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size);
|
||||
|
||||
bool prefer_bigger_buffer_size = impl->isCached();
|
||||
size_t buffer_size = prefer_bigger_buffer_size
|
||||
? std::max<size_t>(read_settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: read_settings.remote_fs_buffer_size;
|
||||
if (object_size)
|
||||
buffer_size = std::min<size_t>(object_size, buffer_size);
|
||||
|
||||
auto & reader = context_->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER);
|
||||
impl = std::make_unique<AsynchronousBoundedReadBuffer>(
|
||||
std::move(impl), reader, modified_read_settings,
|
||||
std::move(impl),
|
||||
reader,
|
||||
modified_read_settings,
|
||||
buffer_size,
|
||||
context_->getAsyncReadCounters(),
|
||||
context_->getFilesystemReadPrefetchesLog());
|
||||
|
||||
|
@ -1,233 +0,0 @@
|
||||
#include <Storages/StorageExternalDistributed.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Core/PostgreSQL/PoolWithFailover.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/parseAddress.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
#include <Storages/StorageMySQL.h>
|
||||
#include <Storages/MySQL/MySQLSettings.h>
|
||||
#include <Storages/StoragePostgreSQL.h>
|
||||
#include <Storages/StorageURL.h>
|
||||
#include <Storages/MySQL/MySQLHelpers.h>
|
||||
#include <Storages/NamedCollectionsHelpers.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Processors/QueryPlan/UnionStep.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace Setting
|
||||
{
|
||||
extern const SettingsUInt64 glob_expansion_max_elements;
|
||||
extern const SettingsUInt64 postgresql_connection_attempt_timeout;
|
||||
extern const SettingsBool postgresql_connection_pool_auto_close_connection;
|
||||
extern const SettingsUInt64 postgresql_connection_pool_retries;
|
||||
extern const SettingsUInt64 postgresql_connection_pool_size;
|
||||
extern const SettingsUInt64 postgresql_connection_pool_wait_timeout;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
StorageExternalDistributed::StorageExternalDistributed(
|
||||
const StorageID & table_id_,
|
||||
std::unordered_set<StoragePtr> && shards_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const String & comment)
|
||||
: IStorage(table_id_)
|
||||
, shards(shards_)
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
storage_metadata.setConstraints(constraints_);
|
||||
storage_metadata.setComment(comment);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
}
|
||||
|
||||
void StorageExternalDistributed::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
size_t num_streams)
|
||||
{
|
||||
std::vector<std::unique_ptr<QueryPlan>> plans;
|
||||
for (const auto & shard : shards)
|
||||
{
|
||||
plans.emplace_back(std::make_unique<QueryPlan>());
|
||||
shard->read(
|
||||
*plans.back(),
|
||||
column_names,
|
||||
storage_snapshot,
|
||||
query_info,
|
||||
context,
|
||||
processed_stage,
|
||||
max_block_size,
|
||||
num_streams
|
||||
);
|
||||
}
|
||||
|
||||
if (plans.empty())
|
||||
{
|
||||
auto header = storage_snapshot->getSampleBlockForColumns(column_names);
|
||||
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info);
|
||||
}
|
||||
|
||||
if (plans.size() == 1)
|
||||
{
|
||||
query_plan = std::move(*plans.front());
|
||||
return;
|
||||
}
|
||||
|
||||
Headers input_headers;
|
||||
input_headers.reserve(plans.size());
|
||||
for (auto & plan : plans)
|
||||
input_headers.emplace_back(plan->getCurrentHeader());
|
||||
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(input_headers));
|
||||
query_plan.unitePlans(std::move(union_step), std::move(plans));
|
||||
}
|
||||
|
||||
void registerStorageExternalDistributed(StorageFactory & factory)
|
||||
{
|
||||
factory.registerStorage("ExternalDistributed", [](const StorageFactory::Arguments & args)
|
||||
{
|
||||
ASTs & engine_args = args.engine_args;
|
||||
if (engine_args.size() < 2)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Engine ExternalDistributed must have at least 2 arguments: "
|
||||
"engine_name, named_collection and/or description");
|
||||
|
||||
auto context = args.getLocalContext();
|
||||
const auto & settings = context->getSettingsRef();
|
||||
size_t max_addresses = settings[Setting::glob_expansion_max_elements];
|
||||
auto get_addresses = [&](const std::string addresses_expr)
|
||||
{
|
||||
return parseRemoteDescription(addresses_expr, 0, addresses_expr.size(), ',', max_addresses);
|
||||
};
|
||||
|
||||
std::unordered_set<StoragePtr> shards;
|
||||
ASTs inner_engine_args(engine_args.begin() + 1, engine_args.end());
|
||||
|
||||
ASTPtr * address_arg = nullptr;
|
||||
|
||||
/// If there is a named collection argument, named `addresses_expr`
|
||||
for (auto & node : inner_engine_args)
|
||||
{
|
||||
if (ASTFunction * func = node->as<ASTFunction>(); func && func->name == "equals" && func->arguments)
|
||||
{
|
||||
if (ASTExpressionList * func_args = func->arguments->as<ASTExpressionList>(); func_args && func_args->children.size() == 2)
|
||||
{
|
||||
if (ASTIdentifier * arg_name = func_args->children[0]->as<ASTIdentifier>(); arg_name && arg_name->name() == "addresses_expr")
|
||||
{
|
||||
address_arg = &func_args->children[1];
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Otherwise it is the first argument.
|
||||
if (!address_arg)
|
||||
address_arg = &inner_engine_args.at(0);
|
||||
|
||||
String addresses_expr = checkAndGetLiteralArgument<String>(*address_arg, "addresses");
|
||||
Strings shards_addresses = get_addresses(addresses_expr);
|
||||
|
||||
auto engine_name = checkAndGetLiteralArgument<String>(engine_args[0], "engine_name");
|
||||
if (engine_name == "URL")
|
||||
{
|
||||
auto format_settings = StorageURL::getFormatSettingsFromArgs(args);
|
||||
for (const auto & shard_address : shards_addresses)
|
||||
{
|
||||
*address_arg = std::make_shared<ASTLiteral>(shard_address);
|
||||
auto configuration = StorageURL::getConfiguration(inner_engine_args, context);
|
||||
auto uri_options = parseRemoteDescription(shard_address, 0, shard_address.size(), '|', max_addresses);
|
||||
if (uri_options.size() > 1)
|
||||
{
|
||||
shards.insert(
|
||||
std::make_shared<StorageURLWithFailover>(
|
||||
uri_options, args.table_id, configuration.format, format_settings,
|
||||
args.columns, args.constraints, context, configuration.compression_method));
|
||||
}
|
||||
else
|
||||
{
|
||||
shards.insert(std::make_shared<StorageURL>(
|
||||
shard_address, args.table_id, configuration.format, format_settings,
|
||||
args.columns, args.constraints, String{}, context, configuration.compression_method));
|
||||
}
|
||||
}
|
||||
}
|
||||
#if USE_MYSQL
|
||||
else if (engine_name == "MySQL")
|
||||
{
|
||||
MySQLSettings mysql_settings;
|
||||
for (const auto & shard_address : shards_addresses)
|
||||
{
|
||||
*address_arg = std::make_shared<ASTLiteral>(shard_address);
|
||||
auto configuration = StorageMySQL::getConfiguration(inner_engine_args, context, mysql_settings);
|
||||
configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 3306);
|
||||
auto pool = createMySQLPoolWithFailover(configuration, mysql_settings);
|
||||
shards.insert(std::make_shared<StorageMySQL>(
|
||||
args.table_id, std::move(pool), configuration.database, configuration.table,
|
||||
/* replace_query = */ false, /* on_duplicate_clause = */ "",
|
||||
args.columns, args.constraints, String{}, context, mysql_settings));
|
||||
}
|
||||
}
|
||||
#endif
|
||||
#if USE_LIBPQXX
|
||||
else if (engine_name == "PostgreSQL")
|
||||
{
|
||||
for (const auto & shard_address : shards_addresses)
|
||||
{
|
||||
*address_arg = std::make_shared<ASTLiteral>(shard_address);
|
||||
auto configuration = StoragePostgreSQL::getConfiguration(inner_engine_args, context);
|
||||
configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 5432);
|
||||
auto pool = std::make_shared<postgres::PoolWithFailover>(
|
||||
configuration,
|
||||
settings[Setting::postgresql_connection_pool_size],
|
||||
settings[Setting::postgresql_connection_pool_wait_timeout],
|
||||
settings[Setting::postgresql_connection_pool_retries],
|
||||
settings[Setting::postgresql_connection_pool_auto_close_connection],
|
||||
settings[Setting::postgresql_connection_attempt_timeout]);
|
||||
shards.insert(std::make_shared<StoragePostgreSQL>(
|
||||
args.table_id, std::move(pool), configuration.table, args.columns, args.constraints, String{}, context));
|
||||
}
|
||||
}
|
||||
#endif
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"External storage engine {} is not supported for StorageExternalDistributed. "
|
||||
"Supported engines are: MySQL, PostgreSQL, URL",
|
||||
engine_name);
|
||||
}
|
||||
|
||||
return std::make_shared<StorageExternalDistributed>(
|
||||
args.table_id,
|
||||
std::move(shards),
|
||||
args.columns,
|
||||
args.constraints,
|
||||
args.comment);
|
||||
},
|
||||
{
|
||||
.source_access_type = AccessType::SOURCES,
|
||||
});
|
||||
}
|
||||
|
||||
}
|
@ -1,43 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Storages MySQL and PostgreSQL use ConnectionPoolWithFailover and support multiple replicas.
|
||||
/// This class unites multiple storages with replicas into multiple shards with replicas.
|
||||
/// A query to external database is passed to one replica on each shard, the result is united.
|
||||
/// Replicas on each shard have the same priority, traversed replicas are moved to the end of the queue.
|
||||
/// Similar approach is used for URL storage.
|
||||
class StorageExternalDistributed final : public DB::IStorage
|
||||
{
|
||||
public:
|
||||
StorageExternalDistributed(
|
||||
const StorageID & table_id_,
|
||||
std::unordered_set<StoragePtr> && shards_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const String & comment);
|
||||
|
||||
std::string getName() const override { return "ExternalDistributed"; }
|
||||
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
size_t num_streams) override;
|
||||
|
||||
private:
|
||||
using Shards = std::unordered_set<StoragePtr>;
|
||||
Shards shards;
|
||||
};
|
||||
|
||||
}
|
@ -155,7 +155,7 @@ StorageMergeTree::StorageMergeTree(
|
||||
|
||||
loadMutations();
|
||||
loadDeduplicationLog();
|
||||
prewarmMarkCache(getActivePartsLoadingThreadPool().get());
|
||||
prewarmMarkCacheIfNeeded(getActivePartsLoadingThreadPool().get());
|
||||
}
|
||||
|
||||
|
||||
|
@ -509,7 +509,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
||||
}
|
||||
|
||||
loadDataParts(skip_sanity_checks, expected_parts_on_this_replica);
|
||||
prewarmMarkCache(getActivePartsLoadingThreadPool().get());
|
||||
prewarmMarkCacheIfNeeded(getActivePartsLoadingThreadPool().get());
|
||||
|
||||
if (LoadingStrictnessLevel::ATTACH <= mode)
|
||||
{
|
||||
|
@ -227,6 +227,194 @@ FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
/// Default per host dashboard for self-managed ClickHouse
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Queries/second" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_Query)
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "CPU Usage (cores)" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSCPUVirtualTimeMicroseconds) / 1000000
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Queries Running" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(CurrentMetric_Query)
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Merges Running" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(CurrentMetric_Merge)
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Selected Bytes/second" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_SelectedBytes)
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "IO Wait" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSIOWaitMicroseconds) / 1000000
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "CPU Wait" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSCPUWaitMicroseconds) / 1000000
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "OS CPU Usage (Userspace)" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value)
|
||||
FROM merge('system', '^asynchronous_metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSUserTimeNormalized'
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "OS CPU Usage (Kernel)" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value)
|
||||
FROM merge('system', '^asynchronous_metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSSystemTimeNormalized'
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Read From Disk" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSReadBytes)
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Read From Filesystem" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSReadChars)
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Memory (tracked)" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(CurrentMetric_MemoryTracking)
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Load Average (15 minutes)" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value)
|
||||
FROM merge('system', '^asynchronous_metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'LoadAverage15'
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Selected Rows/second" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_SelectedRows)
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Inserted Rows/second" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_InsertedRows)
|
||||
FROM merge('system', '^metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Total MergeTree Parts" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value)
|
||||
FROM merge('system', '^asynchronous_metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'TotalPartsOfMergeTreeTables'
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Overview (host)" },
|
||||
{ "title", "Max Parts For Partition" },
|
||||
{ "query", trim(R"EOQ(
|
||||
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, max(value)
|
||||
FROM merge('system', '^asynchronous_metric_log')
|
||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'MaxPartCountForPartition'
|
||||
GROUP BY t, hostname
|
||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
)EOQ") }
|
||||
},
|
||||
/// Default dashboard for ClickHouse Cloud
|
||||
@ -369,7 +557,143 @@ ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||
{ "dashboard", "Cloud overview" },
|
||||
{ "title", "Concurrent network connections" },
|
||||
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(TCP_Connections), max(MySQL_Connections), max(HTTP_Connections) FROM (SELECT event_time, sum(CurrentMetric_TCPConnection) AS TCP_Connections, sum(CurrentMetric_MySQLConnection) AS MySQL_Connections, sum(CurrentMetric_HTTPConnection) AS HTTP_Connections FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
}
|
||||
},
|
||||
/// Default per host dashboard for ClickHouse Cloud
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Queries/second" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_Query) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "CPU Usage (cores)" },
|
||||
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(metric) / 1000000\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSCPUVirtualTimeMicroseconds) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32} GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Queries Running" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_Query) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Merges Running" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_Merge) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Selected Bytes/second" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_SelectedBytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "IO Wait (local fs)" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSIOWaitMicroseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "S3 read wait" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_ReadBufferFromS3Microseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "S3 read errors/sec" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_ReadBufferFromS3RequestsErrors) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "CPU Wait" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSCPUWaitMicroseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "OS CPU Usage (Userspace, normalized)" },
|
||||
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'OSUserTimeNormalized'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "OS CPU Usage (Kernel, normalized)" },
|
||||
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'OSSystemTimeNormalized'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Read From Disk (bytes/sec)" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSReadBytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Read From Filesystem (bytes/sec)" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSReadChars) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Memory (tracked, bytes)" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_MemoryTracking) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Load Average (15 minutes)" },
|
||||
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM (\n SELECT event_time, hostname, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric = 'LoadAverage15'\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Selected Rows/sec" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_SelectedRows) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Inserted Rows/sec" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_InsertedRows) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Total MergeTree Parts" },
|
||||
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, max(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'TotalPartsOfMergeTreeTables'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Max Parts For Partition" },
|
||||
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, max(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'MaxPartCountForPartition'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Read From S3 (bytes/sec)" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_ReadBufferFromS3Bytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Filesystem Cache Size" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_FilesystemCacheSize) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Disk S3 write req/sec" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_DiskS3PutObject + ProfileEvent_DiskS3UploadPart + ProfileEvent_DiskS3CreateMultipartUpload + ProfileEvent_DiskS3CompleteMultipartUpload) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Disk S3 read req/sec" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_DiskS3GetObject + ProfileEvent_DiskS3HeadObject + ProfileEvent_DiskS3ListObjects) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t\nWITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "FS cache hit rate" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_CachedReadBufferReadFromCacheBytes) / (sum(ProfileEvent_CachedReadBufferReadFromCacheBytes) + sum(ProfileEvent_CachedReadBufferReadFromSourceBytes)) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Page cache hit rate" },
|
||||
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, greatest(0, (sum(ProfileEvent_OSReadChars) - sum(ProfileEvent_OSReadBytes)) / (sum(ProfileEvent_OSReadChars) + sum(ProfileEvent_ReadBufferFromS3Bytes))) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Network receive bytes/sec" },
|
||||
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM (\n SELECT event_time, hostname, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkReceiveBytes%'\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
{
|
||||
{ "dashboard", "Cloud overview (host)" },
|
||||
{ "title", "Network send bytes/sec" },
|
||||
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM (\n SELECT event_time, hostname, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkSendBytes%'\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||
},
|
||||
};
|
||||
|
||||
auto add_dashboards = [&](const auto & dashboards)
|
||||
|
@ -93,10 +93,6 @@ void registerStoragePostgreSQL(StorageFactory & factory);
|
||||
void registerStorageMaterializedPostgreSQL(StorageFactory & factory);
|
||||
#endif
|
||||
|
||||
#if USE_MYSQL || USE_LIBPQXX
|
||||
void registerStorageExternalDistributed(StorageFactory & factory);
|
||||
#endif
|
||||
|
||||
#if USE_FILELOG
|
||||
void registerStorageFileLog(StorageFactory & factory);
|
||||
#endif
|
||||
@ -205,10 +201,6 @@ void registerStorages(bool use_legacy_mongodb_integration [[maybe_unused]])
|
||||
registerStorageMaterializedPostgreSQL(factory);
|
||||
#endif
|
||||
|
||||
#if USE_MYSQL || USE_LIBPQXX
|
||||
registerStorageExternalDistributed(factory);
|
||||
#endif
|
||||
|
||||
#if USE_SQLITE
|
||||
registerStorageSQLite(factory);
|
||||
#endif
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/StorageExternalDistributed.h>
|
||||
#include <Storages/NamedCollectionsHelpers.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
|
@ -789,6 +789,7 @@ def get_localzone():
|
||||
return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:]))
|
||||
|
||||
|
||||
# Refer to `tests/integration/helpers/random_settings.py` for integration test random settings
|
||||
class SettingsRandomizer:
|
||||
settings = {
|
||||
"max_insert_threads": lambda: (
|
||||
@ -919,6 +920,9 @@ class SettingsRandomizer:
|
||||
"max_parsing_threads": lambda: random.choice([0, 1, 10]),
|
||||
"optimize_functions_to_subcolumns": lambda: random.randint(0, 1),
|
||||
"parallel_replicas_local_plan": lambda: random.randint(0, 1),
|
||||
"query_plan_join_inner_table_selection": lambda: random.choice(
|
||||
["left", "auto", "right"]
|
||||
),
|
||||
"output_format_native_write_json_as_string": lambda: random.randint(0, 1),
|
||||
"enable_vertical_final": lambda: random.randint(0, 1),
|
||||
}
|
||||
|
@ -67,6 +67,7 @@ DEFAULT_ENV_NAME = ".env"
|
||||
DEFAULT_BASE_CONFIG_DIR = os.environ.get(
|
||||
"CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/"
|
||||
)
|
||||
DOCKER_BASE_TAG = os.environ.get("DOCKER_BASE_TAG", "latest")
|
||||
|
||||
SANITIZER_SIGN = "=================="
|
||||
|
||||
@ -503,7 +504,6 @@ class ClickHouseCluster:
|
||||
"CLICKHOUSE_TESTS_DOCKERD_HOST"
|
||||
)
|
||||
self.docker_api_version = os.environ.get("DOCKER_API_VERSION")
|
||||
self.docker_base_tag = os.environ.get("DOCKER_BASE_TAG", "latest")
|
||||
|
||||
self.base_cmd = ["docker", "compose"]
|
||||
if custom_dockerd_host:
|
||||
@ -1079,7 +1079,7 @@ class ClickHouseCluster:
|
||||
|
||||
env_variables["keeper_binary"] = binary_path
|
||||
env_variables["keeper_cmd_prefix"] = keeper_cmd_prefix
|
||||
env_variables["image"] = "clickhouse/integration-test:" + self.docker_base_tag
|
||||
env_variables["image"] = "clickhouse/integration-test:" + DOCKER_BASE_TAG
|
||||
env_variables["user"] = str(os.getuid())
|
||||
env_variables["keeper_fs"] = "bind"
|
||||
for i in range(1, 4):
|
||||
@ -1675,7 +1675,7 @@ class ClickHouseCluster:
|
||||
)
|
||||
|
||||
if tag is None:
|
||||
tag = self.docker_base_tag
|
||||
tag = DOCKER_BASE_TAG
|
||||
if not env_variables:
|
||||
env_variables = {}
|
||||
self.use_keeper = use_keeper
|
||||
@ -4538,7 +4538,12 @@ class ClickHouseInstance:
|
||||
if len(self.custom_dictionaries_paths):
|
||||
write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir)
|
||||
|
||||
if self.randomize_settings and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR:
|
||||
if (
|
||||
self.randomize_settings
|
||||
and self.image == "clickhouse/integration-test"
|
||||
and self.tag == DOCKER_BASE_TAG
|
||||
and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR
|
||||
):
|
||||
# If custom main config is used, do not apply random settings to it
|
||||
write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml")
|
||||
|
||||
|
@ -5,6 +5,8 @@ def randomize_settings():
|
||||
yield "max_joined_block_size_rows", random.randint(8000, 100000)
|
||||
if random.random() < 0.5:
|
||||
yield "max_block_size", random.randint(8000, 100000)
|
||||
if random.random() < 0.5:
|
||||
yield "query_plan_join_inner_table_selection", random.choice(["auto", "left"])
|
||||
|
||||
|
||||
def write_random_settings_config(destination):
|
||||
|
@ -91,7 +91,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster):
|
||||
with client(name="client1>", log=client_output, command=command_text) as client1:
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number",
|
||||
"SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS query_plan_join_inner_table_selection = 'right'",
|
||||
)
|
||||
client1.expect("Peak memory usage", timeout=60)
|
||||
client1.expect(prompt)
|
||||
|
@ -386,100 +386,6 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('
|
||||
conn.close()
|
||||
|
||||
|
||||
def test_mysql_distributed(started_cluster):
|
||||
table_name = "test_replicas"
|
||||
|
||||
conn1 = get_mysql_conn(started_cluster, started_cluster.mysql8_ip)
|
||||
conn2 = get_mysql_conn(started_cluster, started_cluster.mysql2_ip)
|
||||
conn3 = get_mysql_conn(started_cluster, started_cluster.mysql3_ip)
|
||||
conn4 = get_mysql_conn(started_cluster, started_cluster.mysql4_ip)
|
||||
|
||||
create_mysql_db(conn1, "clickhouse")
|
||||
create_mysql_db(conn2, "clickhouse")
|
||||
create_mysql_db(conn3, "clickhouse")
|
||||
create_mysql_db(conn4, "clickhouse")
|
||||
|
||||
create_mysql_table(conn1, table_name)
|
||||
create_mysql_table(conn2, table_name)
|
||||
create_mysql_table(conn3, table_name)
|
||||
create_mysql_table(conn4, table_name)
|
||||
|
||||
node2.query("DROP TABLE IF EXISTS test_replicas")
|
||||
|
||||
# Storage with with 3 replicas
|
||||
node2.query(
|
||||
"""
|
||||
CREATE TABLE test_replicas
|
||||
(id UInt32, name String, age UInt32, money UInt32)
|
||||
ENGINE = MySQL('mysql{2|3|4}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
|
||||
)
|
||||
|
||||
# Fill remote tables with different data to be able to check
|
||||
nodes = [node1, node2, node2, node2]
|
||||
for i in range(1, 5):
|
||||
nodes[i - 1].query("DROP TABLE IF EXISTS test_replica{}".format(i))
|
||||
nodes[i - 1].query(
|
||||
"""
|
||||
CREATE TABLE test_replica{}
|
||||
(id UInt32, name String, age UInt32, money UInt32)
|
||||
ENGINE = MySQL('mysql{}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse');""".format(
|
||||
i, 80 if i == 1 else i
|
||||
)
|
||||
)
|
||||
nodes[i - 1].query(
|
||||
"INSERT INTO test_replica{} (id, name) SELECT number, 'host{}' from numbers(10) ".format(
|
||||
i, i
|
||||
)
|
||||
)
|
||||
|
||||
# test multiple ports parsing
|
||||
result = node2.query(
|
||||
"""SELECT DISTINCT(name) FROM mysql('mysql{80|2|3}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
|
||||
)
|
||||
assert result == "host1\n" or result == "host2\n" or result == "host3\n"
|
||||
result = node2.query(
|
||||
"""SELECT DISTINCT(name) FROM mysql('mysql80:3306|mysql2:3306|mysql3:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
|
||||
)
|
||||
assert result == "host1\n" or result == "host2\n" or result == "host3\n"
|
||||
|
||||
# check all replicas are traversed
|
||||
query = "SELECT * FROM ("
|
||||
for i in range(3):
|
||||
query += "SELECT name FROM test_replicas UNION DISTINCT "
|
||||
query += "SELECT name FROM test_replicas) ORDER BY name"
|
||||
|
||||
result = node2.query(query)
|
||||
assert result == "host2\nhost3\nhost4\n"
|
||||
|
||||
# Storage with with two shards, each has 2 replicas
|
||||
node2.query("DROP TABLE IF EXISTS test_shards")
|
||||
|
||||
node2.query(
|
||||
"""
|
||||
CREATE TABLE test_shards
|
||||
(id UInt32, name String, age UInt32, money UInt32)
|
||||
ENGINE = ExternalDistributed('MySQL', 'mysql{80|2}:3306,mysql{3|4}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
|
||||
)
|
||||
|
||||
# Check only one replica in each shard is used
|
||||
result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
|
||||
assert result == "host1\nhost3\n"
|
||||
|
||||
# check all replicas are traversed
|
||||
query = "SELECT name FROM ("
|
||||
for i in range(3):
|
||||
query += "SELECT name FROM test_shards UNION DISTINCT "
|
||||
query += "SELECT name FROM test_shards) ORDER BY name"
|
||||
result = node2.query(query)
|
||||
assert result == "host1\nhost2\nhost3\nhost4\n"
|
||||
|
||||
# disconnect mysql
|
||||
started_cluster.pause_container("mysql80")
|
||||
result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
|
||||
started_cluster.unpause_container("mysql80")
|
||||
assert result == "host2\nhost4\n" or result == "host3\nhost4\n"
|
||||
|
||||
|
||||
def test_external_settings(started_cluster):
|
||||
table_name = "test_external_settings"
|
||||
node1.query(f"DROP TABLE IF EXISTS {table_name}")
|
||||
|
@ -449,89 +449,6 @@ def test_concurrent_queries(started_cluster):
|
||||
node1.query("DROP TABLE test.stat;")
|
||||
|
||||
|
||||
def test_postgres_distributed(started_cluster):
|
||||
cursor0 = started_cluster.postgres_conn.cursor()
|
||||
cursor1 = started_cluster.postgres2_conn.cursor()
|
||||
cursor2 = started_cluster.postgres3_conn.cursor()
|
||||
cursor3 = started_cluster.postgres4_conn.cursor()
|
||||
cursors = [cursor0, cursor1, cursor2, cursor3]
|
||||
|
||||
for i in range(4):
|
||||
cursors[i].execute("DROP TABLE IF EXISTS test_replicas")
|
||||
cursors[i].execute("CREATE TABLE test_replicas (id Integer, name Text)")
|
||||
cursors[i].execute(
|
||||
f"""INSERT INTO test_replicas select i, 'host{i+1}' from generate_series(0, 99) as t(i);"""
|
||||
)
|
||||
|
||||
# test multiple ports parsing
|
||||
result = node2.query(
|
||||
"""SELECT DISTINCT(name) FROM postgresql('postgres{1|2|3}:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """
|
||||
)
|
||||
assert result == "host1\n" or result == "host2\n" or result == "host3\n"
|
||||
result = node2.query(
|
||||
"""SELECT DISTINCT(name) FROM postgresql('postgres2:5431|postgres3:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """
|
||||
)
|
||||
assert result == "host3\n" or result == "host2\n"
|
||||
|
||||
# Create storage with with 3 replicas
|
||||
node2.query("DROP TABLE IF EXISTS test_replicas")
|
||||
node2.query(
|
||||
"""
|
||||
CREATE TABLE test_replicas
|
||||
(id UInt32, name String)
|
||||
ENGINE = PostgreSQL('postgres{2|3|4}:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """
|
||||
)
|
||||
|
||||
# Check all replicas are traversed
|
||||
query = "SELECT name FROM ("
|
||||
for i in range(3):
|
||||
query += "SELECT name FROM test_replicas UNION DISTINCT "
|
||||
query += "SELECT name FROM test_replicas) ORDER BY name"
|
||||
result = node2.query(query)
|
||||
assert result == "host2\nhost3\nhost4\n"
|
||||
|
||||
# Create storage with with two two shards, each has 2 replicas
|
||||
node2.query("DROP TABLE IF EXISTS test_shards")
|
||||
|
||||
node2.query(
|
||||
"""
|
||||
CREATE TABLE test_shards
|
||||
(id UInt32, name String, age UInt32, money UInt32)
|
||||
ENGINE = ExternalDistributed('PostgreSQL', 'postgres{1|2}:5432,postgres{3|4}:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """
|
||||
)
|
||||
|
||||
# Check only one replica in each shard is used
|
||||
result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
|
||||
assert result == "host1\nhost3\n"
|
||||
|
||||
node2.query(
|
||||
"""
|
||||
CREATE TABLE test_shards2
|
||||
(id UInt32, name String, age UInt32, money UInt32)
|
||||
ENGINE = ExternalDistributed('PostgreSQL', postgres4, addresses_expr='postgres{1|2}:5432,postgres{3|4}:5432'); """
|
||||
)
|
||||
|
||||
result = node2.query("SELECT DISTINCT(name) FROM test_shards2 ORDER BY name")
|
||||
assert result == "host1\nhost3\n"
|
||||
|
||||
# Check all replicas are traversed
|
||||
query = "SELECT name FROM ("
|
||||
for i in range(3):
|
||||
query += "SELECT name FROM test_shards UNION DISTINCT "
|
||||
query += "SELECT name FROM test_shards) ORDER BY name"
|
||||
result = node2.query(query)
|
||||
assert result == "host1\nhost2\nhost3\nhost4\n"
|
||||
|
||||
# Disconnect postgres1
|
||||
started_cluster.pause_container("postgres1")
|
||||
result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
|
||||
started_cluster.unpause_container("postgres1")
|
||||
assert result == "host2\nhost4\n" or result == "host3\nhost4\n"
|
||||
node2.query("DROP TABLE test_shards2")
|
||||
node2.query("DROP TABLE test_shards")
|
||||
node2.query("DROP TABLE test_replicas")
|
||||
|
||||
|
||||
def test_datetime_with_timezone(started_cluster):
|
||||
cursor = started_cluster.postgres_conn.cursor()
|
||||
cursor.execute("DROP TABLE IF EXISTS test_timezone")
|
||||
@ -850,6 +767,7 @@ def test_filter_pushdown(started_cluster):
|
||||
"INSERT INTO test_filter_pushdown.test_table VALUES (1, 10), (1, 110), (2, 0), (3, 33), (4, 0)"
|
||||
)
|
||||
|
||||
node1.query("DROP TABLE IF EXISTS test_filter_pushdown_pg_table")
|
||||
node1.query(
|
||||
"""
|
||||
CREATE TABLE test_filter_pushdown_pg_table (id UInt32, value UInt32)
|
||||
@ -857,12 +775,14 @@ def test_filter_pushdown(started_cluster):
|
||||
"""
|
||||
)
|
||||
|
||||
node1.query("DROP TABLE IF EXISTS test_filter_pushdown_local_table")
|
||||
node1.query(
|
||||
"""
|
||||
CREATE TABLE test_filter_pushdown_local_table (id UInt32, value UInt32) ENGINE Memory AS SELECT * FROM test_filter_pushdown_pg_table
|
||||
"""
|
||||
)
|
||||
|
||||
node1.query("DROP TABLE IF EXISTS ch_table")
|
||||
node1.query(
|
||||
"CREATE TABLE ch_table (id UInt32, pg_id UInt32) ENGINE MergeTree ORDER BY id"
|
||||
)
|
||||
|
@ -15,9 +15,9 @@ INSERT INTO t2_00826 values (1,1), (1,2);
|
||||
INSERT INTO t2_00826 (a) values (2), (3);
|
||||
|
||||
SELECT '--- cross ---';
|
||||
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a;
|
||||
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL;
|
||||
SELECT '--- cross nullable ---';
|
||||
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b;
|
||||
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL;
|
||||
SELECT '--- cross nullable vs not nullable ---';
|
||||
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b ORDER BY t1_00826.a;
|
||||
SELECT '--- cross self ---';
|
||||
@ -41,14 +41,15 @@ SELECT '--- is null or ---';
|
||||
SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a;
|
||||
|
||||
SELECT '--- do not rewrite alias ---';
|
||||
SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0;
|
||||
SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0 ORDER BY ALL;
|
||||
|
||||
SELECT '--- comma ---';
|
||||
SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a;
|
||||
SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL;
|
||||
SELECT '--- comma nullable ---';
|
||||
SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b;
|
||||
SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL;
|
||||
SELECT '--- comma and or ---';
|
||||
SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2);
|
||||
SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2)
|
||||
ORDER BY ALL;
|
||||
|
||||
|
||||
SELECT '--- cross ---';
|
||||
|
@ -20,42 +20,42 @@ select t.a, s.b, s.a, s.b, y.a, y.b from t
|
||||
left join s on (t.a = s.a and s.b = t.b)
|
||||
left join y on (y.a = s.a and y.b = s.b)
|
||||
order by t.a
|
||||
format PrettyCompactNoEscapes;
|
||||
format PrettyCompactMonoBlock;
|
||||
|
||||
select t.a as t_a from t
|
||||
left join s on s.a = t_a
|
||||
order by t.a
|
||||
format PrettyCompactNoEscapes;
|
||||
format PrettyCompactMonoBlock;
|
||||
|
||||
select t.a, s.a as s_a from t
|
||||
left join s on s.a = t.a
|
||||
left join y on y.b = s.b
|
||||
order by t.a
|
||||
format PrettyCompactNoEscapes;
|
||||
format PrettyCompactMonoBlock;
|
||||
|
||||
select t.a, t.a, t.b as t_b from t
|
||||
left join s on t.a = s.a
|
||||
left join y on y.b = s.b
|
||||
order by t.a
|
||||
format PrettyCompactNoEscapes;
|
||||
format PrettyCompactMonoBlock;
|
||||
|
||||
select s.a, s.a, s.b as s_b, s.b from t
|
||||
left join s on s.a = t.a
|
||||
left join y on s.b = y.b
|
||||
order by t.a
|
||||
format PrettyCompactNoEscapes;
|
||||
format PrettyCompactMonoBlock;
|
||||
|
||||
select y.a, y.a, y.b as y_b, y.b from t
|
||||
left join s on s.a = t.a
|
||||
left join y on y.b = s.b
|
||||
order by t.a
|
||||
format PrettyCompactNoEscapes;
|
||||
format PrettyCompactMonoBlock;
|
||||
|
||||
select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t
|
||||
left join s on t.a = s.a
|
||||
left join y on y.b = s.b
|
||||
order by t.a
|
||||
format PrettyCompactNoEscapes;
|
||||
format PrettyCompactMonoBlock;
|
||||
|
||||
drop table t;
|
||||
drop table s;
|
||||
|
@ -1,5 +1,7 @@
|
||||
SET joined_subquery_requires_alias = 0;
|
||||
|
||||
SET query_plan_join_inner_table_selection = 'auto';
|
||||
|
||||
{% for join_algorithm in ['partial_merge', 'hash'] -%}
|
||||
|
||||
SET join_algorithm = '{{ join_algorithm }}';
|
||||
|
@ -18,28 +18,35 @@
|
||||
0 0
|
||||
|
||||
0 0
|
||||
-
|
||||
1 1
|
||||
1 1
|
||||
|
||||
0 0
|
||||
-
|
||||
1 1
|
||||
1 1
|
||||
|
||||
0 0
|
||||
-
|
||||
1 1
|
||||
1 1
|
||||
|
||||
0 0
|
||||
-
|
||||
1 1
|
||||
1 1
|
||||
|
||||
0 0
|
||||
-
|
||||
1 1
|
||||
|
||||
0 0
|
||||
-
|
||||
1 foo 1 1 300
|
||||
|
||||
0 foo 1 0 300
|
||||
-
|
||||
1 100 1970-01-01 1 100 1970-01-01
|
||||
1 100 1970-01-01 1 200 1970-01-02
|
||||
1 200 1970-01-02 1 100 1970-01-01
|
||||
|
@ -64,39 +64,47 @@ USING (id);
|
||||
|
||||
INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02');
|
||||
|
||||
SELECT '-';
|
||||
SELECT *
|
||||
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l
|
||||
LEFT JOIN (SELECT item_id FROM t ) r
|
||||
ON l.item_id = r.item_id;
|
||||
|
||||
SELECT '-';
|
||||
SELECT *
|
||||
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l
|
||||
RIGHT JOIN (SELECT item_id FROM t ) r
|
||||
ON l.item_id = r.item_id;
|
||||
|
||||
SELECT '-';
|
||||
SELECT *
|
||||
FROM (SELECT item_id FROM t) l
|
||||
LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r
|
||||
ON l.item_id = r.item_id;
|
||||
|
||||
SELECT '-';
|
||||
SELECT *
|
||||
FROM (SELECT item_id FROM t) l
|
||||
RIGHT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r
|
||||
ON l.item_id = r.item_id;
|
||||
|
||||
SELECT '-';
|
||||
SELECT *
|
||||
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l
|
||||
LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r
|
||||
ON l.item_id = r.item_id;
|
||||
|
||||
SELECT '-';
|
||||
SELECT *
|
||||
FROM (SELECT item_id, 'foo' AS key, 1 AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l
|
||||
LEFT JOIN (SELECT item_id, sum(price_sold) AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r
|
||||
ON l.item_id = r.item_id;
|
||||
|
||||
SELECT '-';
|
||||
SELECT *
|
||||
FROM (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date) l
|
||||
LEFT JOIN (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date ) r
|
||||
ON l.item_id = r.item_id;
|
||||
ON l.item_id = r.item_id
|
||||
ORDER BY ALL;
|
||||
|
||||
DROP TABLE t;
|
||||
|
@ -26,7 +26,7 @@ Expression ((Projection + Before ORDER BY))
|
||||
Parts: 1/1
|
||||
Granules: 1/1
|
||||
Expression ((Project names + Projection))
|
||||
Filter ((WHERE + DROP unused columns after JOIN))
|
||||
Filter (WHERE)
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression
|
||||
ReadFromMergeTree (default.t1)
|
||||
|
@ -75,7 +75,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverErro
|
||||
SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id;
|
||||
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id ORDER BY ALL;
|
||||
-- non-equi condition containing columns from different tables doesn't supported yet
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 }
|
||||
SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 }
|
||||
@ -89,10 +89,10 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 }
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 }
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 }
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id ORDER BY ALL;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2 ORDER BY ALL;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd ORDER BY ALL;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd ORDER BY ALL;
|
||||
|
||||
{% endfor -%}
|
||||
|
||||
|
@ -33,23 +33,23 @@
|
||||
2 2
|
||||
2 2
|
||||
-- { echoOn }
|
||||
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1;
|
||||
1 0
|
||||
2 2
|
||||
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
|
||||
2 2
|
||||
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1;
|
||||
0 3
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
|
||||
2 2
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1;
|
||||
1 0
|
||||
2 2
|
||||
0 3
|
||||
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1;
|
||||
1 0
|
||||
2 0
|
||||
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1;
|
||||
0 2
|
||||
0 3
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1;
|
||||
1 0
|
||||
2 0
|
||||
0 2
|
||||
@ -59,11 +59,11 @@ SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2
|
||||
1 ('',0)
|
||||
SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
|
||||
0 ('b',256)
|
||||
SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
|
||||
SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2;
|
||||
1 ('',0)
|
||||
0 ('b',256)
|
||||
SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
|
||||
SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
|
||||
SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2;
|
||||
1 ('',0)
|
||||
2
|
||||
4 2 Nullable(UInt64) UInt8
|
||||
|
@ -73,20 +73,20 @@ SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 0; --
|
||||
SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1;
|
||||
|
||||
-- { echoOn }
|
||||
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1;
|
||||
|
||||
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1;
|
||||
|
||||
SELECT * FROM (SELECT 1 as a) as t1 INNER JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
|
||||
SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
|
||||
SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
|
||||
SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
|
||||
SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2;
|
||||
SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
|
||||
SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL;
|
||||
SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2;
|
||||
|
||||
-- { echoOff }
|
||||
|
||||
|
@ -1,8 +1,8 @@
|
||||
DROP TABLE IF EXISTS t1;
|
||||
DROP TABLE IF EXISTS t2;
|
||||
|
||||
CREATE TABLE t1 (id Int) ENGINE = MergeTree ORDER BY id;
|
||||
CREATE TABLE t2 (id Int) ENGINE = MergeTree ORDER BY id;
|
||||
CREATE TABLE t1 (id Int) ENGINE = TinyLog;
|
||||
CREATE TABLE t2 (id Int) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO t1 VALUES (1), (2);
|
||||
INSERT INTO t2 SELECT number + 5 AS x FROM (SELECT * FROM system.numbers LIMIT 1111);
|
||||
|
@ -12,8 +12,9 @@ CREATE TABLE without_nullable
|
||||
insert into with_nullable values(0,'f'),(0,'usa');
|
||||
insert into without_nullable values(0,'usa'),(0,'us2a');
|
||||
|
||||
select if(t0.country is null ,t2.country,t0.country) "country"
|
||||
from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country;
|
||||
select if(t0.country is null ,t2.country,t0.country) "country"
|
||||
from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
drop table with_nullable;
|
||||
drop table without_nullable;
|
||||
|
@ -48,7 +48,8 @@ SELECT
|
||||
L2SquaredDistance(v1.v, v2.v),
|
||||
cosineDistance(v1.v, v2.v)
|
||||
FROM vec2 v1, vec2 v2
|
||||
WHERE length(v1.v) == length(v2.v);
|
||||
WHERE length(v1.v) == length(v2.v)
|
||||
ORDER BY ALL;
|
||||
|
||||
INSERT INTO vec2f VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1));
|
||||
SELECT
|
||||
@ -61,7 +62,8 @@ SELECT
|
||||
L2SquaredDistance(v1.v, v2.v),
|
||||
cosineDistance(v1.v, v2.v)
|
||||
FROM vec2f v1, vec2f v2
|
||||
WHERE length(v1.v) == length(v2.v);
|
||||
WHERE length(v1.v) == length(v2.v)
|
||||
ORDER BY ALL;
|
||||
|
||||
INSERT INTO vec2d VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1));
|
||||
SELECT
|
||||
@ -74,7 +76,8 @@ SELECT
|
||||
L2SquaredDistance(v1.v, v2.v),
|
||||
cosineDistance(v1.v, v2.v)
|
||||
FROM vec2d v1, vec2d v2
|
||||
WHERE length(v1.v) == length(v2.v);
|
||||
WHERE length(v1.v) == length(v2.v)
|
||||
ORDER BY ALL;
|
||||
|
||||
SELECT
|
||||
v1.id,
|
||||
@ -86,7 +89,8 @@ SELECT
|
||||
L2SquaredDistance(v1.v, v2.v),
|
||||
cosineDistance(v1.v, v2.v)
|
||||
FROM vec2f v1, vec2d v2
|
||||
WHERE length(v1.v) == length(v2.v);
|
||||
WHERE length(v1.v) == length(v2.v)
|
||||
ORDER BY ALL;
|
||||
|
||||
SELECT L1Distance([0, 0], [1]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
|
||||
SELECT L2Distance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
|
@ -148,7 +148,6 @@ Header: key String
|
||||
value String
|
||||
Join
|
||||
Header: __table1.key String
|
||||
__table3.key String
|
||||
__table3.value String
|
||||
Sorting
|
||||
Header: __table1.key String
|
||||
|
@ -9,4 +9,5 @@ CREATE TABLE t2__fuzz_47 (id LowCardinality(Int16)) ENGINE = MergeTree() ORDER B
|
||||
INSERT INTO t1__fuzz_13 VALUES (1);
|
||||
INSERT INTO t2__fuzz_47 VALUES (1);
|
||||
|
||||
SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2;
|
||||
SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2
|
||||
ORDER BY ALL;
|
||||
|
@ -117,7 +117,7 @@ ORDER BY t1.number, t2.number
|
||||
-- explain
|
||||
Expression (Project names)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN)))
|
||||
Expression ((Before ORDER BY + Projection))
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))))
|
||||
ReadFromSystemNumbers
|
||||
@ -161,7 +161,7 @@ ORDER BY t1.number, t2.number
|
||||
-- explain
|
||||
Expression (Project names)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN)))
|
||||
Expression ((Before ORDER BY + Projection))
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))))
|
||||
ReadFromSystemNumbers
|
||||
|
@ -79,7 +79,7 @@ Expression (Project names)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression (Before ORDER BY)
|
||||
Distinct (Preliminary DISTINCT)
|
||||
Expression ((Projection + DROP unused columns after JOIN))
|
||||
Expression (Projection)
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression ((Change column names to column identifiers + Project names))
|
||||
Distinct (DISTINCT)
|
||||
@ -244,7 +244,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression ((Before ORDER BY + Projection))
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression (Change column names to column identifiers)
|
||||
ReadFromSystemNumbers
|
||||
@ -280,7 +280,7 @@ Expression (Project names)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression ((Before ORDER BY + Projection))
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression (Change column names to column identifiers)
|
||||
ReadFromSystemNumbers
|
||||
@ -315,7 +315,7 @@ Expression (Project names)
|
||||
Expression ((Before ORDER BY + Projection))
|
||||
Rollup
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression (Change column names to column identifiers)
|
||||
ReadFromSystemNumbers
|
||||
@ -348,7 +348,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
|
||||
Expression ((Before ORDER BY + Projection))
|
||||
Rollup
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression (Change column names to column identifiers)
|
||||
ReadFromSystemNumbers
|
||||
@ -386,7 +386,7 @@ Expression (Project names)
|
||||
Expression ((Before ORDER BY + Projection))
|
||||
Cube
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression (Change column names to column identifiers)
|
||||
ReadFromSystemNumbers
|
||||
@ -419,7 +419,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
|
||||
Expression ((Before ORDER BY + Projection))
|
||||
Cube
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression (Change column names to column identifiers)
|
||||
ReadFromSystemNumbers
|
||||
@ -457,7 +457,7 @@ Expression (Project names)
|
||||
Expression ((Before ORDER BY + Projection))
|
||||
TotalsHaving
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression (Change column names to column identifiers)
|
||||
ReadFromSystemNumbers
|
||||
@ -491,7 +491,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
|
||||
Expression ((Before ORDER BY + Projection))
|
||||
TotalsHaving
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection))))
|
||||
Join (JOIN FillRightFirst)
|
||||
Expression (Change column names to column identifiers)
|
||||
ReadFromSystemNumbers
|
||||
|
@ -8,24 +8,21 @@ Header: count() UInt64
|
||||
Aggregating
|
||||
Header: __table1.a2 String
|
||||
count() UInt64
|
||||
Expression ((Before GROUP BY + DROP unused columns after JOIN))
|
||||
Expression (Before GROUP BY)
|
||||
Header: __table1.a2 String
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a2 String
|
||||
__table3.c1 UInt64
|
||||
Expression ((JOIN actions + DROP unused columns after JOIN))
|
||||
Expression (JOIN actions)
|
||||
Header: __table1.a2 String
|
||||
__table3.c1 UInt64
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a2 String
|
||||
__table2.b1 UInt64
|
||||
__table3.c1 UInt64
|
||||
Expression ((JOIN actions + DROP unused columns after JOIN))
|
||||
Expression (JOIN actions)
|
||||
Header: __table1.a2 String
|
||||
__table2.b1 UInt64
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a1 UInt64
|
||||
__table1.a2 String
|
||||
Header: __table1.a2 String
|
||||
__table2.b1 UInt64
|
||||
Expression ((JOIN actions + Change column names to column identifiers))
|
||||
Header: __table1.a1 UInt64
|
||||
@ -48,39 +45,32 @@ Header: count() UInt64
|
||||
EXPLAIN PLAN header = 1
|
||||
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
|
||||
d2 String
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a2 String
|
||||
__table1.k UInt64
|
||||
__table4.d2 String
|
||||
Expression (DROP unused columns after JOIN)
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a2 String
|
||||
__table1.k UInt64
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a2 String
|
||||
__table1.k UInt64
|
||||
Expression (DROP unused columns after JOIN)
|
||||
Expression (Change column names to column identifiers)
|
||||
Header: __table1.a2 String
|
||||
__table1.k UInt64
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a2 String
|
||||
__table1.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
|
||||
ReadFromMemoryStorage
|
||||
Header: a2 String
|
||||
k UInt64
|
||||
Expression (Change column names to column identifiers)
|
||||
Header: __table3.k UInt64
|
||||
Header: __table2.k UInt64
|
||||
ReadFromMemoryStorage
|
||||
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)
|
||||
Header: __table4.d2 String
|
||||
__table4.k UInt64
|
||||
@ -106,27 +96,24 @@ Header: bx String
|
||||
Header: __table1.a2 String
|
||||
__table2.bx String
|
||||
__table4.c2 String
|
||||
__table4.c1 UInt64
|
||||
Expression
|
||||
Header: __table1.a2 String
|
||||
__table2.bx String
|
||||
__table4.c2 String
|
||||
__table4.c1 UInt64
|
||||
__table4.c2 String
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a2 String
|
||||
__table2.bx String
|
||||
__table2.b1 UInt64
|
||||
__table4.c2 String
|
||||
__table4.c1 UInt64
|
||||
Expression ((JOIN actions + DROP unused columns after JOIN))
|
||||
__table4.c2 String
|
||||
Expression (JOIN actions)
|
||||
Header: __table1.a2 String
|
||||
__table2.bx String
|
||||
__table2.b1 UInt64
|
||||
__table2.bx String
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a1 UInt64
|
||||
__table1.a2 String
|
||||
__table2.bx String
|
||||
Header: __table1.a2 String
|
||||
__table2.b1 UInt64
|
||||
__table2.bx String
|
||||
Expression ((JOIN actions + Change column names to column identifiers))
|
||||
Header: __table1.a1 UInt64
|
||||
__table1.a2 String
|
||||
|
@ -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');
|
||||
|
||||
SET enable_analyzer = 1;
|
||||
SET query_plan_join_inner_table_selection = 'right';
|
||||
|
||||
-- { echoOn }
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
1
|
||||
1
|
||||
|
||||
1
|
||||
0
|
||||
\N
|
||||
|
||||
100000000000000000000
|
||||
|
@ -1,22 +1,22 @@
|
||||
Expression ((Project names + (Projection + DROP unused columns after JOIN)))
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
value_1 String
|
||||
rhs.id UInt64
|
||||
rhs.value_1 String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value_1 String : 1
|
||||
INPUT : 2 -> __table2.value_1 String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value_1 String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value_1 :: 1 -> value_1 String : 0
|
||||
ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 0 2 1
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2
|
||||
Positions: 4 0 1 2
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value_1 String
|
||||
__table2.value_1 String
|
||||
__table2.id UInt64
|
||||
__table2.value_1 String
|
||||
Type: INNER
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -50,29 +50,25 @@ Positions: 4 0 2 1
|
||||
Parts: 1
|
||||
Granules: 1
|
||||
--
|
||||
Expression ((Project names + (Projection + DROP unused columns after JOIN)))
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
value_1 String
|
||||
rhs.id UInt64
|
||||
rhs.value_1 String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
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 :: 4 -> __table2.value_2 UInt64 : 4
|
||||
INPUT : 5 -> __table2.id UInt64 : 5
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 6
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value_1 :: 1 -> value_1 String : 0
|
||||
ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1
|
||||
ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3
|
||||
Positions: 6 0 3 1
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2
|
||||
Positions: 4 0 1 2
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value_1 String
|
||||
__table1.value_2 UInt64
|
||||
__table2.value_1 String
|
||||
__table2.value_2 UInt64
|
||||
__table2.id UInt64
|
||||
__table2.value_1 String
|
||||
Type: INNER
|
||||
Strictness: ASOF
|
||||
Algorithm: HashJoin
|
||||
|
@ -19,6 +19,8 @@ CREATE TABLE test_table_2
|
||||
INSERT INTO test_table_1 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
|
||||
FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id;
|
||||
|
||||
|
@ -31,6 +31,7 @@
|
||||
8
|
||||
9
|
||||
\N
|
||||
--- analyzer ---
|
||||
0
|
||||
1
|
||||
2
|
||||
|
@ -21,6 +21,8 @@ SETTINGS join_algorithm = 'partial_merge';
|
||||
SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL
|
||||
SETTINGS join_algorithm = 'full_sorting_merge';
|
||||
|
||||
SELECT '--- analyzer ---';
|
||||
|
||||
SET enable_analyzer = 1;
|
||||
|
||||
SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL
|
||||
|
@ -2,7 +2,9 @@
|
||||
|
||||
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
|
||||
WHERE lhs.id = 5;
|
||||
WHERE lhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
Expression ((Project names + (Projection + )))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -10,18 +12,18 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 2 0 1
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 1 0 2
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: INNER
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -69,7 +71,9 @@ SELECT '--';
|
||||
--
|
||||
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
|
||||
WHERE rhs.id = 5;
|
||||
WHERE rhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right';
|
||||
;
|
||||
Expression ((Project names + (Projection + )))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -77,18 +81,18 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 2 0 1
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 1 0 2
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: INNER
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -136,7 +140,9 @@ SELECT '--';
|
||||
--
|
||||
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
|
||||
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 + )))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -144,18 +150,18 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 2 0 1
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 1 0 2
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: INNER
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -206,7 +212,9 @@ SELECT '--';
|
||||
--
|
||||
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
|
||||
WHERE lhs.id = 5;
|
||||
WHERE lhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
Expression ((Project names + (Projection + )))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -214,18 +222,18 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 2 0 1
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 1 0 2
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: LEFT
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -273,7 +281,9 @@ SELECT '--';
|
||||
--
|
||||
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
|
||||
WHERE rhs.id = 5;
|
||||
WHERE rhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -281,31 +291,31 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 2 0 1
|
||||
Filter ((WHERE + DROP unused columns after JOIN))
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 1 0 2
|
||||
Filter (WHERE)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Filter column: equals(__table2.id, 5_UInt8) (removed)
|
||||
Actions: INPUT :: 0 -> __table1.id UInt64 : 0
|
||||
INPUT :: 1 -> __table1.value String : 1
|
||||
INPUT :: 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT :: 3 -> __table2.value String : 3
|
||||
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
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: LEFT
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -347,7 +357,9 @@ SELECT '--';
|
||||
--
|
||||
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
|
||||
WHERE lhs.id = 5;
|
||||
WHERE lhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -355,31 +367,31 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 2 0 1
|
||||
Filter ((WHERE + DROP unused columns after JOIN))
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 1 0 2
|
||||
Filter (WHERE)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Filter column: equals(__table1.id, 5_UInt8) (removed)
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT :: 1 -> __table1.value String : 1
|
||||
INPUT :: 2 -> __table2.value String : 2
|
||||
INPUT :: 3 -> __table2.id UInt64 : 3
|
||||
INPUT :: 2 -> __table2.id UInt64 : 2
|
||||
INPUT :: 3 -> __table2.value String : 3
|
||||
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
|
||||
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5
|
||||
Positions: 5 0 1 2 3
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: RIGHT
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -421,7 +433,9 @@ SELECT '--';
|
||||
--
|
||||
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
|
||||
WHERE rhs.id = 5;
|
||||
WHERE rhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
Expression ((Project names + (Projection + )))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -429,18 +443,18 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 2 0 1
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 1 0 2
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: RIGHT
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -488,7 +502,9 @@ SELECT '--';
|
||||
--
|
||||
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
|
||||
WHERE lhs.id = 5;
|
||||
WHERE lhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -496,31 +512,31 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 2 0 1
|
||||
Filter ((WHERE + DROP unused columns after JOIN))
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 1 0 2
|
||||
Filter (WHERE)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Filter column: equals(__table1.id, 5_UInt8) (removed)
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT :: 1 -> __table1.value String : 1
|
||||
INPUT :: 2 -> __table2.value String : 2
|
||||
INPUT :: 3 -> __table2.id UInt64 : 3
|
||||
INPUT :: 2 -> __table2.id UInt64 : 2
|
||||
INPUT :: 3 -> __table2.value String : 3
|
||||
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
|
||||
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5
|
||||
Positions: 5 0 1 2 3
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: FULL
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -562,7 +578,9 @@ SELECT '--';
|
||||
--
|
||||
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
|
||||
WHERE rhs.id = 5;
|
||||
WHERE rhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -570,31 +588,31 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 2 0 1
|
||||
Filter ((WHERE + DROP unused columns after JOIN))
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 1 0 2
|
||||
Filter (WHERE)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Filter column: equals(__table2.id, 5_UInt8) (removed)
|
||||
Actions: INPUT :: 0 -> __table1.id UInt64 : 0
|
||||
INPUT :: 1 -> __table1.value String : 1
|
||||
INPUT :: 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT :: 3 -> __table2.value String : 3
|
||||
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
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: FULL
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -636,7 +654,9 @@ SELECT '--';
|
||||
--
|
||||
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
|
||||
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))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -644,34 +664,34 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 2 0 1
|
||||
Filter ((WHERE + DROP unused columns after JOIN))
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 1 0 2
|
||||
Filter (WHERE)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed)
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT :: 1 -> __table1.value String : 1
|
||||
INPUT :: 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT :: 3 -> __table2.value String : 3
|
||||
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
|
||||
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 5
|
||||
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
|
||||
Positions: 5 0 1 2 3
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: FULL
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
|
@ -22,7 +22,9 @@ INSERT INTO test_table_2 SELECT number, number FROM numbers(10);
|
||||
|
||||
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
|
||||
WHERE lhs.id = 5;
|
||||
WHERE lhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
@ -33,7 +35,9 @@ SELECT '--';
|
||||
|
||||
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
|
||||
WHERE rhs.id = 5;
|
||||
WHERE rhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right';
|
||||
;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
@ -44,7 +48,9 @@ SELECT '--';
|
||||
|
||||
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
|
||||
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
|
||||
WHERE lhs.id = 5 AND rhs.id = 6;
|
||||
@ -53,7 +59,9 @@ SELECT '--';
|
||||
|
||||
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
|
||||
WHERE lhs.id = 5;
|
||||
WHERE lhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
@ -64,7 +72,9 @@ SELECT '--';
|
||||
|
||||
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
|
||||
WHERE rhs.id = 5;
|
||||
WHERE rhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
@ -75,7 +85,9 @@ SELECT '--';
|
||||
|
||||
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
|
||||
WHERE lhs.id = 5;
|
||||
WHERE lhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
@ -86,7 +98,9 @@ SELECT '--';
|
||||
|
||||
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
|
||||
WHERE rhs.id = 5;
|
||||
WHERE rhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
@ -97,7 +111,9 @@ SELECT '--';
|
||||
|
||||
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
|
||||
WHERE lhs.id = 5;
|
||||
WHERE lhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
@ -108,7 +124,9 @@ SELECT '--';
|
||||
|
||||
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
|
||||
WHERE rhs.id = 5;
|
||||
WHERE rhs.id = 5
|
||||
SETTINGS query_plan_join_inner_table_selection = 'right'
|
||||
;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
@ -119,7 +137,9 @@ SELECT '--';
|
||||
|
||||
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
|
||||
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 '--';
|
||||
|
||||
|
@ -52,7 +52,9 @@ WITH RECURSIVE search_graph AS (
|
||||
FROM graph g, search_graph sg
|
||||
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 3 arc 1 -> 3 false [(1,3)]
|
||||
2 3 arc 2 -> 3 false [(2,3)]
|
||||
|
@ -55,7 +55,9 @@ WITH RECURSIVE search_graph AS (
|
||||
FROM graph g, search_graph sg
|
||||
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
|
||||
WITH RECURSIVE search_graph AS (
|
||||
|
File diff suppressed because one or more lines are too long
@ -5,18 +5,18 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 0 2 1
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 0 1 2
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: INNER
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -75,18 +75,18 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 0 2 1
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 0 1 2
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: INNER
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
@ -145,18 +145,18 @@ Header: id UInt64
|
||||
rhs.value String
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
INPUT : 1 -> __table1.value String : 1
|
||||
INPUT : 2 -> __table2.value String : 2
|
||||
INPUT : 3 -> __table2.id UInt64 : 3
|
||||
INPUT : 2 -> __table2.id UInt64 : 2
|
||||
INPUT : 3 -> __table2.value String : 3
|
||||
ALIAS __table1.id :: 0 -> id UInt64 : 4
|
||||
ALIAS __table1.value :: 1 -> value String : 0
|
||||
ALIAS __table2.value :: 2 -> rhs.value String : 1
|
||||
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
|
||||
Positions: 4 0 2 1
|
||||
ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1
|
||||
ALIAS __table2.value :: 3 -> rhs.value String : 2
|
||||
Positions: 4 0 1 2
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.id UInt64
|
||||
__table1.value String
|
||||
__table2.value String
|
||||
__table2.id UInt64
|
||||
__table2.value String
|
||||
Type: INNER
|
||||
Strictness: ALL
|
||||
Algorithm: HashJoin
|
||||
|
@ -22,7 +22,10 @@ SETTINGS index_granularity = 16
|
||||
INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2');
|
||||
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 '--';
|
||||
|
||||
@ -30,7 +33,9 @@ SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.
|
||||
|
||||
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 '--';
|
||||
|
||||
@ -38,7 +43,9 @@ SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs
|
||||
|
||||
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 '--';
|
||||
|
||||
|
@ -65,8 +65,7 @@ SELECT name FROM users RIGHT JOIN users2 USING name WHERE users2.name ='Alice';
|
||||
Expression ((Project names + (Projection + )))
|
||||
Header: name String
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.name String
|
||||
__table2.name String
|
||||
Header: __table2.name String
|
||||
Filter (( + Change column names to column identifiers))
|
||||
Header: __table1.name String
|
||||
ReadFromMergeTree (default.users)
|
||||
|
@ -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;
|
||||
|
||||
SET max_memory_usage = '1G';
|
||||
SET query_plan_join_inner_table_selection = 'right';
|
||||
|
||||
CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS
|
||||
SELECT id_values.id1 AS id,
|
||||
|
@ -0,0 +1,4 @@
|
||||
20000
|
||||
20000
|
||||
1
|
||||
0
|
@ -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;
|
Loading…
Reference in New Issue
Block a user