Improve join

now adding joined blocks from right table can be run parallelly, speedup the join process
This commit is contained in:
lgbo-ustc 2022-04-19 16:07:30 +08:00
parent 617a291386
commit 3d7338581b
14 changed files with 1006 additions and 13 deletions

View File

@ -351,6 +351,7 @@ class IColumn;
M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \
M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \
M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \
M(Bool, enable_parallel_join, false, "Enable paralle join algorithm.", 0)\
M(String, temporary_files_codec, "LZ4", "Set compression codec for temporary files (sort and join on disk). I.e. LZ4, NONE.", 0) \
\
M(UInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.", 0) \

View File

@ -0,0 +1,325 @@
#include <memory>
#include <mutex>
#include <Interpreters/ConcurrentHashJoin.h>
#include <Interpreters/TableJoin.h>
#include "Columns/FilterDescription.h"
#include "Columns/IColumn.h"
#include "Core/ColumnsWithTypeAndName.h"
#include "Core/NamesAndTypes.h"
#include "IO/WriteBufferFromString.h"
#include "Interpreters/ActionsDAG.h"
#include "Interpreters/ActionsVisitor.h"
#include "Interpreters/ExpressionActions.h"
#include "Interpreters/PreparedSets.h"
#include "Interpreters/SubqueryForSet.h"
#include "Parsers/DumpASTNode.h"
#include "Parsers/ExpressionListParsers.h"
#include "Parsers/IAST_fwd.h"
#include "Parsers/parseQuery.h"
#include <IO/Operators.h>
#include <Interpreters/Context.h>
#include "Common/Exception.h"
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int SET_SIZE_LIMIT_EXCEEDED;
extern const int BAD_ARGUMENTS;
}
namespace JoinStuff
{
ConcurrentHashJoin::ConcurrentHashJoin(ContextPtr context_, std::shared_ptr<TableJoin> table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_)
: context(context_)
, table_join(table_join_)
, slots(slots_)
{
if (!slots_)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid argument slot : {}", slots_);
}
for (size_t i = 0; i < slots; ++i)
{
auto inner_hash_join = std::make_shared<InnerHashJoin>();
inner_hash_join->data = std::make_unique<HashJoin>(table_join_, right_sample_block, any_take_last_row_);
hash_joins.emplace_back(std::move(inner_hash_join));
}
dispatch_datas.emplace_back(std::make_shared<BlockDispatchControlData>());
dispatch_datas.emplace_back(std::make_shared<BlockDispatchControlData>());
}
bool ConcurrentHashJoin::addJoinedBlock(const Block & block, bool check_limits)
{
auto & dispatch_data = getBlockDispatchControlData(block, RIGHT);
std::vector<Block> dispatched_blocks;
Block cloned_block = block;
dispatchBlock(dispatch_data, cloned_block, dispatched_blocks);
for (size_t i = 0; i < dispatched_blocks.size(); ++i)
{
auto & hash_join = hash_joins[i];
auto & dispatched_block = dispatched_blocks[i];
std::unique_lock lock(hash_join->mutex);
hash_join->rows += dispatched_block.rows();
check_total_rows += dispatched_block.rows();
check_total_bytes += dispatched_block.bytes();
// Don't take the real insertion here, because inserting a block into HashTable is a time-consuming operation,
// it may cause serious lock contention and make the whole process slow.
hash_join->pending_right_blocks.emplace_back(std::move(dispatched_block));
}
if (check_limits)
return table_join->sizeLimits().check(
check_total_rows.load(), check_total_bytes.load(), "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
return true;
}
void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed)
{
if (block.rows())
waitAllAddJoinedBlocksFinished();
else
{
std::unique_lock lock(hash_joins[0]->mutex);
hash_joins[0]->data->joinBlock(block, not_processed);
return;
}
auto & dispatch_data = getBlockDispatchControlData(block, LEFT);
std::vector<Block> dispatched_blocks;
Block cloned_block = block;
dispatchBlock(dispatch_data, cloned_block, dispatched_blocks);
for (size_t i = 0; i < dispatched_blocks.size(); ++i)
{
std::shared_ptr<ExtraBlock> none_extra_block;
auto & hash_join = hash_joins[i];
auto & dispatched_block = dispatched_blocks[i];
hash_join->data->joinBlock(dispatched_block, none_extra_block);
if (none_extra_block && !none_extra_block->empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "not_processed should be empty");
}
ColumnsWithTypeAndName final_columns;
MutableColumns mutable_final_columns;
NamesAndTypesList names_and_types = dispatched_blocks[0].getNamesAndTypesList();
auto types = names_and_types.getTypes();
for (auto & dispatched_block : dispatched_blocks)
{
for (size_t pos = 0; pos < dispatched_block.columns(); ++pos)
{
auto & from_column = dispatched_block.getByPosition(pos);
if (mutable_final_columns.size() <= pos)
{
mutable_final_columns.emplace_back(from_column.column->cloneEmpty());
}
if (!from_column.column->empty())
{
mutable_final_columns[pos]->insertRangeFrom(*from_column.column, 0, from_column.column->size());
}
}
}
size_t i = 0;
for (auto & name_and_type : names_and_types)
{
ColumnPtr col_ptr = std::move(mutable_final_columns[i]);
mutable_final_columns[i] = nullptr;
ColumnWithTypeAndName col(col_ptr, name_and_type.type, name_and_type.name);
final_columns.emplace_back(col);
i += 1;
}
block = Block(final_columns);
}
void ConcurrentHashJoin::checkTypesOfKeys(const Block & block) const
{
hash_joins[0]->data->checkTypesOfKeys(block);
}
void ConcurrentHashJoin::setTotals(const Block & block)
{
if (block)
{
std::lock_guard lock(totals_mutex);
totals = block;
}
}
const Block & ConcurrentHashJoin::getTotals() const
{
return totals;
}
size_t ConcurrentHashJoin::getTotalRowCount() const
{
size_t res = 0;
for (const auto & hash_join : hash_joins)
{
res += hash_join->data->getTotalRowCount();
}
return res;
}
size_t ConcurrentHashJoin::getTotalByteCount() const
{
size_t res = 0;
for (const auto & hash_join : hash_joins)
{
res += hash_join->data->getTotalByteCount();
}
return res;
}
bool ConcurrentHashJoin::alwaysReturnsEmptySet() const
{
for (const auto & hash_join : hash_joins)
{
if (!hash_join->data->alwaysReturnsEmptySet() || !hash_join->pending_right_blocks.empty())
return false;
}
return true;
}
std::shared_ptr<NotJoinedBlocks> ConcurrentHashJoin::getNonJoinedBlocks(
const Block & /*left_sample_block*/, const Block & /*result_sample_block*/, UInt64 /*max_block_size*/) const
{
if (table_join->strictness() == ASTTableJoin::Strictness::Asof ||
table_join->strictness() == ASTTableJoin::Strictness::Semi ||
!isRightOrFull(table_join->kind()))
{
return {};
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid join type. join kind: {}, strictness: {}", table_join->kind(), table_join->strictness());
}
std::shared_ptr<ExpressionActions> ConcurrentHashJoin::buildHashExpressionAction(const Block & block, const Strings & based_columns_names, Strings & hash_columns_names)
{
WriteBufferFromOwnString col_buf;
for (size_t i = 0, sz = based_columns_names.size(); i < sz; ++i)
{
if (i)
col_buf << ",";
col_buf << based_columns_names[i];
}
WriteBufferFromOwnString write_buf;
for (size_t i = 0; i < slots; ++i)
{
if (i)
write_buf << ",";
write_buf << "cityHash64(" << col_buf.str() << ")%" << slots << "=" << i;
}
auto settings = context->getSettings();
ParserExpressionList hash_expr_parser(true);
ASTPtr func_ast = parseQuery(hash_expr_parser, write_buf.str(), "Parse Block hash expression", settings.max_query_size, settings.max_parser_depth);
for (auto & child : func_ast->children)
hash_columns_names.emplace_back(child->getColumnName());
DebugASTLog<false> visit_log;
const auto & names_and_types = block.getNamesAndTypesList();
ActionsDAGPtr actions = std::make_shared<ActionsDAG>(names_and_types);
PreparedSets prepared_sets;
SubqueriesForSets subqueries_for_sets;
ActionsVisitor::Data visitor_data(
context,
SizeLimits{settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode},
10,
names_and_types,
std::move(actions),
prepared_sets,
subqueries_for_sets,
true, false, true, false);
ActionsVisitor(visitor_data, visit_log.stream()).visit(func_ast);
actions = visitor_data.getActions();
return std::make_shared<ExpressionActions>(actions);
}
ConcurrentHashJoin::BlockDispatchControlData & ConcurrentHashJoin::getBlockDispatchControlData(const Block & block, TableIndex table_index)
{
auto & data = *dispatch_datas[table_index];
if (data.has_init)[[likely]]
return data;
std::lock_guard lock(data.mutex);
if (data.has_init)
return data;
if (table_join->getClauses().empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "empty join clauses");
const auto & onexpr = table_join->getClauses()[0];
if (table_index == LEFT)
{
data.hash_expression_actions = buildHashExpressionAction(block, onexpr.key_names_left, data.hash_columns_names);
}
else
{
data.hash_expression_actions = buildHashExpressionAction(block, onexpr.key_names_right, data.hash_columns_names);
}
data.header = block.cloneEmpty();
data.has_init = true;
return data;
}
void ConcurrentHashJoin::dispatchBlock(BlockDispatchControlData & dispatch_data, Block & from_block, std::vector<Block> & dispatched_blocks)
{
auto rows_before_filtration = from_block.rows();
dispatch_data.hash_expression_actions->execute(from_block, rows_before_filtration);
for (const auto & filter_column_name : dispatch_data.hash_columns_names)
{
auto full_column = from_block.findByName(filter_column_name)->column->convertToFullColumnIfConst();
auto filter_desc = std::make_unique<FilterDescription>(*full_column);
auto num_filtered_rows = filter_desc->countBytesInFilter();
ColumnsWithTypeAndName filtered_block_columns;
for (size_t i = 0; i < dispatch_data.header.columns(); ++i)
{
auto & from_column = from_block.getByPosition(i);
auto filtered_column = filter_desc->filter(*from_column.column, num_filtered_rows);
filtered_block_columns.emplace_back(filtered_column, from_column.type, from_column.name);
}
dispatched_blocks.emplace_back(std::move(filtered_block_columns));
}
}
void ConcurrentHashJoin::waitAllAddJoinedBlocksFinished()
{
while (finished_add_joined_blocks_tasks < hash_joins.size())[[unlikely]]
{
std::shared_ptr<InnerHashJoin> hash_join;
{
std::unique_lock lock(finished_add_joined_blocks_tasks_mutex);
hash_join = getUnfinishedAddJoinedBlockTaks();
if (!hash_join)
{
while (finished_add_joined_blocks_tasks < hash_joins.size())
{
finished_add_joined_blocks_tasks_cond.wait(lock);
}
return;
}
}
std::unique_lock lock(hash_join->mutex);
while (!hash_join->pending_right_blocks.empty())
{
Block & block = hash_join->pending_right_blocks.front();
hash_join->data->addJoinedBlock(block, true);
hash_join->pending_right_blocks.pop_front();
}
finished_add_joined_blocks_tasks += 1;
finished_add_joined_blocks_tasks_cond.notify_all();
}
}
std::shared_ptr<ConcurrentHashJoin::InnerHashJoin> ConcurrentHashJoin::getUnfinishedAddJoinedBlockTaks()
{
for (auto & hash_join : hash_joins)
{
if (!hash_join->in_inserting)
{
hash_join->in_inserting = true;
return hash_join;
}
}
return nullptr;
}
}
}

View File

@ -0,0 +1,105 @@
#pragma once
#include <condition_variable>
#include <memory>
#include <optional>
#include <Interpreters/HashJoin.h>
#include "Core/BackgroundSchedulePool.h"
#include "Functions/FunctionsLogical.h"
#include "Interpreters/ExpressionActions.h"
#include "Interpreters/IJoin.h"
#include "base/defines.h"
#include "base/types.h"
#include <Common/Stopwatch.h>
#include <Interpreters/Context.h>
namespace DB
{
namespace JoinStuff
{
/**
* Can run addJoinedBlock() parallelly to speedup the join process. On test, it almose linear speedup by
* the degree of parallelism.
*
* The default HashJoin is not thread safe for inserting right table's rows and run it in a single thread. When
* the right table is large, the join process is too slow.
*
* We create multiple HashJoin instances here. In addJoinedBlock(), one input block is split into multiple blocks
* corresponding to the HashJoin instances by hashing every row on the join keys. And make a guarantee that every HashJoin
* instance is written by only one thread.
*
* When come to the left table matching, the blocks from left table are alse split into different HashJoin instances.
*
*/
class ConcurrentHashJoin : public IJoin
{
public:
explicit ConcurrentHashJoin(ContextPtr context_, std::shared_ptr<TableJoin> table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_ = false);
~ConcurrentHashJoin() override = default;
const TableJoin & getTableJoin() const override { return *table_join; }
bool addJoinedBlock(const Block & block, bool check_limits) override;
void checkTypesOfKeys(const Block & block) const override;
void joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed) override;
void setTotals(const Block & block) override;
const Block & getTotals() const override;
size_t getTotalRowCount() const override;
size_t getTotalByteCount() const override;
bool alwaysReturnsEmptySet() const override;
bool supportParallelJoin() const override { return true; }
std::shared_ptr<NotJoinedBlocks>
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override;
private:
struct InnerHashJoin
{
std::mutex mutex;
std::unique_ptr<HashJoin> data;
std::list<Block> pending_right_blocks;
std::atomic<bool> in_inserting = false;
std::atomic<UInt32> rows = 0;
};
ContextPtr context;
std::shared_ptr<TableJoin> table_join;
size_t slots;
std::vector<std::shared_ptr<InnerHashJoin>> hash_joins;
std::atomic<size_t> check_total_rows;
std::atomic<size_t> check_total_bytes;
std::mutex finished_add_joined_blocks_tasks_mutex;
std::condition_variable finished_add_joined_blocks_tasks_cond;
std::atomic<UInt32> finished_add_joined_blocks_tasks = 0;
mutable std::mutex totals_mutex;
Block totals;
enum TableIndex
{
LEFT = 0,
RIGHT = 1
};
struct BlockDispatchControlData
{
std::mutex mutex;
std::atomic<bool> has_init = false;
std::shared_ptr<ExpressionActions> hash_expression_actions;
Strings hash_columns_names;
Block header;
BlockDispatchControlData() = default;
};
std::vector<std::shared_ptr<BlockDispatchControlData>> dispatch_datas;
Poco::Logger * logger = &Poco::Logger::get("ConcurrentHashJoin");
std::shared_ptr<ExpressionActions> buildHashExpressionAction(const Block & block, const Strings & based_columns_names, Strings & hash_columns_names);
BlockDispatchControlData & getBlockDispatchControlData(const Block & block, TableIndex table_index);
static void dispatchBlock(BlockDispatchControlData & dispatch_data, Block & from_block, std::vector<Block> & dispatched_blocks);
void waitAllAddJoinedBlocksFinished();
std::shared_ptr<InnerHashJoin> getUnfinishedAddJoinedBlockTaks();
};
}
}

View File

@ -1,3 +1,4 @@
#include <memory>
#include <Core/Block.h>
#include <Parsers/ASTExpressionList.h>
@ -16,6 +17,7 @@
#include <Interpreters/ArrayJoinAction.h>
#include <Interpreters/Context.h>
#include <Interpreters/ConcurrentHashJoin.h>
#include <Interpreters/DictionaryReader.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/ExpressionActions.h>
@ -933,7 +935,13 @@ static std::shared_ptr<IJoin> chooseJoinAlgorithm(std::shared_ptr<TableJoin> ana
bool allow_merge_join = analyzed_join->allowMergeJoin();
if (analyzed_join->forceHashJoin() || (analyzed_join->preferMergeJoin() && !allow_merge_join))
{
if (analyzed_join->allowConcurrentHashJoin())
{
return std::make_shared<JoinStuff::ConcurrentHashJoin>(context, analyzed_join, context->getSettings().max_threads, sample_block);
}
return std::make_shared<HashJoin>(analyzed_join, sample_block);
}
else if (analyzed_join->forceMergeJoin() || (analyzed_join->preferMergeJoin() && allow_merge_join))
return std::make_shared<MergeJoin>(analyzed_join, sample_block);
return std::make_shared<JoinSwitcher>(analyzed_join, sample_block);

View File

@ -45,6 +45,9 @@ public:
/// Different query plan is used for such joins.
virtual bool isFilled() const { return false; }
// That can run FillingRightJoinSideTransform parallelly
virtual bool supportParallelJoin() const { return false; }
virtual std::shared_ptr<NotJoinedBlocks>
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0;
};

View File

@ -1269,7 +1269,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
query_plan.getCurrentDataStream(),
joined_plan->getCurrentDataStream(),
expressions.join,
settings.max_block_size);
settings.max_block_size,
max_streams);
join_step->setStepDescription("JOIN");
std::vector<QueryPlanPtr> plans;

View File

@ -3,6 +3,7 @@
#include <Common/Exception.h>
#include <base/types.h>
#include <Common/StringUtils/StringUtils.h>
#include "Parsers/ASTTablesInSelectQuery.h"
#include <Interpreters/ActionsDAG.h>
#include <Core/Block.h>
@ -106,6 +107,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_)
, partial_merge_join_left_table_buffer_bytes(settings.partial_merge_join_left_table_buffer_bytes)
, max_files_to_merge(settings.join_on_disk_max_files_to_merge)
, temporary_files_codec(settings.temporary_files_codec)
, enable_parallel_join(settings.enable_parallel_join)
, tmp_volume(tmp_volume_)
{
}
@ -748,4 +750,16 @@ void TableJoin::resetToCross()
this->table_join.kind = ASTTableJoin::Kind::Cross;
}
bool TableJoin::allowConcurrentHashJoin() const
{
if (!enable_parallel_join)
return false;
if (dictionary_reader || join_algorithm != JoinAlgorithm::HASH)
return false;
if (table_join.kind != ASTTableJoin::Kind::Left && table_join.kind != ASTTableJoin::Kind::Inner)
return false;
return true;
}
}

View File

@ -112,6 +112,7 @@ private:
const size_t partial_merge_join_left_table_buffer_bytes = 0;
const size_t max_files_to_merge = 0;
const String temporary_files_codec = "LZ4";
const bool enable_parallel_join = false;
/// the limit has no technical reasons, it supposed to improve safety
const size_t MAX_DISJUNCTS = 16; /// NOLINT
@ -191,6 +192,7 @@ public:
bool allowMergeJoin() const;
bool preferMergeJoin() const { return join_algorithm == JoinAlgorithm::PREFER_PARTIAL_MERGE; }
bool forceMergeJoin() const { return join_algorithm == JoinAlgorithm::PARTIAL_MERGE; }
bool allowConcurrentHashJoin() const;
bool forceHashJoin() const
{
/// HashJoin always used for DictJoin

View File

@ -15,9 +15,11 @@ JoinStep::JoinStep(
const DataStream & left_stream_,
const DataStream & right_stream_,
JoinPtr join_,
size_t max_block_size_)
size_t max_block_size_,
size_t max_streams_)
: join(std::move(join_))
, max_block_size(max_block_size_)
, max_streams(max_streams_)
{
input_streams = {left_stream_, right_stream_};
output_stream = DataStream
@ -31,7 +33,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines
if (pipelines.size() != 2)
throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps");
return QueryPipelineBuilder::joinPipelines(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, &processors);
return QueryPipelineBuilder::joinPipelines(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, &processors, max_streams);
}
void JoinStep::describePipeline(FormatSettings & settings) const

View File

@ -16,7 +16,8 @@ public:
const DataStream & left_stream_,
const DataStream & right_stream_,
JoinPtr join_,
size_t max_block_size_);
size_t max_block_size_,
size_t max_streams_ = 0);
String getName() const override { return "Join"; }
@ -29,6 +30,7 @@ public:
private:
JoinPtr join;
size_t max_block_size;
size_t max_streams;
Processors processors;
};

View File

@ -15,6 +15,7 @@
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/IJoin.h>
#include <Common/typeid_cast.h>
#include <Common/CurrentThread.h>
#include <Processors/DelayedPortsProcessor.h>
@ -298,7 +299,8 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelines(
std::unique_ptr<QueryPipelineBuilder> right,
JoinPtr join,
size_t max_block_size,
Processors * collected_processors)
Processors * collected_processors,
size_t max_streams)
{
left->checkInitializedAndNotCompleted();
right->checkInitializedAndNotCompleted();
@ -336,7 +338,30 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelines(
/// ╞> FillingJoin ─> Resize ╣ ╞> Joining ─> (totals)
/// (totals) ─────────┘ ╙─────┘
size_t num_streams = left->getNumStreams();
size_t num_streams = left->getNumStreams() < max_streams ? max_streams : left->getNumStreams();
left->resize(num_streams);
num_streams = left->getNumStreams();
if (join->supportParallelJoin() && !right->hasTotals())
{
right->resize(num_streams);
auto concurrent_right_filling_transform = [&](OutputPortRawPtrs outports)
{
Processors processors;
for (auto & outport : outports)
{
auto adding_joined = std::make_shared<FillingRightJoinSideTransform>(right->getHeader(), join);
connect(*outport, adding_joined->getInputs().front());
processors.emplace_back(adding_joined);
}
return processors;
};
right->transform(concurrent_right_filling_transform);
right->resize(1);
}
else
{
LOG_TRACE(&Poco::Logger::get("QueryPipelineBuilder"), "run in single thread on right loading");
right->resize(1);
auto adding_joined = std::make_shared<FillingRightJoinSideTransform>(right->getHeader(), join);
@ -345,6 +370,7 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelines(
totals_port = adding_joined->addTotalsPort();
right->addTransform(std::move(adding_joined), totals_port, nullptr);
}
size_t num_streams_including_totals = num_streams + (left->hasTotals() ? 1 : 0);
right->resize(num_streams_including_totals);

View File

@ -101,7 +101,8 @@ public:
std::unique_ptr<QueryPipelineBuilder> right,
JoinPtr join,
size_t max_block_size,
Processors * collected_processors = nullptr);
Processors * collected_processors = nullptr,
size_t max_streams = 0);
/// Add other pipeline and execute it before current one.
/// Pipeline must have empty header, it should not generate any chunk.

View File

@ -0,0 +1,299 @@
2014-03-17 1406958 265108
2014-03-19 1405797 261624
2014-03-18 1383658 258723
2014-03-20 1353623 255328
2014-03-21 1245779 236232
2014-03-23 1046491 202212
2014-03-22 1031592 197354
2014-03-17 1406958 265108
2014-03-19 1405797 261624
2014-03-18 1383658 258723
2014-03-20 1353623 255328
2014-03-21 1245779 236232
2014-03-23 1046491 202212
2014-03-22 1031592 197354
4508153 712428
auto.ru 576845 8935
yandex.ru 410776 111278
korer.ru 277987 0
avito.ru 163820 15556
mail.yandex.ru 152447 1046
mail.ru 87949 22225
best.ru 58537 55
korablitz.ru 51844 0
hurpass.com 49671 1251
37292 0 35642
92887 252214 0
7842 196036 0
42440 0 17837
97811 0 13665
99378 0 11401
6764 0 11052
43564 0 9522
37645 0 9086
76197 0 8379
24271 0 7393
39804 64180 0
78332 0 6167
82837 0 5928
51651 0 5878
63469 52152 0
76593 0 4705
54688 0 4421
36747 44082 0
51763 0 4309
92887 252214 0
7842 196036 0
39804 64180 0
63469 52152 0
36747 44082 0
10963 37562 0
46685 28798 0
24015 23368 0
37615 20857 0
62896 19719 0
67664 19402 0
96597 18557 0
11045 17391 0
25884 17302 0
64910 17279 0
26096 16994 0
96134 16849 0
5788 13974 0
71509 13792 0
67326 13181 0
37292 0 35642
42440 0 17837
97811 0 13665
99378 0 11401
6764 0 11052
43564 0 9522
37645 0 9086
76197 0 8379
24271 0 7393
78332 0 6167
82837 0 5928
51651 0 5878
76593 0 4705
54688 0 4421
51763 0 4309
37882 0 4094
6236 0 3959
28456 0 3746
19008 0 3426
81009 0 3158
2961521519262 2014-03-19 00:12:45 gen_time ['gen_time']
2961521519262 2014-03-19 00:12:45 gen_timestamp Счетчика нет (10 ['gen_timestamp']
2961521519262 2014-03-19 00:12:45 hasTemplates Счетчик есть null ['hasTemplates']
2961521519262 2014-03-19 00:12:45 hasTemplates Счетчик есть null ['hasTemplates']
2961521519262 2014-03-19 00:12:45 hasTemplates Счетчик есть null ['hasTemplates']
2961521519262 2014-03-19 00:12:45 Роль пользователя ['Роль пользователя']
2961521519262 2014-03-19 00:12:46 videoid ['videoid']
2961521519262 2014-03-19 00:12:46 videoid Done ['videoid']
2961521519262 2014-03-19 00:12:46 Поиск ['Поиск']
2961521519262 2014-03-19 00:12:47 gen_timestamp Счетчика нет (10 ['gen_timestamp']
2961521519262 2014-03-19 00:12:47 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
2961521519262 2014-03-19 00:12:51 videoid 8950vvvv ['videoid']
2961521519262 2014-03-19 00:14:11 errors SMS ['errors']
2961521519262 2014-03-19 00:14:12 gen_timestamp Счетчика нет (10 ['gen_timestamp']
2961521519262 2014-03-19 00:15:11 videoid Done ['videoid']
2961521519262 2014-03-19 00:15:12 videoid ['videoid']
2961521519262 2014-03-19 00:15:12 Поиск ['Поиск']
2961521519262 2014-03-19 00:15:12 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
2961521519262 2014-03-19 00:15:17 videoid 8950vvvv ['videoid']
2961521519262 2014-03-19 00:19:21 gen_timestamp 564.815 ['gen_timestamp']
2961521519262 2014-03-19 00:19:22 gen_timestamp 564.815 ['gen_timestamp']
164599821266083 2014-03-22 10:30:50 gen_time views ['gen_time']
164599821266083 2014-03-22 10:30:50 gen_timestamp vkontakte,face element ['gen_timestamp']
164599821266083 2014-03-22 10:30:50 Платность Превьюшки ['Платность']
164599821266083 2014-03-22 10:30:53 Аттачи в списке /pages/biblio ['Аттачи в списке']
164599821266083 2014-03-22 10:31:02 Платность Превьюшки ['Платность']
164599821266083 2014-03-22 10:31:05 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
164599821266083 2014-03-22 10:31:06 Поиск Счетчика ['Поиск']
335386658089392 2014-03-18 04:28:17 gen_time ['gen_time']
335386658089392 2014-03-18 04:28:18 Роль пользователя ['Роль пользователя']
335386658089392 2014-03-18 04:28:25 Поиск ['Поиск']
335386658089392 2014-03-18 04:28:26 videoid ['videoid']
335386658089392 2014-03-18 04:28:26 videoid Done ['videoid']
335386658089392 2014-03-18 04:28:26 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
335386658089392 2014-03-18 05:48:43 gen_time ['gen_time']
335386658089392 2014-03-18 05:48:43 Роль пользователя ['Роль пользователя']
335386658089392 2014-03-18 05:48:44 Дра ['Дра']
419794772979101 2014-03-17 22:08:42 gen_time ['gen_time']
419794772979101 2014-03-17 22:08:43 Роль пользователя ['Роль пользователя']
419794772979101 2014-03-17 22:08:50 Поиск ['Поиск']
419794772979101 2014-03-17 22:08:50 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
419794772979101 2014-03-19 14:26:49 gen_time ['gen_time']
419794772979101 2014-03-19 14:26:49 Дра ['Дра']
419794772979101 2014-03-19 14:26:49 Эксперимент про Счетчик есть null ['Эксперимент про']
419794772979101 2014-03-19 14:26:52 Поиск ['Поиск']
419794772979101 2014-03-19 14:26:52 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
818136115259082 2014-03-18 11:33:51 gen_time ['gen_time']
818136115259082 2014-03-18 11:33:51 Дра ['Дра']
818136115259082 2014-03-18 11:33:52 Роль пользователя ['Роль пользователя']
818136115259082 2014-03-18 11:33:55 Поиск ['Поиск']
818136115259082 2014-03-18 11:33:55 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
818136115259082 2014-03-18 11:34:42 gen_timestamp 564.815 ['gen_timestamp']
818136115259082 2014-03-18 11:34:44 gen_timestamp 564.815 ['gen_timestamp']
870023587671569 2014-03-16 22:17:44 gen_time ['gen_time']
870023587671569 2014-03-16 22:17:44 gen_timestamp vkontakte,face element ['gen_timestamp']
1391426411567902 2014-03-18 10:39:27 gen_time ['gen_time']
1391426411567902 2014-03-18 10:39:27 gen_timestamp Счетчика нет (10 ['gen_timestamp']
1391426411567902 2014-03-18 10:39:27 Платность model ['Платность']
1391426411567902 2014-03-18 10:39:27 Эксперимент про Счетчик есть null ['Эксперимент про']
1391426411567902 2014-03-18 10:39:37 videoid ['videoid']
1391426411567902 2014-03-18 10:39:37 videoid Done ['videoid']
1391426411567902 2014-03-18 10:39:37 Поиск ['Поиск']
1391426411567902 2014-03-18 10:39:37 Поиск ['Поиск']
1391426411567902 2014-03-18 10:39:37 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
1391426411567902 2014-03-18 10:39:37 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
1391426411567902 2014-03-18 10:40:07 gen_timestamp Счетчика нет (10 ['gen_timestamp']
1391426411567902 2014-03-18 10:43:42 Toolbar auto 25%;\n\t\t\t\t<![endif]--> \n <head>\n return ['Toolbar']
1391426411567902 2014-03-18 10:43:42 gen_timestamp Счетчика нет (10 ['gen_timestamp']
1391426411567902 2014-03-18 10:43:42 gen_timestamp Счетчика нет (10 ['gen_timestamp']
1391426411567902 2014-03-18 10:43:42 Права ['Права']
1391426411567902 2014-03-18 10:43:44 gen_timestamp Счетчика нет (10 ['gen_timestamp']
1391426411567902 2014-03-18 10:43:48 Toolbar Счетчик ['Toolbar']
1391426411567902 2014-03-18 10:43:48 gen_timestamp Счетчика нет (10 ['gen_timestamp']
1391426411567902 2014-03-18 10:43:48 Поиск ['Поиск']
1391426411567902 2014-03-18 10:43:48 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
1560113638322066 2014-03-17 13:21:19 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 13:21:19 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 13:28:21 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 13:28:22 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 13:28:22 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 17:01:54 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 17:15:06 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 17:15:09 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 17:15:09 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 17:22:15 errors 1e521117369b39edaa078c8021df25 ['errors']
1560113638322066 2014-03-17 17:22:15 gen_time ['gen_time']
1560113638322066 2014-03-17 17:22:15 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 17:22:15 Дра ['Дра']
1560113638322066 2014-03-17 17:22:15 Поиск ['Поиск']
1560113638322066 2014-03-17 17:22:16 Toolbar Счетчик ['Toolbar']
1560113638322066 2014-03-17 17:22:16 errors 1e521117369b39edaa078c8021df25 ['errors']
1560113638322066 2014-03-17 17:22:16 errors 1e521117369b39edaa078c8021df25 ['errors']
1560113638322066 2014-03-17 17:22:16 errors 1e521117369b39edaa078c8021df25 ['errors']
1560113638322066 2014-03-17 17:22:16 errors 1e521117369b39edaa078c8021df25 ['errors']
1560113638322066 2014-03-17 17:22:16 errors 1e521117369b39edaa078c8021df25 ['errors']
1560113638322066 2014-03-17 17:22:16 errors 1e521117369b39edaa078c8021df25 ['errors']
1560113638322066 2014-03-17 17:22:16 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
1560113638322066 2014-03-17 17:22:16 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
1560113638322066 2014-03-17 17:22:42 hasTemplates Счетчик есть null ['hasTemplates']
1560113638322066 2014-03-17 17:22:46 errors SMS ['errors']
2961521519262 2014-03-19 00:12:45 []
2961521519262 2014-03-19 00:12:45 gen_time ['gen_time']
2961521519262 2014-03-19 00:12:45 gen_timestamp Счетчика нет (10 ['gen_timestamp']
2961521519262 2014-03-19 00:12:45 hasTemplates Счетчик есть null ['hasTemplates']
2961521519262 2014-03-19 00:12:45 hasTemplates Счетчик есть null ['hasTemplates']
2961521519262 2014-03-19 00:12:45 hasTemplates Счетчик есть null ['hasTemplates']
2961521519262 2014-03-19 00:12:45 Роль пользователя ['Роль пользователя']
2961521519262 2014-03-19 00:12:46 videoid ['videoid']
2961521519262 2014-03-19 00:12:46 videoid Done ['videoid']
2961521519262 2014-03-19 00:12:46 Поиск ['Поиск']
2961521519262 2014-03-19 00:12:47 []
2961521519262 2014-03-19 00:12:47 gen_timestamp Счетчика нет (10 ['gen_timestamp']
2961521519262 2014-03-19 00:12:47 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
2961521519262 2014-03-19 00:12:51 []
2961521519262 2014-03-19 00:12:51 videoid 8950vvvv ['videoid']
2961521519262 2014-03-19 00:13:28 []
2961521519262 2014-03-19 00:14:11 errors SMS ['errors']
2961521519262 2014-03-19 00:14:12 gen_timestamp Счетчика нет (10 ['gen_timestamp']
2961521519262 2014-03-19 00:15:11 videoid Done ['videoid']
2961521519262 2014-03-19 00:15:12 []
2961521519262 2014-03-19 00:15:12 videoid ['videoid']
2961521519262 2014-03-19 00:15:12 Поиск ['Поиск']
2961521519262 2014-03-19 00:15:12 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
2961521519262 2014-03-19 00:15:17 []
2961521519262 2014-03-19 00:15:17 videoid 8950vvvv ['videoid']
2961521519262 2014-03-19 00:19:21 gen_timestamp 564.815 ['gen_timestamp']
2961521519262 2014-03-19 00:19:22 gen_timestamp 564.815 ['gen_timestamp']
164599821266083 2014-03-22 10:30:50 []
164599821266083 2014-03-22 10:30:50 []
164599821266083 2014-03-22 10:30:50 gen_time views ['gen_time']
164599821266083 2014-03-22 10:30:50 gen_timestamp vkontakte,face element ['gen_timestamp']
164599821266083 2014-03-22 10:30:50 Платность Превьюшки ['Платность']
164599821266083 2014-03-22 10:30:53 Аттачи в списке /pages/biblio ['Аттачи в списке']
164599821266083 2014-03-22 10:31:02 Платность Превьюшки ['Платность']
164599821266083 2014-03-22 10:31:05 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
164599821266083 2014-03-22 10:31:06 []
164599821266083 2014-03-22 10:31:06 Поиск Счетчика ['Поиск']
335386658089392 2014-03-18 04:28:17 []
335386658089392 2014-03-18 04:28:17 gen_time ['gen_time']
335386658089392 2014-03-18 04:28:18 Роль пользователя ['Роль пользователя']
335386658089392 2014-03-18 04:28:25 Поиск ['Поиск']
335386658089392 2014-03-18 04:28:26 []
335386658089392 2014-03-18 04:28:26 videoid ['videoid']
335386658089392 2014-03-18 04:28:26 videoid Done ['videoid']
335386658089392 2014-03-18 04:28:26 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
335386658089392 2014-03-18 04:28:31 []
335386658089392 2014-03-18 04:28:31 []
335386658089392 2014-03-18 05:48:43 []
335386658089392 2014-03-18 05:48:43 gen_time ['gen_time']
335386658089392 2014-03-18 05:48:43 Роль пользователя ['Роль пользователя']
335386658089392 2014-03-18 05:48:44 Дра ['Дра']
419794772979101 2014-03-17 22:08:42 []
419794772979101 2014-03-17 22:08:42 gen_time ['gen_time']
419794772979101 2014-03-17 22:08:43 Роль пользователя ['Роль пользователя']
419794772979101 2014-03-17 22:08:50 Поиск ['Поиск']
419794772979101 2014-03-17 22:08:50 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
419794772979101 2014-03-17 22:08:51 []
419794772979101 2014-03-19 14:26:49 []
419794772979101 2014-03-19 14:26:49 gen_time ['gen_time']
419794772979101 2014-03-19 14:26:49 Дра ['Дра']
419794772979101 2014-03-19 14:26:49 Эксперимент про Счетчик есть null ['Эксперимент про']
419794772979101 2014-03-19 14:26:52 []
419794772979101 2014-03-19 14:26:52 Поиск ['Поиск']
419794772979101 2014-03-19 14:26:52 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
818136115259082 2014-03-18 11:33:51 []
818136115259082 2014-03-18 11:33:51 gen_time ['gen_time']
818136115259082 2014-03-18 11:33:51 Дра ['Дра']
818136115259082 2014-03-18 11:33:52 Роль пользователя ['Роль пользователя']
818136115259082 2014-03-18 11:33:55 Поиск ['Поиск']
818136115259082 2014-03-18 11:33:55 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
818136115259082 2014-03-18 11:33:56 []
818136115259082 2014-03-18 11:34:42 gen_timestamp 564.815 ['gen_timestamp']
818136115259082 2014-03-18 11:34:44 []
818136115259082 2014-03-18 11:34:44 gen_timestamp 564.815 ['gen_timestamp']
870023587671569 2014-03-16 22:17:44 []
870023587671569 2014-03-16 22:17:44 []
870023587671569 2014-03-16 22:17:44 gen_time ['gen_time']
870023587671569 2014-03-16 22:17:44 gen_timestamp vkontakte,face element ['gen_timestamp']
870023587671569 2014-03-16 22:17:45 []
870023587671569 2014-03-16 22:17:46 []
1026297835113727 2014-03-17 18:41:53 []
1026297835113727 2014-03-18 02:37:28 []
1026297835113727 2014-03-19 18:26:53 []
1026297835113727 2014-03-20 05:37:49 []
1026297835113727 2014-03-20 05:49:38 []
1026297835113727 2014-03-20 13:39:34 []
1026297835113727 2014-03-21 07:38:39 []
1026297835113727 2014-03-22 07:13:26 []
1026297835113727 2014-03-22 10:50:25 []
1391426411567902 2014-03-18 10:39:27 []
1391426411567902 2014-03-18 10:39:27 gen_time ['gen_time']
1391426411567902 2014-03-18 10:39:27 gen_timestamp Счетчика нет (10 ['gen_timestamp']
1391426411567902 2014-03-18 10:39:27 Платность model ['Платность']
1391426411567902 2014-03-18 10:39:27 Эксперимент про Счетчик есть null ['Эксперимент про']
1391426411567902 2014-03-18 10:39:37 []
1391426411567902 2014-03-18 10:39:37 videoid ['videoid']
1391426411567902 2014-03-18 10:39:37 videoid Done ['videoid']
1391426411567902 2014-03-18 10:39:37 Поиск ['Поиск']
1391426411567902 2014-03-18 10:39:37 Поиск ['Поиск']
1391426411567902 2014-03-18 10:39:37 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент']
4 4 2
0 0 0
8 8 4
2 2 1
6 6 3
1 1 2
1 1 3
4 4 8
4 4 9
0 0 0
0 0 1
2 2 4
2 2 5
3 3 6
3 3 7

View File

@ -0,0 +1,204 @@
set enable_parallel_join=true;
SELECT
EventDate,
hits,
visits
FROM
(
SELECT
EventDate,
count() AS hits
FROM test.hits
GROUP BY EventDate
) ANY LEFT JOIN
(
SELECT
StartDate AS EventDate,
sum(Sign) AS visits
FROM test.visits
GROUP BY EventDate
) USING EventDate
ORDER BY hits DESC
LIMIT 10
SETTINGS joined_subquery_requires_alias = 0;
SELECT
EventDate,
count() AS hits,
any(visits)
FROM test.hits ANY LEFT JOIN
(
SELECT
StartDate AS EventDate,
sum(Sign) AS visits
FROM test.visits
GROUP BY EventDate
) USING EventDate
GROUP BY EventDate
ORDER BY hits DESC
LIMIT 10
SETTINGS joined_subquery_requires_alias = 0, enable_parallel_join=true;
SELECT
domain,
hits,
visits
FROM
(
SELECT
domain(URL) AS domain,
count() AS hits
FROM test.hits
GROUP BY domain
) ANY LEFT JOIN
(
SELECT
domain(StartURL) AS domain,
sum(Sign) AS visits
FROM test.visits
GROUP BY domain
) USING domain
ORDER BY hits DESC
LIMIT 10
SETTINGS joined_subquery_requires_alias = 0;
SELECT CounterID FROM test.visits ARRAY JOIN Goals.ID WHERE CounterID = 942285 ORDER BY CounterID;
SELECT
CounterID,
hits,
visits
FROM
(
SELECT
(CounterID % 100000) AS CounterID,
count() AS hits
FROM test.hits
GROUP BY CounterID
) ANY FULL OUTER JOIN
(
SELECT
(CounterID % 100000) AS CounterID,
sum(Sign) AS visits
FROM test.visits
GROUP BY CounterID
HAVING visits > 0
) USING CounterID
WHERE hits = 0 OR visits = 0
ORDER BY
hits + visits * 10 DESC,
CounterID ASC
LIMIT 20
SETTINGS any_join_distinct_right_table_keys = 1, joined_subquery_requires_alias = 0;
SELECT
CounterID,
hits,
visits
FROM
(
SELECT
(CounterID % 100000) AS CounterID,
count() AS hits
FROM test.hits
GROUP BY CounterID
) ANY LEFT JOIN
(
SELECT
(CounterID % 100000) AS CounterID,
sum(Sign) AS visits
FROM test.visits
GROUP BY CounterID
HAVING visits > 0
) USING CounterID
WHERE hits = 0 OR visits = 0
ORDER BY
hits + visits * 10 DESC,
CounterID ASC
LIMIT 20
SETTINGS any_join_distinct_right_table_keys = 1, joined_subquery_requires_alias = 0;
SELECT
CounterID,
hits,
visits
FROM
(
SELECT
(CounterID % 100000) AS CounterID,
count() AS hits
FROM test.hits
GROUP BY CounterID
) ANY RIGHT JOIN
(
SELECT
(CounterID % 100000) AS CounterID,
sum(Sign) AS visits
FROM test.visits
GROUP BY CounterID
HAVING visits > 0
) USING CounterID
WHERE hits = 0 OR visits = 0
ORDER BY
hits + visits * 10 DESC,
CounterID ASC
LIMIT 20
SETTINGS any_join_distinct_right_table_keys = 1, joined_subquery_requires_alias = 0;
SELECT
CounterID,
hits,
visits
FROM
(
SELECT
(CounterID % 100000) AS CounterID,
count() AS hits
FROM test.hits
GROUP BY CounterID
) ANY INNER JOIN
(
SELECT
(CounterID % 100000) AS CounterID,
sum(Sign) AS visits
FROM test.visits
GROUP BY CounterID
HAVING visits > 0
) USING CounterID
WHERE hits = 0 OR visits = 0
ORDER BY
hits + visits * 10 DESC,
CounterID ASC
LIMIT 20
SETTINGS any_join_distinct_right_table_keys = 1, joined_subquery_requires_alias = 0;
SELECT UserID, EventTime::DateTime('Asia/Dubai'), pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100;
SELECT UserID, EventTime::DateTime('Asia/Dubai'), pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits LEFT ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100;
SELECT a.*, b.* FROM
(
SELECT number AS k FROM system.numbers LIMIT 10
) AS a
ANY INNER JOIN
(
SELECT number * 2 AS k, number AS joined FROM system.numbers LIMIT 10
) AS b
USING k
SETTINGS any_join_distinct_right_table_keys = 1;
SELECT a.*, b.* FROM
(
SELECT number AS k FROM system.numbers LIMIT 10
) AS a
ALL INNER JOIN
(
SELECT intDiv(number, 2) AS k, number AS joined FROM system.numbers LIMIT 10
) AS b
USING k;