mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-04 13:32:13 +00:00
Merge branch 'master' of github.com:ClickHouse/ClickHouse into measure_logger_time
This commit is contained in:
commit
a44e09a0c4
@ -1924,6 +1924,13 @@ See also:
|
||||
For single JOIN in case of identifier ambiguity prefer left table
|
||||
)", IMPORTANT) \
|
||||
\
|
||||
DECLARE(BoolAuto, query_plan_join_swap_table, Field("auto"), R"(
|
||||
Determine which side of the join should be the build table (also called inner, the one inserted into the hash table for a hash join) in the query plan. This setting is supported only for `ALL` join strictness with the `JOIN ON` clause. Possible values are:
|
||||
- 'auto': Let the planner decide which table to use as the build table.
|
||||
- 'false': Never swap tables (the right table is the build table).
|
||||
- 'true': Always swap tables (the left table is the build table).
|
||||
)", 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) \
|
||||
|
@ -45,6 +45,7 @@ class WriteBuffer;
|
||||
#define COMMON_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, ArrowCompression) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, BoolAuto) \
|
||||
M(CLASS_NAME, CapnProtoEnumComparingMode) \
|
||||
M(CLASS_NAME, Char) \
|
||||
M(CLASS_NAME, DateTimeInputFormat) \
|
||||
|
@ -60,6 +60,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
{
|
||||
{"24.12",
|
||||
{
|
||||
{"query_plan_join_swap_table", "false", "auto", "New setting. Right table was always chosen before."},
|
||||
{"max_size_to_preallocate_for_aggregation", 100'000'000, 1'000'000'000'000, "Enable optimisation for bigger tables."},
|
||||
{"max_size_to_preallocate_for_joins", 100'000'000, 1'000'000'000'000, "Enable optimisation for bigger tables."},
|
||||
{"parallel_replicas_index_analysis_only_on_coordinator", false, true, "Index analysis done only on replica-coordinator and skipped on other replicas. Effective only with enabled parallel_replicas_local_plan"},
|
||||
|
@ -128,10 +128,8 @@ constexpr auto getEnumValues();
|
||||
DECLARE_SETTING_ENUM(LoadBalancing)
|
||||
|
||||
DECLARE_SETTING_ENUM(JoinStrictness)
|
||||
|
||||
DECLARE_SETTING_MULTI_ENUM(JoinAlgorithm)
|
||||
|
||||
|
||||
/// Which rows should be included in TOTALS.
|
||||
enum class TotalsMode : uint8_t
|
||||
{
|
||||
|
@ -2,13 +2,13 @@
|
||||
|
||||
#include <chrono>
|
||||
#include <string_view>
|
||||
#include <optional>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/MultiEnum.h>
|
||||
#include <base/types.h>
|
||||
#include <Poco/Timespan.h>
|
||||
#include <Poco/URI.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
@ -125,8 +125,10 @@ struct SettingAutoWrapper
|
||||
void readBinary(ReadBuffer & in) { changed = true; is_auto = false; base.readBinary(in); }
|
||||
|
||||
Type valueOr(Type default_value) const { return is_auto ? default_value : base.value; }
|
||||
std::optional<Type> get() const { return is_auto ? std::nullopt : std::make_optional(base.value); }
|
||||
};
|
||||
|
||||
using SettingFieldBoolAuto = SettingAutoWrapper<SettingFieldBool>;
|
||||
using SettingFieldUInt64Auto = SettingAutoWrapper<SettingFieldUInt64>;
|
||||
using SettingFieldInt64Auto = SettingAutoWrapper<SettingFieldInt64>;
|
||||
using SettingFieldFloatAuto = SettingAutoWrapper<SettingFieldFloat>;
|
||||
|
@ -63,6 +63,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_,
|
||||
|
@ -431,6 +431,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() && strictness != JoinStrictness::RightAny;
|
||||
}
|
||||
|
||||
void HashJoin::initRightBlockStructure(Block & saved_block_sample)
|
||||
{
|
||||
if (isCrossOrComma(kind))
|
||||
@ -442,8 +452,10 @@ 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)
|
||||
|| isRightOrFull(kind) || multiple_disjuncts || table_join->getMixedJoinExpression();
|
||||
bool save_key_columns = isUsedByAnotherAlgorithm() ||
|
||||
isRightOrFull(kind) ||
|
||||
multiple_disjuncts ||
|
||||
table_join->getMixedJoinExpression();
|
||||
if (save_key_columns)
|
||||
{
|
||||
saved_block_sample = right_table_keys.cloneEmpty();
|
||||
@ -1356,7 +1368,10 @@ HashJoin::getNonJoinedBlocks(const Block & left_sample_block, const Block & resu
|
||||
{
|
||||
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)
|
||||
@ -1365,14 +1380,9 @@ HashJoin::getNonJoinedBlocks(const Block & left_sample_block, const Block & resu
|
||||
size_t expected_columns_count = left_columns_count + required_right_keys.columns() + sample_block_with_columns_to_add.columns();
|
||||
if (expected_columns_count != result_sample_block.columns())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Unexpected number of columns in result sample block: {} instead of {} ({} + {} + {})",
|
||||
result_sample_block.columns(),
|
||||
expected_columns_count,
|
||||
left_columns_count,
|
||||
required_right_keys.columns(),
|
||||
sample_block_with_columns_to_add.columns());
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected number of columns in result sample block: {} expected {} ([{}] + [{}] + [{}])",
|
||||
result_sample_block.columns(), expected_columns_count,
|
||||
left_sample_block.dumpNames(), required_right_keys.dumpNames(), sample_block_with_columns_to_add.dumpNames());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -126,7 +126,7 @@ public:
|
||||
|
||||
bool isCloneSupported() const override
|
||||
{
|
||||
return true;
|
||||
return !getTotals() && getTotalRowCount() == 0;
|
||||
}
|
||||
|
||||
std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_,
|
||||
@ -484,6 +484,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;
|
||||
|
||||
|
@ -80,6 +80,7 @@ ScatteredBlock 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]);
|
||||
}
|
||||
|
||||
auto & source_block = block.getSourceBlock();
|
||||
size_t existing_columns = source_block.columns();
|
||||
|
||||
@ -121,6 +122,20 @@ ScatteredBlock HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
|
||||
|
||||
block.filterBySelector();
|
||||
|
||||
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 < source_block.columns(); ++i)
|
||||
{
|
||||
if (!left_output_columns.contains(source_block.getByPosition(i).name))
|
||||
block_columns_to_erase.insert(i);
|
||||
}
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < added_columns.size(); ++i)
|
||||
source_block.insert(added_columns.moveColumn(i));
|
||||
|
||||
@ -178,6 +193,7 @@ ScatteredBlock HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
|
||||
block.getSourceBlock().setColumns(columns);
|
||||
block = ScatteredBlock(std::move(block).getSourceBlock());
|
||||
}
|
||||
block.getSourceBlock().erase(block_columns_to_erase);
|
||||
return remaining_block;
|
||||
}
|
||||
|
||||
|
@ -1881,7 +1881,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
settings[Setting::max_block_size],
|
||||
0,
|
||||
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;
|
||||
|
@ -870,10 +870,7 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
else if (command.type == MutationCommand::MATERIALIZE_TTL)
|
||||
{
|
||||
mutation_kind.set(MutationKind::MUTATE_OTHER);
|
||||
bool suitable_for_ttl_optimization = (*source.getMergeTreeData()->getSettings())[MergeTreeSetting::ttl_only_drop_parts]
|
||||
&& metadata_snapshot->hasOnlyRowsTTL();
|
||||
|
||||
if (materialize_ttl_recalculate_only || suitable_for_ttl_optimization)
|
||||
if (materialize_ttl_recalculate_only)
|
||||
{
|
||||
// just recalculate ttl_infos without remove expired data
|
||||
auto all_columns_vec = all_columns.getNames();
|
||||
|
@ -42,6 +42,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;
|
||||
@ -144,6 +145,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])
|
||||
{
|
||||
}
|
||||
|
||||
@ -162,6 +164,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();
|
||||
@ -204,6 +208,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;
|
||||
@ -352,9 +370,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
|
||||
@ -997,5 +1024,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");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -173,6 +173,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.
|
||||
@ -208,6 +211,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
|
||||
@ -271,6 +276,8 @@ public:
|
||||
|
||||
VolumePtr getGlobalTemporaryVolume() { return tmp_volume; }
|
||||
|
||||
bool enableEnalyzer() const { return enable_analyzer; }
|
||||
void assertEnableEnalyzer() const;
|
||||
TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data ? tmp_data->childScope(CurrentMetrics::TemporaryFilesForJoin) : nullptr; }
|
||||
|
||||
ActionsDAG createJoinedBlockActions(ContextPtr context) const;
|
||||
@ -290,6 +297,7 @@ public:
|
||||
}
|
||||
|
||||
bool allowParallelHashJoin() const;
|
||||
void swapSides();
|
||||
|
||||
bool joinUseNulls() const { return join_use_nulls; }
|
||||
|
||||
@ -380,6 +388,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;
|
||||
@ -405,11 +416,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 "248372b7-02c4-4c88-a5e1-282a83cc572a" 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 SettingsBoolAuto query_plan_join_swap_table;
|
||||
extern const SettingsUInt64 min_joined_block_size_bytes;
|
||||
}
|
||||
|
||||
@ -1270,6 +1271,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,
|
||||
@ -1544,24 +1594,48 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
const Block & right_header = right_plan.getCurrentHeader();
|
||||
auto join_algorithm = chooseJoinAlgorithm(
|
||||
table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context, select_query_info);
|
||||
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())
|
||||
{
|
||||
/// We should add all duplicated columns, because join algorithm add either all column with specified name or none
|
||||
auto set_used_column_with_duplicates = [&](const NamesAndTypesList & columns, JoinTableSide join_table_side)
|
||||
{
|
||||
const auto & column_name = columns.front().name;
|
||||
for (const auto & column : columns)
|
||||
if (column.name == column_name)
|
||||
table_join->setUsedColumn(column, join_table_side);
|
||||
};
|
||||
|
||||
if (!columns_from_left_table.empty())
|
||||
set_used_column_with_duplicates(columns_from_left_table, JoinTableSide::Left);
|
||||
else if (!columns_from_right_table.empty())
|
||||
set_used_column_with_duplicates(columns_from_right_table, JoinTableSide::Right);
|
||||
}
|
||||
|
||||
auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context, select_query_info);
|
||||
auto result_plan = QueryPlan();
|
||||
|
||||
bool is_filled_join = join_algorithm->isFilled();
|
||||
@ -1647,6 +1721,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(
|
||||
}
|
||||
|
||||
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(),
|
||||
@ -1654,7 +1738,11 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(
|
||||
settings[Setting::max_block_size],
|
||||
settings[Setting::min_joined_block_size_bytes],
|
||||
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->swap_join_tables = settings[Setting::query_plan_join_swap_table].get();
|
||||
|
||||
join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type));
|
||||
|
||||
@ -1665,47 +1753,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(
|
||||
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);
|
||||
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Processors/Transforms/ColumnPermuteTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -37,6 +38,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(
|
||||
@ -46,12 +78,16 @@ JoinStep::JoinStep(
|
||||
size_t max_block_size_,
|
||||
size_t min_block_size_bytes_,
|
||||
size_t max_streams_,
|
||||
bool 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_)
|
||||
, min_block_size_bytes(min_block_size_bytes_)
|
||||
, 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_});
|
||||
}
|
||||
@ -61,32 +97,52 @@ 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]);
|
||||
|
||||
std::unique_ptr<QueryPipelineBuilder> joined_pipeline;
|
||||
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);
|
||||
joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped(
|
||||
std::move(pipelines[0]), std::move(pipelines[1]), join, join_algorithm_header, max_block_size, &processors);
|
||||
joined_pipeline->resize(max_streams);
|
||||
return joined_pipeline;
|
||||
}
|
||||
else
|
||||
{
|
||||
joined_pipeline = QueryPipelineBuilder::joinPipelinesRightLeft(
|
||||
std::move(pipelines[0]),
|
||||
std::move(pipelines[1]),
|
||||
join,
|
||||
join_algorithm_header,
|
||||
max_block_size,
|
||||
min_block_size_bytes,
|
||||
max_streams,
|
||||
keep_left_read_in_order,
|
||||
&processors);
|
||||
}
|
||||
|
||||
auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft(
|
||||
std::move(pipelines[0]),
|
||||
std::move(pipelines[1]),
|
||||
join,
|
||||
*output_header,
|
||||
max_block_size,
|
||||
min_block_size_bytes,
|
||||
max_streams,
|
||||
keep_left_read_in_order,
|
||||
&processors);
|
||||
if (!use_new_analyzer)
|
||||
return joined_pipeline;
|
||||
|
||||
auto column_permutation = getPermutationForBlock(joined_pipeline->getHeader(), lhs_header, rhs_header, required_output);
|
||||
if (!column_permutation.empty())
|
||||
{
|
||||
joined_pipeline->addSimpleTransform([&column_permutation](const Block & header)
|
||||
{
|
||||
return std::make_shared<ColumnPermuteTransform>(header, column_permutation);
|
||||
});
|
||||
}
|
||||
|
||||
if (join->supportParallelJoin())
|
||||
{
|
||||
pipeline->addSimpleTransform([&](const Block & header)
|
||||
joined_pipeline->addSimpleTransform([&](const Block & header)
|
||||
{ return std::make_shared<SimpleSquashingChunksTransform>(header, 0, min_block_size_bytes); });
|
||||
}
|
||||
|
||||
return pipeline;
|
||||
return joined_pipeline;
|
||||
}
|
||||
|
||||
bool JoinStep::allowPushDownToRight() const
|
||||
@ -105,17 +161,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
|
||||
{
|
||||
@ -20,7 +21,9 @@ public:
|
||||
size_t max_block_size_,
|
||||
size_t min_block_size_bytes_,
|
||||
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"; }
|
||||
|
||||
@ -32,17 +35,28 @@ 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;
|
||||
|
||||
/// Swap automatically if not set, otherwise always or never, depending on the value
|
||||
std::optional<bool> swap_join_tables = false;
|
||||
|
||||
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 min_block_size_bytes;
|
||||
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.
|
||||
|
103
src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp
Normal file
103
src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp
Normal file
@ -0,0 +1,103 @@
|
||||
#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 all JOIN kinds, so changing from LEFT to RIGHT is not always possible
|
||||
bool allow_outer_join = typeid_cast<const HashJoin *>(join.get());
|
||||
if (table_join.kind() != JoinKind::Inner && !allow_outer_join)
|
||||
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->swap_join_tables.has_value())
|
||||
{
|
||||
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->swap_join_tables.value())
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
@ -101,6 +101,7 @@ IMergingAlgorithm::Status PasteJoinAlgorithm::merge()
|
||||
return Status(0);
|
||||
if (last_used_row[1] >= chunks[1].getNumRows())
|
||||
return Status(1);
|
||||
|
||||
/// We have unused rows from both inputs
|
||||
size_t result_num_rows = std::min(chunks[0].getNumRows() - last_used_row[0], chunks[1].getNumRows() - last_used_row[1]);
|
||||
|
||||
@ -110,6 +111,7 @@ IMergingAlgorithm::Status PasteJoinAlgorithm::merge()
|
||||
result.addColumn(col->cut(last_used_row[source_num], result_num_rows));
|
||||
last_used_row[0] += result_num_rows;
|
||||
last_used_row[1] += result_num_rows;
|
||||
|
||||
return Status(std::move(result));
|
||||
}
|
||||
|
||||
|
@ -2405,8 +2405,6 @@ void MergeTreeData::prewarmCaches(ThreadPool & pool, MarkCachePtr mark_cache, Pr
|
||||
return to_tuple(lhs) > to_tuple(rhs);
|
||||
});
|
||||
|
||||
ThreadPoolCallbackRunnerLocal<void> runner(pool, "PrewarmCaches");
|
||||
|
||||
double marks_ratio_to_prewarm = getContext()->getServerSettings()[ServerSetting::mark_cache_prewarm_ratio];
|
||||
double index_ratio_to_prewarm = getContext()->getServerSettings()[ServerSetting::primary_index_cache_prewarm_ratio];
|
||||
|
||||
@ -2418,6 +2416,10 @@ void MergeTreeData::prewarmCaches(ThreadPool & pool, MarkCachePtr mark_cache, Pr
|
||||
columns_to_prewarm_marks = getColumnsToPrewarmMarks(*getSettings(), metadata_snaphost->getColumns().getAllPhysical());
|
||||
}
|
||||
|
||||
/// Allocate runner on stack after all used local variables to make its destructor
|
||||
/// is called first and all tasks stopped before local variables are being destroyed.
|
||||
ThreadPoolCallbackRunnerLocal<void> runner(pool, "PrewarmCaches");
|
||||
|
||||
for (const auto & part : data_parts)
|
||||
{
|
||||
bool added_task = false;
|
||||
|
@ -132,7 +132,6 @@ static void splitAndModifyMutationCommands(
|
||||
const MutationCommands & commands,
|
||||
MutationCommands & for_interpreter,
|
||||
MutationCommands & for_file_renames,
|
||||
bool suitable_for_ttl_optimization,
|
||||
LoggerPtr log)
|
||||
{
|
||||
auto part_columns = part->getColumnsDescription();
|
||||
@ -142,7 +141,6 @@ static void splitAndModifyMutationCommands(
|
||||
{
|
||||
NameSet mutated_columns;
|
||||
NameSet dropped_columns;
|
||||
NameSet ignored_columns;
|
||||
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
@ -168,15 +166,6 @@ static void splitAndModifyMutationCommands(
|
||||
for_interpreter.push_back(command);
|
||||
for (const auto & [column_name, expr] : command.column_to_update_expression)
|
||||
mutated_columns.emplace(column_name);
|
||||
|
||||
if (command.type == MutationCommand::Type::MATERIALIZE_TTL && suitable_for_ttl_optimization)
|
||||
{
|
||||
for (const auto & col : part_columns)
|
||||
{
|
||||
if (!mutated_columns.contains(col.name))
|
||||
ignored_columns.emplace(col.name);
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (command.type == MutationCommand::Type::DROP_INDEX
|
||||
|| command.type == MutationCommand::Type::DROP_PROJECTION
|
||||
@ -237,7 +226,7 @@ static void splitAndModifyMutationCommands(
|
||||
/// from disk we just don't read dropped columns
|
||||
for (const auto & column : part_columns)
|
||||
{
|
||||
if (!mutated_columns.contains(column.name) && !ignored_columns.contains(column.name))
|
||||
if (!mutated_columns.contains(column.name))
|
||||
{
|
||||
if (!metadata_snapshot->getColumns().has(column.name) && !part->storage.getVirtualsPtr()->has(column.name))
|
||||
{
|
||||
@ -1951,88 +1940,6 @@ private:
|
||||
std::unique_ptr<PartMergerWriter> part_merger_writer_task{nullptr};
|
||||
};
|
||||
|
||||
/*
|
||||
* Decorator that'll drop expired parts by replacing them with empty ones.
|
||||
* Main use case (only use case for now) is to decorate `MutateSomePartColumnsTask`,
|
||||
* which is used to recalculate TTL. If the part is expired, this class will replace it with
|
||||
* an empty one.
|
||||
*
|
||||
* Triggered when `ttl_only_drop_parts` is set and the only TTL is rows TTL.
|
||||
* */
|
||||
class ExecutableTaskDropTTLExpiredPartsDecorator : public IExecutableTask
|
||||
{
|
||||
public:
|
||||
explicit ExecutableTaskDropTTLExpiredPartsDecorator(
|
||||
std::unique_ptr<IExecutableTask> executable_task_,
|
||||
MutationContextPtr ctx_
|
||||
)
|
||||
: executable_task(std::move(executable_task_)), ctx(ctx_) {}
|
||||
|
||||
void onCompleted() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); }
|
||||
StorageID getStorageID() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); }
|
||||
Priority getPriority() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); }
|
||||
String getQueryId() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); }
|
||||
|
||||
bool executeStep() override
|
||||
{
|
||||
switch (state)
|
||||
{
|
||||
case State::NEED_EXECUTE:
|
||||
{
|
||||
if (executable_task->executeStep())
|
||||
return true;
|
||||
|
||||
if (isRowsMaxTTLExpired())
|
||||
replacePartWithEmpty();
|
||||
|
||||
state = State::SUCCESS;
|
||||
return true;
|
||||
}
|
||||
case State::SUCCESS:
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void cancel() noexcept override
|
||||
{
|
||||
executable_task->cancel();
|
||||
}
|
||||
|
||||
private:
|
||||
enum class State
|
||||
{
|
||||
NEED_EXECUTE,
|
||||
|
||||
SUCCESS
|
||||
};
|
||||
|
||||
State state{State::NEED_EXECUTE};
|
||||
|
||||
std::unique_ptr<IExecutableTask> executable_task;
|
||||
MutationContextPtr ctx;
|
||||
|
||||
bool isRowsMaxTTLExpired() const
|
||||
{
|
||||
const auto ttl = ctx->new_data_part->ttl_infos.table_ttl;
|
||||
return ttl.max && ttl.max <= ctx->time_of_mutation;
|
||||
}
|
||||
|
||||
void replacePartWithEmpty()
|
||||
{
|
||||
MergeTreePartInfo part_info = ctx->new_data_part->info;
|
||||
part_info.level += 1;
|
||||
|
||||
MergeTreePartition partition = ctx->new_data_part->partition;
|
||||
std::string part_name = ctx->new_data_part->getNewName(part_info);
|
||||
|
||||
auto [mutable_empty_part, _] = ctx->data->createEmptyPart(part_info, partition, part_name, ctx->txn);
|
||||
ctx->new_data_part = std::move(mutable_empty_part);
|
||||
}
|
||||
};
|
||||
|
||||
MutateTask::MutateTask(
|
||||
FutureMergedMutatedPartPtr future_part_,
|
||||
StorageMetadataPtr metadata_snapshot_,
|
||||
@ -2276,7 +2183,6 @@ bool MutateTask::prepare()
|
||||
context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0));
|
||||
context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1);
|
||||
|
||||
bool suitable_for_ttl_optimization = ctx->metadata_snapshot->hasOnlyRowsTTL() && (*ctx->data->getSettings())[MergeTreeSetting::ttl_only_drop_parts];
|
||||
MutationHelpers::splitAndModifyMutationCommands(
|
||||
ctx->source_part,
|
||||
ctx->metadata_snapshot,
|
||||
@ -2284,7 +2190,6 @@ bool MutateTask::prepare()
|
||||
ctx->commands_for_part,
|
||||
ctx->for_interpreter,
|
||||
ctx->for_file_renames,
|
||||
suitable_for_ttl_optimization,
|
||||
ctx->log);
|
||||
|
||||
ctx->stage_progress = std::make_unique<MergeStageProgress>(1.0);
|
||||
@ -2391,12 +2296,7 @@ bool MutateTask::prepare()
|
||||
/// The blobs have to be removed along with the part, this temporary part owns them and does not share them yet.
|
||||
ctx->new_data_part->remove_tmp_policy = IMergeTreeDataPart::BlobsRemovalPolicyForTemporaryParts::REMOVE_BLOBS;
|
||||
|
||||
bool drop_expired_parts = suitable_for_ttl_optimization && !(*ctx->data->getSettings())[MergeTreeSetting::materialize_ttl_recalculate_only];
|
||||
if (drop_expired_parts)
|
||||
task = std::make_unique<ExecutableTaskDropTTLExpiredPartsDecorator>(std::make_unique<MutateAllPartColumnsTask>(ctx), ctx);
|
||||
else
|
||||
task = std::make_unique<MutateAllPartColumnsTask>(ctx);
|
||||
|
||||
task = std::make_unique<MutateAllPartColumnsTask>(ctx);
|
||||
ProfileEvents::increment(ProfileEvents::MutationAllPartColumns);
|
||||
}
|
||||
else /// TODO: check that we modify only non-key columns in this case.
|
||||
@ -2456,12 +2356,7 @@ bool MutateTask::prepare()
|
||||
/// Keeper has to be asked with unlock request to release the references to the blobs
|
||||
ctx->new_data_part->remove_tmp_policy = IMergeTreeDataPart::BlobsRemovalPolicyForTemporaryParts::ASK_KEEPER;
|
||||
|
||||
bool drop_expired_parts = suitable_for_ttl_optimization && !(*ctx->data->getSettings())[MergeTreeSetting::materialize_ttl_recalculate_only];
|
||||
if (drop_expired_parts)
|
||||
task = std::make_unique<ExecutableTaskDropTTLExpiredPartsDecorator>(std::make_unique<MutateSomePartColumnsTask>(ctx), ctx);
|
||||
else
|
||||
task = std::make_unique<MutateSomePartColumnsTask>(ctx);
|
||||
|
||||
task = std::make_unique<MutateSomePartColumnsTask>(ctx);
|
||||
ProfileEvents::increment(ProfileEvents::MutationSomePartColumns);
|
||||
}
|
||||
|
||||
|
@ -260,12 +260,6 @@ bool StorageInMemoryMetadata::hasAnyTableTTL() const
|
||||
return hasAnyMoveTTL() || hasRowsTTL() || hasAnyRecompressionTTL() || hasAnyGroupByTTL() || hasAnyRowsWhereTTL();
|
||||
}
|
||||
|
||||
bool StorageInMemoryMetadata::hasOnlyRowsTTL() const
|
||||
{
|
||||
bool has_any_other_ttl = hasAnyMoveTTL() || hasAnyRecompressionTTL() || hasAnyGroupByTTL() || hasAnyRowsWhereTTL() || hasAnyColumnTTL();
|
||||
return hasRowsTTL() && !has_any_other_ttl;
|
||||
}
|
||||
|
||||
TTLColumnsDescription StorageInMemoryMetadata::getColumnTTLs() const
|
||||
{
|
||||
return column_ttls_by_name;
|
||||
|
@ -144,9 +144,6 @@ struct StorageInMemoryMetadata
|
||||
/// Returns true if there is set table TTL, any column TTL or any move TTL.
|
||||
bool hasAnyTTL() const { return hasAnyColumnTTL() || hasAnyTableTTL(); }
|
||||
|
||||
/// Returns true if only rows TTL is set, not even rows where.
|
||||
bool hasOnlyRowsTTL() const;
|
||||
|
||||
/// Common tables TTLs (for rows and moves).
|
||||
TTLTableDescription getTableTTLs() const;
|
||||
bool hasAnyTableTTL() const;
|
||||
|
@ -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,7 @@ 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_swap_table": lambda: random.choice(["auto", "false", "true"]),
|
||||
"output_format_native_write_json_as_string": lambda: random.randint(0, 1),
|
||||
"enable_vertical_final": lambda: random.randint(0, 1),
|
||||
}
|
||||
|
@ -68,6 +68,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 = "=================="
|
||||
|
||||
@ -504,7 +505,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:
|
||||
@ -1082,7 +1082,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):
|
||||
@ -1682,7 +1682,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
|
||||
@ -4624,7 +4624,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_swap_table", random.choice(["auto", "true", "false"])
|
||||
|
||||
|
||||
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 SETTINGS join_algorithm='hash'",
|
||||
"SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS query_plan_join_swap_table = 'false', join_algorithm='hash'",
|
||||
)
|
||||
client1.expect("Peak memory usage", timeout=60)
|
||||
client1.expect(prompt)
|
||||
|
@ -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 order by all;
|
||||
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 order by all;
|
||||
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 order by all;
|
||||
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 order by all;
|
||||
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 order by all;
|
||||
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) ORDER BY ALL;
|
||||
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_swap_table = '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,37 +33,37 @@
|
||||
2 2
|
||||
2 2
|
||||
-- { echoOn }
|
||||
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 order by all 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 order by all 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;
|
||||
0 3
|
||||
2 2
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 order by all SETTINGS enable_analyzer = 1;
|
||||
0 3
|
||||
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
|
||||
SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 order by all SETTINGS enable_analyzer = 1;
|
||||
0 3
|
||||
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 order by all SETTINGS enable_analyzer = 1;
|
||||
0 2
|
||||
0 3
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 order by all 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 ORDER BY 2, 1 SETTINGS enable_analyzer = 1;
|
||||
1 0
|
||||
2 0
|
||||
0 2
|
||||
0 3
|
||||
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;
|
||||
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 order by all SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 order by all SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 order by all 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 order by all SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 order by all SETTINGS enable_analyzer = 1;
|
||||
SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 order by all 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);
|
||||
|
@ -1,3 +1,6 @@
|
||||
SET query_plan_join_swap_table = false;
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
EXPLAIN PIPELINE
|
||||
SELECT * FROM
|
||||
(
|
||||
|
@ -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;
|
||||
|
@ -50,7 +50,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
|
||||
@ -63,7 +64,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
|
||||
@ -76,7 +78,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,
|
||||
@ -88,7 +91,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_swap_table = 'false';
|
||||
|
||||
-- { 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: ConcurrentHashJoin
|
||||
@ -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_swap_table = 'false';
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
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: ConcurrentHashJoin
|
||||
@ -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_swap_table = 'false';
|
||||
;
|
||||
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: ConcurrentHashJoin
|
||||
@ -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_swap_table = 'false'
|
||||
;
|
||||
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: ConcurrentHashJoin
|
||||
@ -214,7 +220,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_swap_table = 'false'
|
||||
;
|
||||
Expression ((Project names + (Projection + )))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -222,18 +230,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: ConcurrentHashJoin
|
||||
@ -281,7 +289,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_swap_table = 'false'
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -289,31 +299,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: ConcurrentHashJoin
|
||||
@ -355,7 +365,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_swap_table = 'false'
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -363,31 +375,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
|
||||
@ -429,7 +441,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_swap_table = 'false'
|
||||
;
|
||||
Expression ((Project names + (Projection + )))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -437,18 +451,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
|
||||
@ -496,7 +510,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_swap_table = 'false'
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -504,31 +520,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
|
||||
@ -570,7 +586,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_swap_table = 'false'
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -578,31 +596,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
|
||||
@ -644,7 +662,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_swap_table = 'false'
|
||||
;
|
||||
Expression ((Project names + Projection))
|
||||
Header: id UInt64
|
||||
rhs.id UInt64
|
||||
@ -652,18 +672,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
|
||||
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
|
||||
AND column: equals(__table1.id, 5_UInt8)
|
||||
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 1
|
||||
@ -672,18 +692,18 @@ Positions: 4 2 0 1
|
||||
Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed)
|
||||
Actions: INPUT :: 1 -> __table1.id UInt64 : 0
|
||||
INPUT :: 2 -> __table1.value String : 1
|
||||
INPUT :: 3 -> __table2.value String : 2
|
||||
INPUT : 4 -> __table2.id UInt64 : 3
|
||||
INPUT : 3 -> __table2.id UInt64 : 2
|
||||
INPUT :: 4 -> __table2.value String : 3
|
||||
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 4
|
||||
INPUT : 0 -> equals(__table1.id, 5_UInt8) UInt8 : 5
|
||||
FUNCTION equals(__table2.id : 3, 6_UInt8 :: 4) -> equals(__table2.id, 6_UInt8) UInt8 : 6
|
||||
FUNCTION equals(__table2.id : 2, 6_UInt8 :: 4) -> equals(__table2.id, 6_UInt8) UInt8 : 6
|
||||
FUNCTION and(equals(__table1.id, 5_UInt8) :: 5, equals(__table2.id, 6_UInt8) :: 6) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 4
|
||||
Positions: 4 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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false';
|
||||
;
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
|
@ -53,7 +53,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_swap_table = 'false'
|
||||
;
|
||||
1 2 arc 1 -> 2 false [(1,2)]
|
||||
1 3 arc 1 -> 3 false [(1,3)]
|
||||
2 3 arc 2 -> 3 false [(2,3)]
|
||||
|
@ -56,7 +56,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_swap_table = 'false'
|
||||
;
|
||||
|
||||
-- ordering by the path column has same effect as SEARCH DEPTH FIRST
|
||||
WITH RECURSIVE search_graph AS (
|
||||
|
@ -3,6 +3,7 @@
|
||||
|
||||
SET join_algorithm = 'default'; -- for 'full_sorting_merge' the query is 10x slower
|
||||
SET enable_analyzer = 1; -- old analyzer returns TOO_DEEP_SUBQUERIES
|
||||
SET query_plan_join_swap_table = 'auto'; -- 'true' is slower
|
||||
SET max_threads = 8;
|
||||
|
||||
-- Bug 33446, marked as 'long' because it still runs around 10 sec
|
||||
|
@ -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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false'
|
||||
;
|
||||
|
||||
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_swap_table = 'false';
|
||||
|
||||
CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS
|
||||
SELECT id_values.id1 AS id,
|
||||
|
4
tests/queries/0_stateless/03267_join_swap_bug.reference
Normal file
4
tests/queries/0_stateless/03267_join_swap_bug.reference
Normal file
@ -0,0 +1,4 @@
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
33
tests/queries/0_stateless/03267_join_swap_bug.sql
Normal file
33
tests/queries/0_stateless/03267_join_swap_bug.sql
Normal file
@ -0,0 +1,33 @@
|
||||
DROP TABLE IF EXISTS t0;
|
||||
CREATE TABLE t0 (c0 Int) ENGINE = MergeTree ORDER BY tuple();
|
||||
INSERT INTO TABLE t0 (c0) VALUES (1);
|
||||
|
||||
SELECT 1 FROM t0 PASTE JOIN (SELECT 1 c0) tx PASTE JOIN t0 t1 GROUP BY tx.c0;
|
||||
SELECT count() FROM t0 PASTE JOIN (SELECT 1 c0) tx PASTE JOIN t0 t1 GROUP BY tx.c0;
|
||||
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT *
|
||||
FROM
|
||||
(
|
||||
SELECT *
|
||||
FROM system.one
|
||||
) AS a
|
||||
INNER JOIN
|
||||
(
|
||||
SELECT *
|
||||
FROM system.one
|
||||
) AS b USING (dummy)
|
||||
INNER JOIN
|
||||
(
|
||||
SELECT *
|
||||
FROM system.one
|
||||
) AS c USING (dummy)
|
||||
SETTINGS join_algorithm = 'full_sorting_merge';
|
||||
|
||||
|
||||
SELECT count(1)
|
||||
FROM ( SELECT 1 AS x, x ) AS t1
|
||||
RIGHT JOIN (SELECT materialize(2) AS x) AS t2
|
||||
ON t1.x = t2.x
|
||||
;
|
@ -0,0 +1,64 @@
|
||||
-- Tags: long, no-asan, no-tsan, no-ubsan, no-msan
|
||||
|
||||
-- This test verifies assertions and logical errors when selecting different subsets of columns using various join types and algorithms
|
||||
|
||||
DROP TABLE IF EXISTS t0;
|
||||
DROP TABLE IF EXISTS t1;
|
||||
|
||||
CREATE TABLE t0 (c0 UInt32, c1 String, c2 String) ENGINE = MergeTree ORDER BY c0;
|
||||
INSERT INTO t0 VALUES (1, 'a', 'b'), (2, 'c', 'd'), (3, 'e', 'f');
|
||||
|
||||
CREATE TABLE t1 (c0 UInt32, c1 String, c2 String) ENGINE = MergeTree ORDER BY c0;
|
||||
INSERT INTO t1 VALUES (2, 'c', 'd'), (3, 'e', 'f'), (4, 'g', 'h');
|
||||
|
||||
{% set columns_list =[
|
||||
'*',
|
||||
't0.c0',
|
||||
't1.c0',
|
||||
't0.c2',
|
||||
't1.c2',
|
||||
't1.c2, t1.c0',
|
||||
't1.c2, t0.c0',
|
||||
'count()',
|
||||
'1',
|
||||
] -%}
|
||||
|
||||
{%- for kind in [
|
||||
'INNER',
|
||||
'LEFT',
|
||||
'RIGHT',
|
||||
'FULL',
|
||||
'ANY INNER',
|
||||
'ANY LEFT',
|
||||
'ANY RIGHT',
|
||||
'ANTI LEFT',
|
||||
'ANTI RIGHT',
|
||||
'SEMI LEFT',
|
||||
'SEMI RIGHT',
|
||||
] -%}
|
||||
|
||||
{%- for columns in columns_list -%}
|
||||
{%- for join_algorithm in ['default', 'hash', 'full_sorting_merge', 'parallel_hash', 'grace_hash'] -%}
|
||||
{%- for condition in ['ON t1.c0 = t0.c0', 'USING (c0)'] -%}
|
||||
|
||||
{%- set is_not_supported = (
|
||||
('ANTI' in kind and join_algorithm == 'full_sorting_merge') or
|
||||
('SEMI' in kind and join_algorithm == 'full_sorting_merge')
|
||||
) -%}
|
||||
|
||||
SELECT {{ columns }} FROM t0 {{ kind }} JOIN t1 {{ condition }}
|
||||
SETTINGS join_algorithm = '{{ join_algorithm }}'
|
||||
FORMAT Null; {{ '-- { serverError NOT_IMPLEMENTED }' if is_not_supported else '' }}
|
||||
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
|
||||
{% for kind in ['ASOF', 'ASOF LEFT'] -%}
|
||||
{%- for columns in columns_list -%}
|
||||
|
||||
SELECT {{ columns }} FROM t0 {{ kind }} JOIN t1 ON t1.c1 = t0.c1 AND t1.c0 <= t0.c0 FORMAT Null;
|
||||
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
Loading…
Reference in New Issue
Block a user